Add ability to provide multi-region (global) data consistency across HMS in different regions (#2542)
[global-hive-sync-tool] Add a global hive sync tool to sync hudi table across clusters. Add a way to rollback the replicated time stamp if we fail to sync or if we partly sync Co-authored-by: Jagmeet Bali <jsbali@uber.com>
This commit is contained in:
@@ -39,6 +39,14 @@ import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getTableMetaCl
|
||||
* InputPathHandler takes in a set of input paths and incremental tables list. Then, classifies the
|
||||
* input paths to incremental, snapshot paths and non-hoodie paths. This is then accessed later to
|
||||
* mutate the JobConf before processing incremental mode queries and snapshot queries.
|
||||
*
|
||||
* Note: We are adding jobConf of a mapreduce or spark job. The properties in the jobConf are two
|
||||
* type: session properties and table properties from metastore. While session property is common
|
||||
* for all the tables in a query the table properties are unique per table so there is no need to
|
||||
* check if it belongs to the table for which the path handler is now instantiated. The jobConf has
|
||||
* all table properties such as name, last modification time and so on which are unique to a table.
|
||||
* This class is written in such a way that it can handle multiple tables and properties unique to
|
||||
* a table but for table level property such check is not required.
|
||||
*/
|
||||
public class InputPathHandler {
|
||||
|
||||
@@ -63,7 +71,6 @@ public class InputPathHandler {
|
||||
/**
|
||||
* Takes in the original InputPaths and classifies each of them into incremental, snapshot and
|
||||
* non-hoodie InputPaths. The logic is as follows:
|
||||
*
|
||||
* 1. Check if an inputPath starts with the same basepath as any of the metadata basepaths we know
|
||||
* 1a. If yes, this belongs to a Hoodie table that we already know about. Simply classify this
|
||||
* as incremental or snapshot - We can get the table name of this inputPath from the
|
||||
|
||||
@@ -18,13 +18,14 @@
|
||||
|
||||
package org.apache.hudi.hadoop.utils;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -73,6 +74,7 @@ public class HoodieHiveUtils {
|
||||
public static final int MAX_COMMIT_ALL = -1;
|
||||
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
|
||||
public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode");
|
||||
public static final String GLOBALLY_CONSISTENT_READ_TIMESTAMP = "last_replication_timestamp";
|
||||
|
||||
public static boolean stopAtCompaction(JobContext job, String tableName) {
|
||||
String compactionPropName = String.format(HOODIE_STOP_AT_COMPACTION_PATTERN, tableName);
|
||||
|
||||
@@ -442,6 +442,7 @@ public class HoodieInputFormatUtils {
|
||||
}
|
||||
|
||||
HoodieTimeline timeline = HoodieHiveUtils.getTableTimeline(metaClient.getTableConfig().getTableName(), job, metaClient);
|
||||
|
||||
HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient ->
|
||||
FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, buildMetadataConfig(job), timeline));
|
||||
List<HoodieBaseFile> filteredBaseFiles = new ArrayList<>();
|
||||
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestGloballyConsistentTimeStampFilteringInputFormat
|
||||
extends TestHoodieParquetInputFormat {
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
super.setUp();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputFormatLoad() throws IOException {
|
||||
super.testInputFormatLoad();
|
||||
|
||||
// set filtering timestamp to 0 now the timeline wont have any commits.
|
||||
InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "0");
|
||||
|
||||
Assertions.assertThrows(HoodieIOException.class, () -> inputFormat.getSplits(jobConf, 10));
|
||||
Assertions.assertThrows(HoodieIOException.class, () -> inputFormat.listStatus(jobConf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputFormatUpdates() throws IOException {
|
||||
super.testInputFormatUpdates();
|
||||
|
||||
// set the globally replicated timestamp to 199 so only 100 is read and update is ignored.
|
||||
InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "100");
|
||||
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(10, files.length);
|
||||
|
||||
ensureFilesInCommit("5 files have been updated to commit 200. but should get filtered out ",
|
||||
files,"200", 0);
|
||||
ensureFilesInCommit("We should see 10 files from commit 100 ", files, "100", 10);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testIncrementalSimple() throws IOException {
|
||||
// setting filtering timestamp to zero should not in any way alter the result of the test which
|
||||
// pulls in zero files due to incremental ts being the actual commit time
|
||||
jobConf.set(HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP, "0");
|
||||
super.testIncrementalSimple();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testIncrementalWithMultipleCommits() throws IOException {
|
||||
super.testIncrementalWithMultipleCommits();
|
||||
|
||||
// set globally replicated timestamp to 400 so commits from 500, 600 does not show up
|
||||
InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "400");
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtils.MAX_COMMIT_ALL);
|
||||
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
|
||||
assertEquals(
|
||||
5, files.length,"Pulling ALL commits from 100, should get us the 3 files from 400 commit, 1 file from 300 "
|
||||
+ "commit and 1 file from 200 commit");
|
||||
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit",
|
||||
files, "400", 3);
|
||||
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit",
|
||||
files, "300", 1);
|
||||
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit",
|
||||
files, "200", 1);
|
||||
|
||||
List<String> commits = Arrays.asList("100", "200", "300", "400", "500", "600");
|
||||
for (int idx = 0; idx < commits.size(); ++idx) {
|
||||
for (int jdx = 0; jdx < commits.size(); ++jdx) {
|
||||
InputFormatTestUtil.setupIncremental(jobConf, commits.get(idx), HoodieHiveUtils.MAX_COMMIT_ALL);
|
||||
InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, commits.get(jdx));
|
||||
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
|
||||
if (jdx <= idx) {
|
||||
assertEquals(0, files.length,"all commits should be filtered");
|
||||
} else {
|
||||
// only commits upto the timestamp is allowed
|
||||
for (FileStatus file : files) {
|
||||
String commitTs = FSUtils.getCommitTime(file.getPath().getName());
|
||||
assertTrue(commits.indexOf(commitTs) <= jdx);
|
||||
assertTrue(commits.indexOf(commitTs) > idx);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -65,8 +65,8 @@ import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestHoodieParquetInputFormat {
|
||||
|
||||
private HoodieParquetInputFormat inputFormat;
|
||||
private JobConf jobConf;
|
||||
protected HoodieParquetInputFormat inputFormat;
|
||||
protected JobConf jobConf;
|
||||
private final HoodieFileFormat baseFileFormat = HoodieFileFormat.PARQUET;
|
||||
private final String baseFileExtension = baseFileFormat.getFileExtension();
|
||||
|
||||
|
||||
@@ -23,10 +23,12 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
@@ -169,6 +171,21 @@ public class TestInputPathHandler {
|
||||
assertTrue(actualComparesToExpected(actualPaths, nonHoodiePaths));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputPathHandlerWithGloballyReplicatedTimeStamp() throws IOException {
|
||||
JobConf jobConf = new JobConf();
|
||||
jobConf.set(HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP, "1");
|
||||
inputPathHandler = new InputPathHandler(dfs.getConf(), inputPaths.toArray(
|
||||
new Path[inputPaths.size()]), incrementalTables);
|
||||
List<Path> actualPaths = inputPathHandler.getGroupedIncrementalPaths().values().stream()
|
||||
.flatMap(List::stream).collect(Collectors.toList());
|
||||
assertTrue(actualComparesToExpected(actualPaths, incrementalPaths));
|
||||
actualPaths = inputPathHandler.getSnapshotPaths();
|
||||
assertTrue(actualComparesToExpected(actualPaths, snapshotPaths));
|
||||
actualPaths = inputPathHandler.getNonHoodieInputPaths();
|
||||
assertTrue(actualComparesToExpected(actualPaths, nonHoodiePaths));
|
||||
}
|
||||
|
||||
private boolean actualComparesToExpected(List<Path> actualPaths, List<Path> expectedPaths) {
|
||||
if (actualPaths.size() != expectedPaths.size()) {
|
||||
return false;
|
||||
|
||||
Reference in New Issue
Block a user