1
0

[HUDI-2757] Implement Hudi AWS Glue sync (#5076)

This commit is contained in:
Raymond Xu
2022-03-28 11:54:59 -07:00
committed by GitHub
parent 4ed84b216d
commit 6ccbae4d2a
25 changed files with 1151 additions and 204 deletions

View File

@@ -138,12 +138,12 @@ public class TestHiveSyncTool {
HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size() + 1,
@@ -176,9 +176,9 @@ public class TestHiveSyncTool {
ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME
+ "` PARTITION (`datestr`='2050-01-01') SET LOCATION '/some/new/location'");
List<Partition> hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<org.apache.hudi.sync.common.model.Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty());
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
assertEquals(PartitionEventType.UPDATE, partitionEvents.iterator().next().eventType,
"The one partition event must of type UPDATE");
@@ -211,20 +211,20 @@ public class TestHiveSyncTool {
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true");
reinitHiveSyncClient();
assertDoesNotThrow((this::reSyncHiveTable));
assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME),
assertTrue(hiveClient.databaseExists(HiveTestUtil.DB_NAME),
"DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes");
// while autoCreateDatabase is false and database exists;
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false");
reinitHiveSyncClient();
assertDoesNotThrow((this::reSyncHiveTable));
assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME),
assertTrue(hiveClient.databaseExists(HiveTestUtil.DB_NAME),
"DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes");
// while autoCreateDatabase is true and database exists;
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true");
assertDoesNotThrow((this::reSyncHiveTable));
assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME),
assertTrue(hiveClient.databaseExists(HiveTestUtil.DB_NAME),
"DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes");
}
@@ -457,8 +457,8 @@ public class TestHiveSyncTool {
reSyncHiveTable();
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1));
assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit");
List<Partition> hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince);
List<org.apache.hudi.sync.common.model.Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD");
@@ -581,11 +581,11 @@ public class TestHiveSyncTool {
String roTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE;
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(roTableName), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
assertFalse(hiveClient.tableExists(roTableName), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(roTableName), "Table " + roTableName + " should exist after sync completes");
assertTrue(hiveClient.tableExists(roTableName), "Table " + roTableName + " should exist after sync completes");
if (useSchemaFromCommitMetadata) {
assertEquals(hiveClient.getTableSchema(roTableName).size(),
@@ -643,14 +643,14 @@ public class TestHiveSyncTool {
String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE;
HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(snapshotTableName),
assertFalse(hiveClient.tableExists(snapshotTableName),
"Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
+ " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(snapshotTableName),
assertTrue(hiveClient.tableExists(snapshotTableName),
"Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
+ " should exist after sync completes");
@@ -713,11 +713,11 @@ public class TestHiveSyncTool {
HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size() + 3,
@@ -736,8 +736,8 @@ public class TestHiveSyncTool {
reinitHiveSyncClient();
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(instantTime));
assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit");
List<Partition> hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince);
List<org.apache.hudi.sync.common.model.Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD");
@@ -755,7 +755,7 @@ public class TestHiveSyncTool {
reinitHiveSyncClient();
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size() + 3,
@@ -776,12 +776,12 @@ public class TestHiveSyncTool {
HiveTestUtil.createCOWTable(instantTime, 1, true);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size() + 1,
@@ -820,11 +820,11 @@ public class TestHiveSyncTool {
HiveTestUtil.createCOWTable(instantTime, 1, true);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size() + 1,
@@ -860,11 +860,11 @@ public class TestHiveSyncTool {
HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size(),
@@ -882,13 +882,13 @@ public class TestHiveSyncTool {
HiveTestUtil.createMORTable(commitTime, "", 5, false, true);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
assertFalse(hiveClient.tableExists(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
+ " should not exist initially");
// Lets do the sync
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
assertTrue(hiveClient.tableExists(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
+ " should exist after sync completes");
// Schema being read from compacted base files
@@ -925,7 +925,7 @@ public class TestHiveSyncTool {
HiveTestUtil.createCOWTable(instantTime, 5, false);
reinitHiveSyncClient();
HoodieHiveClient prevHiveClient = hiveClient;
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
// Lets do the sync
@@ -936,12 +936,12 @@ public class TestHiveSyncTool {
reSyncHiveTable();
assertNull(hiveClient);
assertFalse(prevHiveClient.doesTableExist(HiveTestUtil.TABLE_NAME),
assertFalse(prevHiveClient.tableExists(HiveTestUtil.TABLE_NAME),
"Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
}
private void verifyOldParquetFileTest(HoodieHiveClient hiveClient, String emptyCommitTime) throws Exception {
assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes");
assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(),
hiveClient.getDataSchema().getColumns().size() + 1,
"Hive Schema should match the table schema + partition field");
@@ -973,7 +973,7 @@ public class TestHiveSyncTool {
final String emptyCommitTime = "200";
HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true);
reinitHiveSyncClient();
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
reinitHiveSyncClient();
reSyncHiveTable();
@@ -1000,7 +1000,7 @@ public class TestHiveSyncTool {
reinitHiveSyncClient();
assertFalse(
hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, getHiveConf(), fileSystem);
// now delete the evolved commit instant
@@ -1017,7 +1017,7 @@ public class TestHiveSyncTool {
}
// table should not be synced yet
assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist at all");
assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist at all");
}
@ParameterizedTest
@@ -1033,7 +1033,7 @@ public class TestHiveSyncTool {
//HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime);
reinitHiveSyncClient();
assertFalse(
hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially");
reSyncHiveTable();
@@ -1120,7 +1120,7 @@ public class TestHiveSyncTool {
reinitHiveSyncClient();
reSyncHiveTable();
assertTrue(hiveClient.doesTableExist(tableName));
assertTrue(hiveClient.tableExists(tableName));
assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get());
HiveTestUtil.addMORPartitions(0, true, true, true, ZonedDateTime.now().plusDays(2), commitTime1, commitTime2);
@@ -1138,7 +1138,7 @@ public class TestHiveSyncTool {
private void reinitHiveSyncClient() {
hiveSyncTool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem);
hiveClient = hiveSyncTool.hoodieHiveClient;
hiveClient = (HoodieHiveClient) hiveSyncTool.hoodieHiveClient;
}
private int getPartitionFieldSize() {