diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java b/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java index 3df40bec2b477..6b33b1be44c11 100644 --- a/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java @@ -90,7 +90,7 @@ public void setUp() throws Exception { fileSystem = hiveSyncConfig.getHadoopFileSystem(); fileSystem.mkdirs(new Path(tablePath)); StorageConfiguration configuration = HadoopFSUtils.getStorageConf(new Configuration()); - HoodieTableMetaClient metaClient = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java b/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java index 55c07f07e7875..0ba4a0496f9e3 100644 --- a/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java @@ -104,7 +104,7 @@ public static void createHoodieTable() throws IOException { if (fileSystem.exists(path)) { fileSystem.delete(path, true); } - metaClient = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index bb63a051d9b9e..497470ab36f3b 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -267,10 +267,10 @@ protected static void clean(JavaSparkContext jsc, String basePath, String propsF protected static int deleteMarker(JavaSparkContext jsc, String instantTime, String basePath) { try (SparkRDDWriteClient client = createHoodieClient(jsc, basePath, false)) { - HoodieWriteConfig config = client.getConfig(); HoodieEngineContext context = client.getEngineContext(); HoodieSparkTable table = HoodieSparkTable.create(config, context); + client.validateAgainstTableProperties(table.getMetaClient().getTableConfig(), config); WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); return 0; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java index 773e93c6ca25e..211d401325311 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java @@ -27,6 +27,7 @@ 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.table.HoodieTableVersion; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -112,6 +113,16 @@ public String connect( return "Metadata for table " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " loaded"; } + public String createTable( + final String path, + final String name, + final String tableTypeStr, + String archiveFolder, + Integer layoutVersion, + final String payloadClass) throws IOException { + return createTable(path, name, tableTypeStr, archiveFolder, layoutVersion, HoodieTableVersion.current().versionCode(), payloadClass); + } + /** * Create a Hoodie Table if it does not exist. * @@ -130,6 +141,8 @@ public String createTable( defaultValue = ShellOption.NULL) String archiveFolder, @ShellOption(value = {"--layoutVersion"}, help = "Specific Layout Version to use", defaultValue = ShellOption.NULL) Integer layoutVersion, + @ShellOption(value = {"--tableVersion"}, help = "Specific table Version to create table as", + defaultValue = ShellOption.NULL) Integer tableVersion, @ShellOption(value = {"--payloadClass"}, defaultValue = "org.apache.hudi.common.model.HoodieAvroPayload", help = "Payload Class") final String payloadClass) throws IOException { @@ -149,12 +162,13 @@ public String createTable( throw new IllegalStateException("Table already existing in path : " + path); } - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(tableTypeStr) .setTableName(name) .setArchiveLogFolder(archiveFolder) .setPayloadClassName(payloadClass) .setTimelineLayoutVersion(layoutVersion) + .setTableVersion(tableVersion == null ? HoodieTableVersion.current().versionCode() : tableVersion) .initTable(HoodieCLI.conf.newInstance(), path); // Now connect to ensure loading works return connect(path, layoutVersion, false, 0, 0, 0, diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java index e121ff808b2cf..e52e7c0f89b94 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/UpgradeOrDowngradeCommand.java @@ -89,7 +89,7 @@ static String getHoodieTableVersionName(String versionOption, boolean overrideWi try { int versionCode = Integer.parseInt(versionOption); - return HoodieTableVersion.versionFromCode(versionCode).name(); + return HoodieTableVersion.fromVersionCode(versionCode).name(); } catch (NumberFormatException e) { // The version option from the CLI is not a number, returns the original String return versionOption; diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java index c089ef2a397d0..573cb33ee3a1a 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java @@ -68,7 +68,7 @@ public void init() throws IOException { @Test public void testMetadataDelete() throws Exception { - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE.name()) .setTableName(tableName()) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java index 493584f26062f..e4e8da5dd4f29 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java @@ -318,7 +318,7 @@ public void testShowFailedCommits() { @Test public void testRepairDeprecatedPartition() throws IOException { tablePath = tablePath + "/repair_test/"; - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE.name()) .setTableName(tableName()) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) @@ -386,7 +386,7 @@ public void testRepairDeprecatedPartition() throws IOException { @Test public void testRenamePartition() throws IOException { tablePath = tablePath + "/rename_partition_test/"; - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE.name()) .setTableName(tableName()) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java index 2b53ed349b30a..b968d2c1424dc 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java @@ -29,6 +29,7 @@ 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.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimeGeneratorType; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -158,13 +159,14 @@ public void testDefaultCreate() { public void testCreateWithSpecifiedValues() { // Test create with specified values Object result = shell.evaluate(() -> "create --path " + tablePath + " --tableName " + tableName - + " --tableType MERGE_ON_READ --archiveLogFolder archive"); + + " --tableType MERGE_ON_READ --archiveLogFolder archive --tableVersion 6"); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); assertEquals("Metadata for table " + tableName + " loaded", result.toString()); HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); assertEquals(metaPath + StoragePath.SEPARATOR + "archive", client.getArchivePath()); assertEquals(tablePath, client.getBasePath().toString()); assertEquals(metaPath, client.getMetaPath().toString()); + assertEquals(HoodieTableVersion.SIX, client.getTableConfig().getTableVersion()); assertEquals(HoodieTableType.MERGE_ON_READ, client.getTableType()); } @@ -237,7 +239,8 @@ public void testFetchTableSchema() throws Exception { HoodieCLI.conf = storageConf(); new TableCommand().createTable( tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); String schemaStr = "{\n" + " \"type\" : \"record\",\n" diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java index ce44debb50699..d356da22fe686 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestClusteringCommand.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -77,7 +78,8 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( basePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); initMetaClient(); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java index c74679432f0d2..80d9bc9fa4a53 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java @@ -24,6 +24,7 @@ import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestTable; @@ -76,7 +77,8 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( basePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); initMetaClient(); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java index 409a828a95675..d15d444c76bf7 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCompactionCommand.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -83,7 +84,8 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( basePath, tableName, HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); initMetaClient(); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java index 25dd3c2152cde..d03ce6a0e2abb 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java @@ -23,6 +23,7 @@ import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.storage.StoragePath; @@ -59,7 +60,8 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); } /** diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java index 55e7989f0790d..ee223321e8222 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -101,7 +102,8 @@ public void init() throws Exception { // Create cow table and connect new TableCommand().createTable( cowTablePath, "cow_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieSparkWriteableTestTable cowTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); @@ -124,7 +126,8 @@ public void init() throws Exception { // Create mor table and connect new TableCommand().createTable( morTablePath, "mor_table", HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieSparkWriteableTestTable morTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); morTable.addDeltaCommit("20160401010101"); @@ -146,7 +149,8 @@ public void init() throws Exception { // Create cow table and connect new TableCommand().createTable( cowNonPartitionedTablePath, "cow_table_non_partitioned", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieSparkWriteableTestTable cowNonPartitionedTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java index ced1cf7a3ef00..4912db60d3d63 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java @@ -25,6 +25,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -71,7 +72,8 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "org.apache.hudi.common.model.HoodieAvroPayload"); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 4599de494117b..368879c089ef7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -65,6 +65,7 @@ import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.table.action.rollback.RollbackUtils; import org.apache.hudi.table.marker.WriteMarkersFactory; +import org.apache.hudi.util.CommonClientUtils; import com.codahale.metrics.Timer; import org.slf4j.Logger; @@ -79,6 +80,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -675,7 +677,22 @@ protected Option scheduleTableServiceInternal(String instantTime, Option return option; } - protected abstract HoodieTable createTable(HoodieWriteConfig config, StorageConfiguration storageConf); + protected HoodieTable createTableAndValidate(HoodieWriteConfig config, + BiFunction createTableFn, + boolean skipValidation) { + HoodieTable table = createTableFn.apply(config, context); + if (!skipValidation) { + CommonClientUtils.validateTableVersion(table.getMetaClient().getTableConfig(), config); + } + return table; + } + + protected HoodieTable createTable(HoodieWriteConfig config, StorageConfiguration storageConf) { + return createTable(config, storageConf, false); + } + + protected abstract HoodieTable createTable(HoodieWriteConfig config, StorageConfiguration storageConf, boolean skipValidation); /** * Executes a clustering plan on a table, serially before or after an insert/upsert action. @@ -942,6 +959,10 @@ protected void rollbackFailedWrites(Map> instantsToRollback, boolean skipLocking) { + rollbackFailedWrites(instantsToRollback, skipLocking, false); + } + + protected void rollbackFailedWrites(Map> instantsToRollback, boolean skipLocking, boolean skipVersionCheck) { // sort in reverse order of commit times LinkedHashMap> reverseSortedRollbackInstants = instantsToRollback.entrySet() .stream().sorted((i1, i2) -> i2.getKey().compareTo(i1.getKey())) @@ -956,7 +977,7 @@ protected void rollbackFailedWrites(Map getInstantsToRollbackForLazyCleanPolicy(HoodieTableMetaClie * will be removed in future in favor of {@link BaseHoodieWriteClient#restoreToInstant(String, boolean) */ @Deprecated - public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException { + public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, + boolean skipLocking, boolean skipVersionCheck) throws HoodieRollbackException { final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp()) .orElseGet(() -> createNewInstantTime(!skipLocking)); - return rollback(commitInstantTime, pendingRollbackInfo, rollbackInstantTime, skipLocking); + return rollback(commitInstantTime, pendingRollbackInfo, rollbackInstantTime, skipLocking, skipVersionCheck); } /** @@ -1047,11 +1069,11 @@ public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, String rollbackInstantTime, - boolean skipLocking) throws HoodieRollbackException { + boolean skipLocking, boolean skipVersionCheck) throws HoodieRollbackException { LOG.info("Begin rollback of instant " + commitInstantTime); final Timer.Context timerContext = this.metrics.getRollbackCtx(); try { - HoodieTable table = createTable(config, storageConf); + HoodieTable table = createTable(config, storageConf, skipVersionCheck); Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstantsAsStream() .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) .findFirst()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 4f3a2fa02e272..e250a8d4f7656 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -33,15 +33,18 @@ import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.config.HoodieCommonConfig; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.TableSchemaResolver; @@ -74,6 +77,7 @@ import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils; import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; import org.apache.hudi.internal.schema.utils.SerDeHelper; +import org.apache.hudi.keygen.constant.KeyGeneratorType; import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.MetadataPartitionType; @@ -86,6 +90,7 @@ import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; import org.apache.hudi.table.upgrade.UpgradeDowngrade; +import org.apache.hudi.util.CommonClientUtils; import com.codahale.metrics.Timer; import org.apache.avro.Schema; @@ -99,8 +104,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.function.BiConsumer; +import java.util.function.BiFunction; import java.util.stream.Collectors; import static org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName; @@ -320,6 +327,27 @@ private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCom } } + protected HoodieTable createTableAndValidate(HoodieWriteConfig writeConfig, + BiFunction createTableFn) { + HoodieTable table = createTableFn.apply(writeConfig, context); + CommonClientUtils.validateTableVersion(table.getMetaClient().getTableConfig(), writeConfig); + return table; + } + + @FunctionalInterface + protected interface TriFunction { + R apply(T t, U u, V v); + } + + protected HoodieTable createTableAndValidate(HoodieWriteConfig writeConfig, + HoodieTableMetaClient metaClient, + TriFunction createTableFn) { + HoodieTable table = createTableFn.apply(writeConfig, context, metaClient); + CommonClientUtils.validateTableVersion(table.getMetaClient().getTableConfig(), writeConfig); + return table; + } + protected abstract HoodieTable createTable(HoodieWriteConfig config); protected abstract HoodieTable createTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient); @@ -742,14 +770,14 @@ public void restoreToSavepoint(String savepointTime) { public boolean rollback(final String commitInstantTime) throws HoodieRollbackException { HoodieTable table = initTable(WriteOperationType.UNKNOWN, Option.empty()); Option pendingRollbackInfo = tableServiceClient.getPendingRollbackInfo(table.getMetaClient(), commitInstantTime); - return tableServiceClient.rollback(commitInstantTime, pendingRollbackInfo, false); + return tableServiceClient.rollback(commitInstantTime, pendingRollbackInfo, false, false); } @Deprecated public boolean rollback(final String commitInstantTime, String rollbackInstantTimestamp) throws HoodieRollbackException { HoodieTable table = initTable(WriteOperationType.UNKNOWN, Option.empty()); Option pendingRollbackInfo = tableServiceClient.getPendingRollbackInfo(table.getMetaClient(), commitInstantTime); - return tableServiceClient.rollback(commitInstantTime, pendingRollbackInfo, rollbackInstantTimestamp, false); + return tableServiceClient.rollback(commitInstantTime, pendingRollbackInfo, rollbackInstantTimestamp, false, false); } /** @@ -881,6 +909,10 @@ public String startCommit() { * Provides a new commit time for a write operation (insert/update/delete/insert_overwrite/insert_overwrite_table) with specified action. */ public String startCommit(String actionType, HoodieTableMetaClient metaClient) { + if (needsUpgradeOrDowngrade(metaClient)) { + executeUsingTxnManager(Option.empty(), () -> tryUpgrade(metaClient, Option.empty())); + } + CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), HoodieTimeline.COMMIT_ACTION, () -> tableServiceClient.rollbackFailedWrites()); @@ -907,22 +939,26 @@ public void startCommitWithTime(String instantTime, String actionType) { } /** - * Completes a new commit time for a write operation (insert/update/delete) with specified action. + * Starts a new commit time for a write operation (insert/update/delete) with specified action. */ private void startCommitWithTime(String instantTime, String actionType, HoodieTableMetaClient metaClient) { + if (needsUpgradeOrDowngrade(metaClient)) { + // unclear what instant to use, since upgrade does have a given instant. + executeUsingTxnManager(Option.empty(), () -> tryUpgrade(metaClient, Option.empty())); + } CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), HoodieTimeline.COMMIT_ACTION, () -> tableServiceClient.rollbackFailedWrites()); startCommit(instantTime, actionType, metaClient); } private void startCommit(String instantTime, String actionType, HoodieTableMetaClient metaClient) { - LOG.info("Generate a new instant time: " + instantTime + " action: " + actionType); + LOG.info("Generate a new instant time: {} action: {}", instantTime, actionType); // check there are no inflight restore before starting a new commit. HoodieTimeline inflightRestoreTimeline = metaClient.getActiveTimeline().getRestoreTimeline().filterInflightsAndRequested(); ValidationUtils.checkArgument(inflightRestoreTimeline.countInstants() == 0, "Found pending restore in active timeline. Please complete the restore fully before proceeding. As of now, " + "table could be in an inconsistent state. Pending restores: " + Arrays.toString(inflightRestoreTimeline.getInstantsAsStream() - .map(instant -> instant.getTimestamp()).collect(Collectors.toList()).toArray())); + .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).toArray())); if (config.getFailedWritesCleanPolicy().isLazy()) { this.heartbeatClient.start(instantTime); @@ -1224,10 +1260,17 @@ protected void doInitTable(WriteOperationType operationType, HoodieTableMetaClie if (instantTime.isPresent()) { ownerInstant = Option.of(new HoodieInstant(true, CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), instantTime.get())); } - this.txnManager.beginTransaction(ownerInstant, Option.empty()); - try { + executeUsingTxnManager(ownerInstant, () -> { tryUpgrade(metaClient, instantTime); + // TODO: this also does MT table management.. initMetadataTable(instantTime, metaClient); + }); + } + + private void executeUsingTxnManager(Option ownerInstant, Runnable r) { + this.txnManager.beginTransaction(ownerInstant, Option.empty()); + try { + r.run(); } finally { this.txnManager.endTransaction(ownerInstant); } @@ -1271,7 +1314,7 @@ public final HoodieTable initTable(WriteOperationType operationType, Option insta if (!instantsToRollback.isEmpty()) { Map> pendingRollbacks = tableServiceClient.getPendingRollbackInfos(metaClient); instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); - tableServiceClient.rollbackFailedWrites(pendingRollbacks, true); + tableServiceClient.rollbackFailedWrites(pendingRollbacks, true, true); } new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper) @@ -1375,6 +1454,11 @@ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option insta } } + private boolean needsUpgradeOrDowngrade(HoodieTableMetaClient metaClient) { + UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper); + return upgradeDowngrade.needsUpgradeOrDowngrade(config.getWriteVersion()); + } + /** * Rolls back the failed delta commits corresponding to the indexing action. *

diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 68efe732b9ae3..09df47b44521c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -50,6 +50,7 @@ import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -135,6 +136,23 @@ public class HoodieWriteConfig extends HoodieConfig { .noDefaultValue() .withDocumentation("Table name that will be used for registering with metastores like HMS. Needs to be same across runs."); + public static final ConfigProperty WRITE_TABLE_VERSION = ConfigProperty + .key("hoodie.write.table.version") + .defaultValue(HoodieTableVersion.current().versionCode()) + .withValidValues( + String.valueOf(HoodieTableVersion.SIX.versionCode()), + String.valueOf(HoodieTableVersion.current().versionCode()) + ) + .sinceVersion("1.0.0") + .withDocumentation("The table version this writer is storing the table in. This should match the current table version."); + + public static final ConfigProperty AUTO_UPGRADE_VERSION = ConfigProperty + .key("hoodie.write.auto.upgrade") + .defaultValue(true) + .sinceVersion("1.0.0") + .withDocumentation("If enabled, writers automatically migrate the table to the specified write table version " + + "if the current table version is lower."); + public static final ConfigProperty TAGGED_RECORD_STORAGE_LEVEL_VALUE = ConfigProperty .key("hoodie.write.tagged.record.storage.level") .defaultValue("MEMORY_AND_DISK_SER") @@ -1258,6 +1276,18 @@ public String getWriteSchema() { return getSchema(); } + public HoodieTableVersion getWriteVersion() { + Integer versionCode = getInt(WRITE_TABLE_VERSION); + if (versionCode != null) { + WRITE_TABLE_VERSION.checkValues(versionCode.toString()); + } + return HoodieTableVersion.fromVersionCode(getIntOrDefault(WRITE_TABLE_VERSION)); + } + + public boolean autoUpgrade() { + return getBoolean(AUTO_UPGRADE_VERSION); + } + public String getTaggedRecordStorageLevel() { return getString(TAGGED_RECORD_STORAGE_LEVEL_VALUE); } @@ -2837,6 +2867,16 @@ public Builder withSchemaEvolutionEnable(boolean enable) { return this; } + public Builder withWriteTableVersion(int writeVersion) { + writeConfig.setValue(WRITE_TABLE_VERSION, String.valueOf(HoodieTableVersion.fromVersionCode(writeVersion).versionCode())); + return this; + } + + public Builder withAutoUpgradeVersion(boolean enable) { + writeConfig.setValue(AUTO_UPGRADE_VERSION, String.valueOf(enable)); + return this; + } + public Builder withAvroSchemaValidate(boolean enable) { writeConfig.setValue(AVRO_SCHEMA_VALIDATE_ENABLE, String.valueOf(enable)); return this; @@ -3382,6 +3422,11 @@ private void autoAdjustConfigsForConcurrencyMode(boolean isLockProviderPropertyS } private void validate() { + if (HoodieTableVersion.SIX.equals(writeConfig.getWriteVersion())) { + LOG.warn("HoodieTableVersion.SIX is not yet fully supported by the writer. " + + "Please expect some unexpected behavior, until its fully implemented."); + } + String layoutVersion = writeConfig.getString(TIMELINE_LAYOUT_VERSION_NUM); // Ensure Layout Version is good new TimelineLayoutVersion(Integer.parseInt(layoutVersion)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java index 97796bb71d8e4..0c6345a3ce291 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java @@ -20,8 +20,8 @@ public class HoodieUpgradeDowngradeException extends HoodieException { - public HoodieUpgradeDowngradeException(String msg, Throwable t) { - super(msg, t); + public HoodieUpgradeDowngradeException(String msg) { + super(msg); } public HoodieUpgradeDowngradeException(int fromVersion, int toVersion, boolean upgrade) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index e9657e7145daf..65dd38db09bea 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -185,9 +185,7 @@ protected HoodieBackedTableMetadataWriter(StorageConfiguration storageConf, ValidationUtils.checkArgument(!initialized || this.metadata != null, "MDT Reader should have been opened post initialization"); } - protected HoodieTable getHoodieTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) { - return null; - } + abstract HoodieTable getTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient); private void initMetadataReader() { if (this.metadata != null) { @@ -591,7 +589,7 @@ private List> getPartitionFileSlicePairs() throws IOExce private Pair> initializeRecordIndexPartition() throws IOException { final HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(dataMetaClient, dataMetaClient.getActiveTimeline(), metadata); - final HoodieTable hoodieTable = getHoodieTable(dataWriteConfig, dataMetaClient); + final HoodieTable hoodieTable = getTable(dataWriteConfig, dataMetaClient); // Collect the list of latest base files present in each partition List partitions = metadata.getAllPartitionPaths(); @@ -730,9 +728,11 @@ private Set getPendingDataInstants(HoodieTableMetaClient dataMetaClient) } private HoodieTableMetaClient initializeMetaClient() throws IOException { - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) .setTableName(dataWriteConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX) + // MT version should match DT, such that same readers can read both. + .setTableVersion(dataWriteConfig.getWriteVersion()) .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName(HoodieMetadataPayload.class.getName()) .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java index 1143220e34d6f..b1e83bd5eac10 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java @@ -106,6 +106,7 @@ public static HoodieWriteConfig createMetadataWriteConfig( HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() .withEngineType(writeConfig.getEngineType()) .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withWriteTableVersion(writeConfig.getWriteVersion().versionCode()) .withMergeAllowDuplicateOnInserts(false) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() .withConsistencyCheckEnabled(writeConfig.getConsistencyGuardConfig().isConsistencyCheckEnabled()) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 89b58a9f116e6..60834fa3013ba 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -820,7 +820,7 @@ private void waitForAllFiles(HoodieEngineContext context, Map> partitionFilePaths, FileVisibility visibility) { - final HoodieStorage storage = metaClient.getRawHoodieStorage(); + final HoodieStorage storage = metaClient.getRawStorage(); List fileList = partitionFilePaths.map(Pair::getValue).collect(Collectors.toList()); try { getConsistencyGuard(storage, config.getConsistencyGuardConfig()) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java index df5135b162bc1..f4f176db14230 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; @@ -92,13 +93,12 @@ public Map downgrade(HoodieWriteConfig config, HoodieEng } downgradePartitionFields(config, context, upgradeDowngradeHelper, tablePropsToAdd); - // Prepare parameters. if (metaClient.getTableConfig().isMetadataTableAvailable()) { // Delete unsupported metadata partitions in table version 7. downgradeMetadataPartitions(context, metaClient.getStorage(), metaClient, tablePropsToAdd); + UpgradeDowngradeUtils.updateMetadataTableVersion(context, HoodieTableVersion.SEVEN, metaClient); } - return tablePropsToAdd; } @@ -124,19 +124,16 @@ static void downgradeMetadataPartitions(HoodieEngineContext context, HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()); // Fetch metadata partition paths. - List metadataPartitions = FSUtils.getAllPartitionPaths( - context, + List metadataPartitions = FSUtils.getAllPartitionPaths(context, hoodieStorage, metadataTableBasePath, false); // Delete partitions. - List validPartitionPaths = - deleteMetadataPartition(context, metaClient, metadataPartitions); + List validPartitionPaths = deleteMetadataPartition(context, metaClient, metadataPartitions); // Clean the configuration. - tablePropsToAdd.put( - TABLE_METADATA_PARTITIONS, String.join(",", validPartitionPaths)); + tablePropsToAdd.put(TABLE_METADATA_PARTITIONS, String.join(",", validPartitionPaths)); } static List deleteMetadataPartition(HoodieEngineContext context, @@ -160,7 +157,6 @@ private static Set getSupportedMetadataPartitionPaths() { supportedPartitionPaths.add(COLUMN_STATS.getPartitionPath()); supportedPartitionPaths.add(FILES.getPartitionPath()); supportedPartitionPaths.add(RECORD_INDEX.getPartitionPath()); - return supportedPartitionPaths; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToSixDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToSixDowngradeHandler.java index 5cf9b42bab21f..7a5280a57a32e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToSixDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToSixDowngradeHandler.java @@ -20,7 +20,9 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; import java.util.Collections; import java.util.Map; @@ -35,6 +37,8 @@ public class SevenToSixDowngradeHandler implements DowngradeHandler { @Override public Map downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, SupportsUpgradeDowngrade upgradeDowngradeHelper) { + final HoodieTable table = upgradeDowngradeHelper.getTable(config, context); + UpgradeDowngradeUtils.updateMetadataTableVersion(context, HoodieTableVersion.SIX, table.getMetaClient()); return Collections.emptyMap(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java index 2d4e8abf89c52..20b351ff6be6f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java @@ -62,10 +62,16 @@ public UpgradeDowngrade( this.upgradeDowngradeHelper = upgradeDowngradeHelper; } - public boolean needsUpgradeOrDowngrade(HoodieTableVersion toVersion) { - HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion(); - // Ensure versions are same - return toVersion.versionCode() != fromVersion.versionCode(); + public boolean needsUpgradeOrDowngrade(HoodieTableVersion toWriteVersion) { + HoodieTableVersion fromTableVersion = metaClient.getTableConfig().getTableVersion(); + + if (!config.autoUpgrade() && fromTableVersion.versionCode() < toWriteVersion.versionCode()) { + throw new HoodieUpgradeDowngradeException(String.format("Table version mismatch. " + + "Please upgrade table from version %s to %s. ", fromTableVersion, toWriteVersion)); + } + + // allow upgrades/downgrades otherwise. + return toWriteVersion.versionCode() != fromTableVersion.versionCode(); } /** @@ -136,14 +142,14 @@ public void run(HoodieTableVersion toVersion, String instantTime) { if (fromVersion.versionCode() < toVersion.versionCode()) { // upgrade while (fromVersion.versionCode() < toVersion.versionCode()) { - HoodieTableVersion nextVersion = HoodieTableVersion.versionFromCode(fromVersion.versionCode() + 1); + HoodieTableVersion nextVersion = HoodieTableVersion.fromVersionCode(fromVersion.versionCode() + 1); tableProps.putAll(upgrade(fromVersion, nextVersion, instantTime)); fromVersion = nextVersion; } } else { // downgrade while (fromVersion.versionCode() > toVersion.versionCode()) { - HoodieTableVersion prevVersion = HoodieTableVersion.versionFromCode(fromVersion.versionCode() - 1); + HoodieTableVersion prevVersion = HoodieTableVersion.fromVersionCode(fromVersion.versionCode() - 1); tableProps.putAll(downgrade(fromVersion, prevVersion, instantTime)); fromVersion = prevVersion; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java index b6e4db6e8e6ea..1b5cbe591387c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java @@ -22,7 +22,9 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; 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.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; @@ -89,4 +91,20 @@ public static void syncCompactionRequestedFileToAuxiliaryFolder(HoodieTable tabl } }); } + + static void updateMetadataTableVersion(HoodieEngineContext context, HoodieTableVersion toVersion, HoodieTableMetaClient dataMetaClient) throws HoodieIOException { + try { + StoragePath metadataBasePath = new StoragePath(dataMetaClient.getBasePath(), HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); + if (dataMetaClient.getStorage().exists(metadataBasePath)) { + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(context.getStorageConf().newInstance()) + .setBasePath(metadataBasePath) + .build(); + metaClient.getTableConfig().setTableVersion(toVersion); + HoodieTableConfig.update(metaClient.getStorage(), metaClient.getMetaPath(), metaClient.getTableConfig().getProps()); + } + } catch (IOException e) { + throw new HoodieIOException("Error while updating metadata table version", e); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/CommonClientUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/CommonClientUtils.java new file mode 100644 index 0000000000000..320839af55bfc --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/CommonClientUtils.java @@ -0,0 +1,35 @@ +/* + * + * * 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.util; + +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; + +public class CommonClientUtils { + public static void validateTableVersion(HoodieTableConfig tableConfig, HoodieWriteConfig writeConfig) { + // mismatch of table versions. + if (!tableConfig.getTableVersion().equals(writeConfig.getWriteVersion())) { + throw new HoodieNotSupportedException(String.format("Table version (%s) and Writer version (%s) do not match.", + tableConfig.getTableVersion(), writeConfig.getWriteVersion())); + } + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java index 54ed0ff90633d..9f51cb41dc92a 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java @@ -65,7 +65,7 @@ public static HoodieTestTable of(HoodieTableMetaClient metaClient, HoodieTableMetadataWriter writer, Option context) { testTableState = HoodieTestTableState.of(); - return new HoodieMetadataTestTable(metaClient.getBasePath().toString(), metaClient.getRawHoodieStorage(), + return new HoodieMetadataTestTable(metaClient.getBasePath().toString(), metaClient.getRawStorage(), metaClient, writer, context); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java index b39ac241407e9..9be904ee92567 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java @@ -29,8 +29,10 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.config.HoodieWriteConfig.Builder; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -43,10 +45,12 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.Arrays; import java.util.Date; import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.function.Function; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -63,6 +67,7 @@ public void testPropertyLoading(boolean withAlternative) throws IOException { params.put(HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1"); params.put(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "5"); params.put(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "2"); + params.put(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), "6"); if (withAlternative) { params.put("hoodie.avro.schema.externalTransformation", "true"); } else { @@ -83,6 +88,36 @@ public void testPropertyLoading(boolean withAlternative) throws IOException { assertTrue(config.allowDuplicateInserts()); } + @Test + public void testSupportedTableWriteVersions() { + Set supportedVersions = CollectionUtils.createSet( + HoodieTableVersion.SIX, HoodieTableVersion.EIGHT + ); + Arrays.stream(HoodieTableVersion.values()) + .filter(version -> !supportedVersions.contains(version)) + .forEach(version -> { + assertThrows(IllegalArgumentException.class, () -> HoodieWriteConfig.newBuilder() + .withPath("/tmp").withWriteTableVersion(version.versionCode()).build()); + + Properties props = new Properties(); + props.setProperty(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), String.valueOf(version.versionCode())); + assertThrows(IllegalArgumentException.class, () -> HoodieWriteConfig.newBuilder() + .withPath("/tmp").withProperties(props).build()); + }); + Arrays.stream(HoodieTableVersion.values()) + .filter(supportedVersions::contains) + .forEach(version -> { + + assertEquals(version, HoodieWriteConfig.newBuilder() + .withPath("/tmp").withWriteTableVersion(version.versionCode()).build().getWriteVersion()); + + Properties props = new Properties(); + props.setProperty(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), String.valueOf(version.versionCode())); + assertEquals(version, HoodieWriteConfig.newBuilder() + .withPath("/tmp").withProperties(props).build().getWriteVersion()); + }); + } + @Test public void testDefaultIndexAccordingToEngineType() { testEngineSpecificConfig(HoodieWriteConfig::getIndexType, diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java index 20c85a396c747..1b6c35eb4f751 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java @@ -1088,7 +1088,10 @@ protected void testHoodieConcatHandle(boolean populateMetaFields, boolean isPrep HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); HoodieWriteConfig hoodieWriteConfig = cfgBuilder.withMergeAllowDuplicateOnInserts(true).withTimelineLayoutVersion(VERSION_0).build(); - HoodieTableMetaClient.withPropertyBuilder().fromMetaClient(metaClient).setTimelineLayoutVersion(VERSION_0) + + HoodieTableMetaClient.newTableBuilder() + .fromMetaClient(metaClient) + .setTimelineLayoutVersion(VERSION_0) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); BaseHoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig); @@ -1153,8 +1156,11 @@ protected void testUpsertsInternal(Function3> compact(String compactionInstantTime, boolean shouldComplete) { // only used for metadata table, the compaction happens in single thread - HoodieWriteMetadata> compactionMetadata = getHoodieTable().compact(context, compactionInstantTime); + HoodieWriteMetadata> compactionMetadata = createTable(config, storageConf).compact(context, compactionInstantTime); commitCompaction(compactionInstantTime, compactionMetadata.getCommitMetadata().get(), Option.empty()); return compactionMetadata; } @@ -186,12 +186,8 @@ protected HoodieData convertToWriteStatus(HoodieWriteMetadata>, List, List> createTable(HoodieWriteConfig config, StorageConfiguration storageConf) { - return HoodieFlinkTable.create(config, context); - } - - public HoodieFlinkTable getHoodieTable() { - return HoodieFlinkTable.create(config, context); + protected HoodieTable createTable(HoodieWriteConfig config, StorageConfiguration storageConf, boolean skipValidation) { + return createTableAndValidate(config, HoodieFlinkTable::create, skipValidation); } /** @@ -204,7 +200,7 @@ private HoodieBackedTableMetadataWriter initMetadataWriter(Option latest } public void initMetadataTable() { - HoodieFlinkTable table = getHoodieTable(); + HoodieFlinkTable table = (HoodieFlinkTable) createTable(config, storageConf, false); if (config.isMetadataTableEnabled()) { Option latestPendingInstant = table.getActiveTimeline() .filterInflightsAndRequested().lastInstant().map(HoodieInstant::getTimestamp); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index d6b1730566f95..3bac02db25ac1 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -114,12 +114,12 @@ public boolean commit(String instantTime, List writeStatuses, Optio @Override protected HoodieTable createTable(HoodieWriteConfig config) { - return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); + return createTableAndValidate(config, HoodieFlinkTable::create); } @Override protected HoodieTable createTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context, metaClient); + return createTableAndValidate(config, metaClient, HoodieFlinkTable::create); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index b37990227c570..6a79b1f1bb1d6 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -195,7 +195,7 @@ protected HoodieData getFunctionalIndexRecords(List convertToWriteStatus(HoodieWriteMetadata>, ?, List> createTable(HoodieWriteConfig config, StorageConfiguration storageConf) { - return HoodieJavaTable.create(config, context); + protected HoodieTable>, ?, List> createTable(HoodieWriteConfig config, StorageConfiguration storageConf, boolean skipValidation) { + return createTableAndValidate(config, HoodieJavaTable::create, skipValidation); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index 9552804ac9796..bba0e9f840af6 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -94,12 +94,12 @@ public boolean commit(String instantTime, @Override protected HoodieTable createTable(HoodieWriteConfig config) { - return HoodieJavaTable.create(config, context); + return createTableAndValidate(config, HoodieJavaTable::create); } @Override protected HoodieTable createTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - return HoodieJavaTable.create(config, context, metaClient); + return createTableAndValidate(config, metaClient, HoodieJavaTable::create); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java index 85b5e9f25ce7f..64c8ace922a01 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/metadata/JavaHoodieBackedTableMetadataWriter.java @@ -34,6 +34,8 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.table.HoodieJavaTable; +import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; @@ -60,6 +62,11 @@ protected JavaHoodieBackedTableMetadataWriter(StorageConfiguration storageCon super(storageConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp); } + @Override + HoodieTable getTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) { + return HoodieJavaTable.create(writeConfig, engineContext, metaClient); + } + public static HoodieTableMetadataWriter create(StorageConfiguration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context, diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestMultipleHoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestMultipleHoodieJavaWriteClient.java index d8332323bde7f..e25e3913dc087 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestMultipleHoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestMultipleHoodieJavaWriteClient.java @@ -78,7 +78,7 @@ void testOccWithMultipleWriters(HoodieTableType tableType) throws IOException { final String tableName = "hudiTestTable"; final String basePath = tablePath.toAbsolutePath().toString() + "/" + tableTypeName; - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(tableTypeName) .setTableName(tableName) .setPayloadClassName(HoodieAvroPayload.class.getName()) diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java index 2a610e5ad59d3..c48c22d3f1fcf 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java @@ -193,7 +193,7 @@ public void commitToTable(List recordList, String operation, Map initConfigs = new HashMap<>(writeConfigs); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(writeConfigs.getOrDefault("hoodie.datasource.write.table.type", "MERGE_ON_READ")) .setTableName(writeConfigs.get("hoodie.table.name")) .setPartitionFields(writeConfigs.getOrDefault("hoodie.datasource.write.partitionpath.field", "")) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java index 3521441ed61cb..847ab48115e4c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java @@ -70,8 +70,8 @@ protected HoodieData convertToWriteStatus(HoodieWriteMetadata>, ?, HoodieData> createTable(HoodieWriteConfig config, StorageConfiguration storageConf) { - return HoodieSparkTable.create(config, context); + protected HoodieTable>, ?, HoodieData> createTable(HoodieWriteConfig config, StorageConfiguration storageConf, boolean skipValidation) { + return createTableAndValidate(config, HoodieSparkTable::create, skipValidation); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index f186c112e9b5d..2e639b2819297 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -70,17 +70,6 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig client this(context, clientConfig, Option.empty()); } - @Deprecated - public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) { - this(context, writeConfig, Option.empty()); - } - - @Deprecated - public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending, - Option timelineService) { - this(context, writeConfig, timelineService); - } - public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, Option timelineService) { super(context, writeConfig, timelineService, SparkUpgradeDowngradeHelper.getInstance()); @@ -106,12 +95,12 @@ public boolean commit(String instantTime, JavaRDD writeStatuses, Op @Override protected HoodieTable createTable(HoodieWriteConfig config) { - return HoodieSparkTable.create(config, context); + return createTableAndValidate(config, HoodieSparkTable::create); } @Override protected HoodieTable createTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { - return HoodieSparkTable.create(config, context, metaClient); + return createTableAndValidate(config, metaClient, HoodieSparkTable::create); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 3f8eaa84cd443..9d7f63f354b45 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -216,13 +216,13 @@ protected HoodieData getFunctionalIndexRecords(List, ?, ?> initializeWriteClient() { - return new SparkRDDWriteClient(engineContext, metadataWriteConfig, true); + return new SparkRDDWriteClient(engineContext, metadataWriteConfig, Option.empty()); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java index 230f684d165e2..0dd34ca3964ec 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java @@ -103,7 +103,7 @@ protected HoodieWriteConfig getHoodieWriteConfig(String basePath) { @Test public void readLocalWriteHDFS() throws Exception { // Initialize table and filesystem - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(TABLE_TYPE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) @@ -113,7 +113,7 @@ public void readLocalWriteHDFS() throws Exception { HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath); HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(TABLE_TYPE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) @@ -140,7 +140,7 @@ public void readLocalWriteHDFS() throws Exception { assertEquals(readRecords.count(), records.size()); // Write to local - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(TABLE_TYPE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index e0598ee2252b9..d2279a4419fa6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -165,7 +165,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { tableType = HoodieTableType.MERGE_ON_READ; // Create the table - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) .setTableType(HoodieTableType.MERGE_ON_READ) .setRecordMergeMode(RecordMergeMode.valueOf(RECORD_MERGE_MODE.defaultValue())) @@ -255,7 +255,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { @ValueSource(booleans = {false, true}) public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Exception { // Create the table - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index fd94073788026..680b5e743c54c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -2736,7 +2736,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, false, false).build(), - true)) { + Option.empty())) { String newCommitTime = client.createNewInstantTime(); client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 10); @@ -2769,7 +2769,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, false, false).build(), - true)) { + Option.empty())) { String newCommitTime = client.startCommit(); // Next insert List records = dataGen.generateInserts(newCommitTime, 20); @@ -2969,7 +2969,7 @@ public void testErrorCases() throws Exception { // should be rolled back to last valid commit. try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, false, false).build(), - true)) { + Option.empty())) { String newCommitTime = client.createNewInstantTime(); client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 10); @@ -2995,7 +2995,7 @@ public void testErrorCases() throws Exception { try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, false, false).build(), - true)) { + Option.empty())) { String newCommitTime = client.startCommit(); // Next insert List records = dataGen.generateInserts(newCommitTime, 5); @@ -3410,7 +3410,7 @@ public void testOutOfOrderCommits() throws Exception { metadataProps.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "3"); HoodieWriteConfig metadataWriteConfig = HoodieWriteConfig.newBuilder() .withProperties(metadataProps).build(); - try (SparkRDDWriteClient metadataWriteClient = new SparkRDDWriteClient(context, metadataWriteConfig, true)) { + try (SparkRDDWriteClient metadataWriteClient = new SparkRDDWriteClient(context, metadataWriteConfig, Option.empty())) { final String compactionInstantTime = client.createNewInstantTime(); assertTrue(metadataWriteClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())); metadataWriteClient.compact(compactionInstantTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index 81c508f5d3c79..b013287af6486 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -513,7 +513,7 @@ public void testRestoreWithSavepointBeyondArchival() throws Exception { .withProps(config.getProps()).withTimelineLayoutVersion( VERSION_0).build(); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) .setTimelineLayoutVersion(VERSION_0) .setPopulateMetaFields(config.populateMetaFields()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index 22c9d9eeb6156..a5fe877671903 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -43,6 +43,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieUpgradeDowngradeException; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.keygen.TimestampBasedKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -171,6 +172,17 @@ public void cleanUp() throws Exception { cleanupResources(); } + @Test + public void testAutoUpgradeFailure() throws IOException { + Map params = new HashMap<>(); + addNewTableParamsToProps(params); + HoodieWriteConfig cfg = getConfigBuilder().withAutoUpgradeVersion(false).withProps(params).build(); + HoodieTableMetaClient metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableVersion.SIX); + assertThrows(HoodieUpgradeDowngradeException.class, () -> + new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance()).run(HoodieTableVersion.EIGHT, null) + ); + } + @Test public void testLeftOverUpdatedPropFileCleanup() throws IOException { testUpgradeZeroToOneInternal(true, true, HoodieTableType.MERGE_ON_READ); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java index c95f71dc12773..79272b20aeac4 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java @@ -123,13 +123,12 @@ public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storage @Override public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath, Properties props) throws IOException { - props = HoodieTableMetaClient.withPropertyBuilder() + return HoodieTableMetaClient.newTableBuilder() .setTableName(RAW_TRIPS_TEST_NAME) .setTableType(COPY_ON_WRITE) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); - return HoodieTableMetaClient.initTableAndGetMetaClient(storageConf.newInstance(), basePath, props); + .initTable(storageConf.newInstance(), basePath); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java index 5244dec114702..d011971bb5446 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java @@ -67,7 +67,7 @@ public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter, Option context) { return new HoodieSparkWriteableTestTable(metaClient.getBasePath().toString(), - metaClient.getRawHoodieStorage(), + metaClient.getRawStorage(), metaClient, schema, filter, null, context); } @@ -79,7 +79,7 @@ public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter, Option context) { return new HoodieSparkWriteableTestTable(metaClient.getBasePath().toString(), - metaClient.getRawHoodieStorage(), + metaClient.getRawStorage(), metaClient, schema, filter, metadataWriter, context); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index 7cfaa5c42c098..d2b2f7ec137e5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -175,13 +175,12 @@ public HoodieTableMetaClient getHoodieMetaClient(HoodieTableType tableType, Prop } public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath, HoodieTableType tableType, Properties props) throws IOException { - props = HoodieTableMetaClient.withPropertyBuilder() + return HoodieTableMetaClient.newTableBuilder() .setTableName(RAW_TRIPS_TEST_NAME) .setTableType(tableType) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); - return HoodieTableMetaClient.initTableAndGetMetaClient(storageConf.newInstance(), basePath, props); + .initTable(storageConf.newInstance(), basePath); } public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath) throws IOException { @@ -190,13 +189,12 @@ public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storage @Override public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath, Properties props) throws IOException { - props = HoodieTableMetaClient.withPropertyBuilder() + return HoodieTableMetaClient.newTableBuilder() .setTableName(RAW_TRIPS_TEST_NAME) .setTableType(COPY_ON_WRITE) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); - return HoodieTableMetaClient.initTableAndGetMetaClient(storageConf.newInstance(), basePath, props); + .initTable(storageConf.newInstance(), basePath); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapIndexType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapIndexType.java index ed5a5502e3fcc..0d10191ee6262 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapIndexType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BootstrapIndexType.java @@ -34,7 +34,7 @@ public enum BootstrapIndexType { @EnumFieldDescription("Maintains mapping in HFile format.") HFILE(HFileBootstrapIndex.class.getName()), @EnumFieldDescription("No-op, an empty implementation.") - NO_OP(NoOpBootstrapIndex.class.getName()); + NONE(NoOpBootstrapIndex.class.getName()); private final String className; @@ -57,7 +57,7 @@ public static BootstrapIndexType fromClassName(String className) { public static String getBootstrapIndexClassName(HoodieConfig config) { if (!config.getBooleanOrDefault(BOOTSTRAP_INDEX_ENABLE)) { - return BootstrapIndexType.NO_OP.getClassName(); + return BootstrapIndexType.NONE.getClassName(); } if (config.contains(BOOTSTRAP_INDEX_CLASS_NAME)) { return config.getString(BOOTSTRAP_INDEX_CLASS_NAME); @@ -68,9 +68,13 @@ public static String getBootstrapIndexClassName(HoodieConfig config) { } public static String getDefaultBootstrapIndexClassName(HoodieConfig config) { + return getBootstrapIndexType(config).getClassName(); + } + + public static BootstrapIndexType getBootstrapIndexType(HoodieConfig config) { if (!config.getBooleanOrDefault(BOOTSTRAP_INDEX_ENABLE)) { - return BootstrapIndexType.NO_OP.getClassName(); + return BootstrapIndexType.NONE; } - return BootstrapIndexType.valueOf(BOOTSTRAP_INDEX_TYPE.defaultValue()).getClassName(); + return BootstrapIndexType.valueOf(BOOTSTRAP_INDEX_TYPE.defaultValue()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 8a76decd6201b..a00273fc22f65 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -40,8 +40,10 @@ import org.apache.hudi.common.util.BinaryUtil; import org.apache.hudi.common.util.ConfigUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -59,6 +61,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; import java.time.Instant; import java.util.Arrays; import java.util.HashSet; @@ -67,6 +71,7 @@ import java.util.Properties; import java.util.Set; import java.util.function.BiConsumer; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.DATE_TIME_PARSER; @@ -99,8 +104,8 @@ public class HoodieTableConfig extends HoodieConfig { public static final ConfigProperty DATABASE_NAME = ConfigProperty .key("hoodie.database.name") - .noDefaultValue("Database name can't have default value as it's used to toggle Hive incremental query feature. See HUDI-2837") - .withDocumentation("Database name that will be used for incremental query.If different databases have the same table name during incremental query, " + .noDefaultValue() + .withDocumentation("Database name. If different databases have the same table name during incremental query, " + "we can set it to limit the table name under a specific database"); public static final ConfigProperty NAME = ConfigProperty @@ -111,7 +116,7 @@ public class HoodieTableConfig extends HoodieConfig { public static final ConfigProperty TYPE = ConfigProperty .key("hoodie.table.type") .defaultValue(HoodieTableType.COPY_ON_WRITE) - .withDocumentation("The table type for the underlying data, for this write. This can’t change between writes."); + .withDocumentation("The table type for the underlying data."); public static final ConfigProperty VERSION = ConfigProperty .key("hoodie.table.version") @@ -120,12 +125,14 @@ public class HoodieTableConfig extends HoodieConfig { + "breaking/backwards compatible changes."); public static final ConfigProperty INITIAL_VERSION = ConfigProperty - .key("hoodie.table.initial.version") - .defaultValue(HoodieTableVersion.ZERO) - .withDocumentation("Initial Version of table when the table was created. Used for upgrade/downgrade" - + " to identify what upgrade/downgrade paths happened on the table. This is only configured " - + "when the table is initially setup."); + .key("hoodie.table.initial.version") + .defaultValue(HoodieTableVersion.current()) + .sinceVersion("1.0.0") + .withDocumentation("Initial Version of table when the table was created. Used for upgrade/downgrade" + + " to identify what upgrade/downgrade paths happened on the table. This is only configured " + + "when the table is initially setup."); + // TODO: is this this called precombine in 1.0. .. public static final ConfigProperty PRECOMBINE_FIELD = ConfigProperty .key("hoodie.table.precombine.field") .noDefaultValue() @@ -135,8 +142,8 @@ public class HoodieTableConfig extends HoodieConfig { public static final ConfigProperty PARTITION_FIELDS = ConfigProperty .key("hoodie.table.partition.fields") .noDefaultValue() - .withDocumentation("Fields used to partition the table. Concatenated values of these fields are used as " - + "the partition path, by invoking toString(). These fields also include the partition type which is used by custom key generators"); + .withDocumentation("Comma separated field names used to partition the table. These field names also include " + + "the partition type which is used by custom key generators"); public static final ConfigProperty RECORDKEY_FIELDS = ConfigProperty .key("hoodie.table.recordkey.fields") @@ -156,10 +163,11 @@ public class HoodieTableConfig extends HoodieConfig { .withDocumentation(HoodieCDCSupplementalLoggingMode.class) .sinceVersion("0.13.0"); + // TODO: is this necessary? won't we just use a table schema. public static final ConfigProperty CREATE_SCHEMA = ConfigProperty .key("hoodie.table.create.schema") .noDefaultValue() - .withDocumentation("Schema used when creating the table, for the first time."); + .withDocumentation("Schema used when creating the table"); public static final ConfigProperty BASE_FILE_FORMAT = ConfigProperty .key("hoodie.table.base.file.format") @@ -178,6 +186,7 @@ public class HoodieTableConfig extends HoodieConfig { .noDefaultValue() .withDocumentation("Version of timeline used, by the table."); + //TODO: why is this the default? not OVERWRITE_WITH_LATEST? public static final ConfigProperty RECORD_MERGE_MODE = ConfigProperty .key("hoodie.record.merge.mode") .defaultValue(RecordMergeMode.EVENT_TIME_ORDERING.name()) @@ -188,7 +197,7 @@ public class HoodieTableConfig extends HoodieConfig { .key("hoodie.compaction.payload.class") .defaultValue(DefaultHoodieRecordPayload.class.getName()) .deprecatedAfter("1.0.0") - .withDocumentation("Payload class to use for performing compactions, i.e merge delta logs with current base file and then " + .withDocumentation("Payload class to use for performing merges, compactions, i.e merge delta logs with current base file and then " + " produce a new base file."); public static final ConfigProperty PAYLOAD_TYPE = ConfigProperty @@ -259,6 +268,7 @@ public class HoodieTableConfig extends HoodieConfig { .sinceVersion("1.0.0") .withDocumentation("Key Generator type to determine key generator class"); + // TODO: this has to be UTC. why is it not the default? public static final ConfigProperty TIMELINE_TIMEZONE = ConfigProperty .key("hoodie.table.timeline.timezone") .defaultValue(HoodieTimelineTimeZone.LOCAL) @@ -303,6 +313,7 @@ public class HoodieTableConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Table checksum is used to guard against partial writes in HDFS. It is added as the last entry in hoodie.properties and then used to validate while reading table config."); + // TODO: understand why is writing/changing all these. this has to work on both HDFS and Cloud. public static final ConfigProperty TABLE_METADATA_PARTITIONS_INFLIGHT = ConfigProperty .key("hoodie.table.metadata.partitions.inflight") .noDefaultValue() @@ -331,6 +342,23 @@ public class HoodieTableConfig extends HoodieConfig { private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // . + static List> definedTableConfigs() { + Field[] fields = ReflectionUtils.getClass(HoodieTableConfig.class.getName()).getDeclaredFields(); + return Arrays.stream(fields) + .filter(f -> f.getType().equals(ConfigProperty.class) + && Modifier.isPublic(f.getModifiers()) + && Modifier.isStatic(f.getModifiers()) + ) + .map(f -> { + try { + return (ConfigProperty) f.get(null); + } catch (IllegalAccessException e) { + throw new HoodieException("Error reading defined table configs, for " + f.getName(), e); + } + }) + .collect(Collectors.toList()); + } + public HoodieTableConfig(HoodieStorage storage, StoragePath metaPath, String payloadClassName, String recordMergerStrategyId) { super(); StoragePath propertyPath = new StoragePath(metaPath, HOODIE_PROPERTIES_FILE); @@ -354,13 +382,12 @@ public HoodieTableConfig(HoodieStorage storage, StoragePath metaPath, String pay needStore = true; } if (needStore) { - // FIXME(vc): wonder if this can be removed. Need to look into history. try (OutputStream outputStream = storage.create(propertyPath)) { storeProperties(props, outputStream); } } } catch (IOException e) { - throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e); + throw new HoodieIOException("Could not load properties from " + propertyPath, e); } } @@ -470,6 +497,8 @@ public static void delete(HoodieStorage storage, StoragePath metadataFolder, Set /** * Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties). + * + * TODO: this directory creation etc should happen in the HoodieTableMetaClient. */ public static void create(HoodieStorage storage, StoragePath metadataFolder, Properties properties) throws IOException { @@ -478,13 +507,16 @@ public static void create(HoodieStorage storage, StoragePath metadataFolder, Pro } HoodieConfig hoodieConfig = new HoodieConfig(properties); StoragePath propertyPath = new StoragePath(metadataFolder, HOODIE_PROPERTIES_FILE); + HoodieTableVersion tableVersion = getTableVersion(hoodieConfig); try (OutputStream outputStream = storage.create(propertyPath)) { if (!hoodieConfig.contains(NAME)) { throw new IllegalArgumentException(NAME.key() + " property needs to be specified"); } hoodieConfig.setDefaultValue(TYPE); if (hoodieConfig.getString(TYPE).equals(HoodieTableType.MERGE_ON_READ.name())) { - hoodieConfig.setDefaultValue(PAYLOAD_TYPE); + if (tableVersion.greaterThan(HoodieTableVersion.SEVEN)) { + hoodieConfig.setDefaultValue(PAYLOAD_TYPE); + } hoodieConfig.setDefaultValue(RECORD_MERGER_STRATEGY); } hoodieConfig.setDefaultValue(ARCHIVELOG_FOLDER); @@ -493,14 +525,19 @@ public static void create(HoodieStorage storage, StoragePath metadataFolder, Pro hoodieConfig.setValue(TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString()); } if (hoodieConfig.contains(BOOTSTRAP_BASE_PATH)) { - // Use the default bootstrap index class. - hoodieConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_NAME, BootstrapIndexType.getDefaultBootstrapIndexClassName(hoodieConfig)); + if (tableVersion.greaterThan(HoodieTableVersion.SEVEN)) { + hoodieConfig.setDefaultValue(BOOTSTRAP_INDEX_TYPE, BootstrapIndexType.getBootstrapIndexType(hoodieConfig).toString()); + } else { + // Use the default bootstrap index class. + hoodieConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_NAME, BootstrapIndexType.getDefaultBootstrapIndexClassName(hoodieConfig)); + } } if (hoodieConfig.contains(TIMELINE_TIMEZONE)) { HoodieInstantTimeGenerator.setCommitTimeZone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getString(TIMELINE_TIMEZONE))); } hoodieConfig.setDefaultValue(DROP_PARTITION_COLUMNS); + dropInvalidConfigs(hoodieConfig); storeProperties(hoodieConfig.getProps(), outputStream); } } @@ -518,6 +555,36 @@ public static boolean validateChecksum(Properties props) { return Long.parseLong(props.getProperty(TABLE_CHECKSUM.key())) == generateChecksum(props); } + static void dropInvalidConfigs(HoodieConfig config) { + HoodieTableVersion tableVersion = getTableVersion(config); + Map> definedTableConfigs = HoodieTableConfig.definedTableConfigs() + .stream().collect(Collectors.toMap(ConfigProperty::key, Function.identity())); + List invalidConfigs = config.getProps().keySet().stream() + .map(k -> (String) k) + // TODO: this can be eventually tightened to ensure all table configs are defined. + .filter(key -> definedTableConfigs.containsKey(key) + && !validateConfigVersion(definedTableConfigs.get(key), tableVersion)) + .collect(Collectors.toList()); + invalidConfigs.forEach(key -> { + config.getProps().remove(key); + }); + } + + static boolean validateConfigVersion(ConfigProperty configProperty, HoodieTableVersion tableVersion) { + // TODO: this can be tightened up, once all configs have a since version. + if (!configProperty.getSinceVersion().isPresent()) { + return true; + } + // validate that the table version is greater than or equal to the config version + HoodieTableVersion firstVersion = HoodieTableVersion.fromReleaseVersion(configProperty.getSinceVersion().get()); + boolean valid = tableVersion.greaterThan(firstVersion) || tableVersion.equals(firstVersion); + if (!valid) { + LOG.warn("Table version {} is lower than or equal to config's first version {}. Config {} will be ignored.", + tableVersion, firstVersion, configProperty.key()); + } + return valid; + } + /** * This function returns the partition fields joined by BaseKeyGenerator.FIELD_SEPARATOR. It will also * include the key generator partition type with the field. The key generator partition type is used for @@ -575,7 +642,7 @@ public static String getPartitionFieldWithoutKeyGenPartitionType(String partitio */ public static HoodieTableVersion getTableVersion(HoodieConfig config) { return contains(VERSION, config) - ? HoodieTableVersion.versionFromCode(config.getInt(VERSION)) + ? HoodieTableVersion.fromVersionCode(config.getInt(VERSION)) : VERSION.defaultValue(); } @@ -604,7 +671,7 @@ public HoodieTableVersion getTableVersion() { */ public HoodieTableVersion getTableInitialVersion() { return contains(INITIAL_VERSION) - ? HoodieTableVersion.versionFromCode(getInt(INITIAL_VERSION)) + ? HoodieTableVersion.fromVersionCode(getInt(INITIAL_VERSION)) : INITIAL_VERSION.defaultValue(); } @@ -612,6 +679,10 @@ public void setTableVersion(HoodieTableVersion tableVersion) { setValue(VERSION, Integer.toString(tableVersion.versionCode())); } + public void setInitialVersion(HoodieTableVersion initialVersion) { + setValue(INITIAL_VERSION, Integer.toString(initialVersion.versionCode())); + } + public RecordMergeMode getRecordMergeMode() { return RecordMergeMode.valueOf(getStringOrDefault(RECORD_MERGE_MODE).toUpperCase()); } @@ -671,32 +742,6 @@ public String getPartitionFieldProp() { return getPartitionFieldProp(this).orElse(""); } - /** - * Read the payload class for HoodieRecords from the table properties. - */ - public String getBootstrapIndexClass() { - if (!props.getBoolean(BOOTSTRAP_INDEX_ENABLE.key(), BOOTSTRAP_INDEX_ENABLE.defaultValue())) { - return BootstrapIndexType.NO_OP.getClassName(); - } - String bootstrapIndexClassName; - if (contains(BOOTSTRAP_INDEX_TYPE)) { - bootstrapIndexClassName = BootstrapIndexType.valueOf(getString(BOOTSTRAP_INDEX_TYPE)).getClassName(); - } else if (contains(BOOTSTRAP_INDEX_CLASS_NAME)) { - bootstrapIndexClassName = getString(BOOTSTRAP_INDEX_CLASS_NAME); - } else { - bootstrapIndexClassName = BootstrapIndexType.valueOf(BOOTSTRAP_INDEX_TYPE.defaultValue()).getClassName(); - } - return bootstrapIndexClassName; - } - - public static String getDefaultBootstrapIndexClass(Properties props) { - HoodieConfig hoodieConfig = new HoodieConfig(props); - if (!hoodieConfig.getBooleanOrDefault(BOOTSTRAP_INDEX_ENABLE)) { - return BootstrapIndexType.NO_OP.getClassName(); - } - return BootstrapIndexType.valueOf(BOOTSTRAP_INDEX_TYPE.defaultValue()).getClassName(); - } - public Option getBootstrapBasePath() { return Option.ofNullable(getString(BOOTSTRAP_BASE_PATH)); } @@ -801,13 +846,6 @@ public boolean isMultipleBaseFileFormatsEnabled() { return getBooleanOrDefault(MULTIPLE_BASE_FILE_FORMATS_ENABLE); } - /** - * Read the table checksum. - */ - private Long getTableChecksum() { - return getLong(TABLE_CHECKSUM); - } - public Set getMetadataPartitionsInflight() { return new HashSet<>(StringUtils.split( getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, StringUtils.EMPTY_STRING), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 68e391912dd94..5c55c37483fd2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -50,6 +50,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.TableNotFoundException; @@ -80,8 +81,8 @@ import static org.apache.hudi.common.model.HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; import static org.apache.hudi.common.model.HoodieRecordMerger.OVERWRITE_MERGER_STRATEGY_UUID; -import static org.apache.hudi.common.table.HoodieTableConfig.INITIAL_VERSION; import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGE_MODE; +import static org.apache.hudi.common.table.HoodieTableConfig.VERSION; import static org.apache.hudi.common.util.ConfigUtils.containsConfigProperty; import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; @@ -150,9 +151,9 @@ public class HoodieTableMetaClient implements Serializable { * Can only be called if table already exists */ protected HoodieTableMetaClient(HoodieStorage storage, String basePath, boolean loadActiveTimelineOnLoad, - ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, - String payloadClassName, String recordMergerStrategy, HoodieTimeGeneratorConfig timeGeneratorConfig, - FileSystemRetryConfig fileSystemRetryConfig) { + ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, + String payloadClassName, String recordMergerStrategy, HoodieTimeGeneratorConfig timeGeneratorConfig, + FileSystemRetryConfig fileSystemRetryConfig) { LOG.info("Loading HoodieTableMetaClient from " + basePath); this.timeGeneratorConfig = timeGeneratorConfig; this.consistencyGuardConfig = consistencyGuardConfig; @@ -406,11 +407,11 @@ private static HoodieStorage getStorage(StoragePath path, consistencyGuard); } - public void setHoodieStorage(HoodieStorage storage) { + public void setStorage(HoodieStorage storage) { this.storage = storage; } - public HoodieStorage getRawHoodieStorage() { + public HoodieStorage getRawStorage() { return getStorage().getRawStorage(); } @@ -528,68 +529,10 @@ private HoodieArchivedTimeline instantiateArchivedTimeline(String startTs) { : new HoodieArchivedTimeline(this, startTs); } - /** - * Validate table properties. - * - * @param properties Properties from writeConfig. - */ - public void validateTableProperties(Properties properties) { - // Once meta fields are disabled, it cant be re-enabled for a given table. - if (!getTableConfig().populateMetaFields() - && Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue().toString()))) { - throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back"); - } - - // Meta fields can be disabled only when either {@code SimpleKeyGenerator}, {@code ComplexKeyGenerator}, {@code NonpartitionedKeyGenerator} is used - if (!getTableConfig().populateMetaFields()) { - String keyGenClass = KeyGeneratorType.getKeyGeneratorClassName(new HoodieConfig(properties)); - if (StringUtils.isNullOrEmpty(keyGenClass)) { - keyGenClass = "org.apache.hudi.keygen.SimpleKeyGenerator"; - } - if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator") - && !keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator") - && !keyGenClass.equals("org.apache.hudi.keygen.ComplexKeyGenerator")) { - throw new HoodieException("Only simple, non-partitioned or complex key generator are supported when meta-fields are disabled. Used: " + keyGenClass); - } - } - - //Check to make sure it's not a COW table with consistent hashing bucket index - if (tableType == HoodieTableType.COPY_ON_WRITE) { - String indexType = properties.getProperty("hoodie.index.type"); - if (indexType != null && indexType.equals("BUCKET")) { - String bucketEngine = properties.getProperty("hoodie.index.bucket.engine"); - if (bucketEngine != null && bucketEngine.equals("CONSISTENT_HASHING")) { - throw new HoodieException("Consistent hashing bucket index does not work with COW table. Use simple bucket index or an MOR table."); - } - } - } - } - - /** - * Helper method to initialize a given path as a hoodie table with configs passed in as Properties. - * - * @return Instance of HoodieTableMetaClient - */ - public static HoodieTableMetaClient initTableAndGetMetaClient(StorageConfiguration storageConf, String basePath, - Properties props) throws IOException { - return initTableAndGetMetaClient(storageConf, new StoragePath(basePath), props); - } - - public static HoodieTableMetaClient initTableAndGetMetaClient(StorageConfiguration storageConf, StoragePath basePath, - Properties props) throws IOException { - initTableMetaClient(storageConf, basePath, props); - // We should not use fs.getConf as this might be different from the original configuration - // used to create the fs in unit tests - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(storageConf).setBasePath(basePath) - .setMetaserverConfig(props) - .build(); - LOG.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() + " from " + basePath); - return metaClient; - } - - private static void initTableMetaClient(StorageConfiguration storageConf, StoragePath basePath, - Properties props) throws IOException { - LOG.info("Initializing " + basePath + " as hoodie table " + basePath); + private static void createTableLayoutOnStorage(StorageConfiguration storageConf, + StoragePath basePath, + Properties props) throws IOException { + LOG.info("Initializing {} as hoodie table", basePath); final HoodieStorage storage = HoodieStorageUtils.getStorage(basePath, storageConf); if (!storage.exists(basePath)) { storage.createDirectory(basePath); @@ -626,8 +569,6 @@ private static void initTableMetaClient(StorageConfiguration storageConf, Sto } initializeBootstrapDirsIfNotExists(basePath, storage); - // When the table is initialized, set the initial version to be the current version. - props.put(INITIAL_VERSION.key(), String.valueOf(HoodieTableVersion.current().versionCode())); HoodieTableConfig.create(storage, metaPathDir, props); } @@ -801,14 +742,16 @@ private static HoodieTableMetaClient newMetaClient(HoodieStorage storage, String String payloadClassName, String recordMergerStrategy, HoodieTimeGeneratorConfig timeGeneratorConfig, FileSystemRetryConfig fileSystemRetryConfig, HoodieMetaserverConfig metaserverConfig) { - return metaserverConfig.isMetaserverEnabled() ? (HoodieTableMetaClient) ReflectionUtils.loadClass( - "org.apache.hudi.common.table.HoodieTableMetaserverClient", - new Class[] {HoodieStorage.class, String.class, ConsistencyGuardConfig.class, String.class, HoodieTimeGeneratorConfig.class, - FileSystemRetryConfig.class, Option.class, Option.class, HoodieMetaserverConfig.class}, - storage, basePath, consistencyGuardConfig, recordMergerStrategy, timeGeneratorConfig, fileSystemRetryConfig, - Option.ofNullable(metaserverConfig.getDatabaseName()), Option.ofNullable(metaserverConfig.getTableName()), metaserverConfig) - : new HoodieTableMetaClient(storage, basePath, loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, recordMergerStrategy, - timeGeneratorConfig, fileSystemRetryConfig); + if (metaserverConfig.isMetaserverEnabled()) { + return (HoodieTableMetaClient) ReflectionUtils.loadClass("org.apache.hudi.common.table.HoodieTableMetaserverClient", + new Class[] {HoodieStorage.class, String.class, ConsistencyGuardConfig.class, String.class, HoodieTimeGeneratorConfig.class, + FileSystemRetryConfig.class, Option.class, Option.class, HoodieMetaserverConfig.class}, + storage, basePath, consistencyGuardConfig, recordMergerStrategy, timeGeneratorConfig, fileSystemRetryConfig, + Option.ofNullable(metaserverConfig.getDatabaseName()), Option.ofNullable(metaserverConfig.getTableName()), metaserverConfig); + } else { + return new HoodieTableMetaClient(storage, basePath, loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, recordMergerStrategy, + timeGeneratorConfig, fileSystemRetryConfig); + } } public static Builder builder() { @@ -914,19 +857,20 @@ public HoodieTableMetaClient build() { } } - public static PropertyBuilder withPropertyBuilder() { - return new PropertyBuilder(); + public static TableBuilder newTableBuilder() { + return new TableBuilder(); } /** * Builder for {@link Properties}. */ - public static class PropertyBuilder { + public static class TableBuilder { private HoodieTableType tableType; private String databaseName; private String tableName; private String tableCreateSchema; + private HoodieTableVersion tableVersion; private String recordKeyFields; private String secondaryKeyFields; private String archiveLogFolder; @@ -964,184 +908,192 @@ public static class PropertyBuilder { */ private Properties others = new Properties(); - private PropertyBuilder() { - + TableBuilder() { } - public PropertyBuilder setTableType(HoodieTableType tableType) { + public TableBuilder setTableType(HoodieTableType tableType) { this.tableType = tableType; return this; } - public PropertyBuilder setTableType(String tableType) { + public TableBuilder setTableType(String tableType) { return setTableType(HoodieTableType.valueOf(tableType)); } - public PropertyBuilder setDatabaseName(String databaseName) { + public TableBuilder setDatabaseName(String databaseName) { this.databaseName = databaseName; return this; } - public PropertyBuilder setTableName(String tableName) { + public TableBuilder setTableName(String tableName) { this.tableName = tableName; return this; } - public PropertyBuilder setTableCreateSchema(String tableCreateSchema) { + public TableBuilder setTableVersion(HoodieTableVersion tableVersion) { + this.tableVersion = tableVersion; + return this; + } + + public TableBuilder setTableVersion(int tableVersion) { + return setTableVersion(HoodieTableVersion.fromVersionCode(tableVersion)); + } + + public TableBuilder setTableCreateSchema(String tableCreateSchema) { this.tableCreateSchema = tableCreateSchema; return this; } - public PropertyBuilder setRecordKeyFields(String recordKeyFields) { + public TableBuilder setRecordKeyFields(String recordKeyFields) { this.recordKeyFields = recordKeyFields; return this; } - public PropertyBuilder setSecondaryKeyFields(String secondaryKeyFields) { + public TableBuilder setSecondaryKeyFields(String secondaryKeyFields) { this.secondaryKeyFields = secondaryKeyFields; return this; } - public PropertyBuilder setArchiveLogFolder(String archiveLogFolder) { + public TableBuilder setArchiveLogFolder(String archiveLogFolder) { this.archiveLogFolder = archiveLogFolder; return this; } - public PropertyBuilder setRecordMergeMode(RecordMergeMode recordMergeMode) { + public TableBuilder setRecordMergeMode(RecordMergeMode recordMergeMode) { this.recordMergeMode = recordMergeMode; return this; } - public PropertyBuilder setPayloadClassName(String payloadClassName) { + public TableBuilder setPayloadClassName(String payloadClassName) { this.payloadClassName = payloadClassName; return this; } - public PropertyBuilder setPayloadType(String payloadType) { + public TableBuilder setPayloadType(String payloadType) { this.payloadType = payloadType; return this; } - public PropertyBuilder setRecordMergerStrategy(String recordMergerStrategy) { + public TableBuilder setRecordMergerStrategy(String recordMergerStrategy) { this.recordMergerStrategy = recordMergerStrategy; return this; } - public PropertyBuilder setPayloadClass(Class payloadClass) { + public TableBuilder setPayloadClass(Class payloadClass) { return setPayloadClassName(payloadClass.getName()); } - public PropertyBuilder setTimelineLayoutVersion(Integer timelineLayoutVersion) { + public TableBuilder setTimelineLayoutVersion(Integer timelineLayoutVersion) { this.timelineLayoutVersion = timelineLayoutVersion; return this; } - public PropertyBuilder setBaseFileFormat(String baseFileFormat) { + public TableBuilder setBaseFileFormat(String baseFileFormat) { this.baseFileFormat = baseFileFormat; return this; } - public PropertyBuilder setPreCombineField(String preCombineField) { + public TableBuilder setPreCombineField(String preCombineField) { this.preCombineField = preCombineField; return this; } - public PropertyBuilder setPartitionFields(String partitionFields) { + public TableBuilder setPartitionFields(String partitionFields) { this.partitionFields = partitionFields; return this; } - public PropertyBuilder setCDCEnabled(boolean cdcEnabled) { + public TableBuilder setCDCEnabled(boolean cdcEnabled) { this.cdcEnabled = cdcEnabled; return this; } - public PropertyBuilder setCDCSupplementalLoggingMode(String cdcSupplementalLoggingMode) { + public TableBuilder setCDCSupplementalLoggingMode(String cdcSupplementalLoggingMode) { this.cdcSupplementalLoggingMode = cdcSupplementalLoggingMode.toUpperCase(); return this; } - public PropertyBuilder setBootstrapIndexClass(String bootstrapIndexClass) { + public TableBuilder setBootstrapIndexClass(String bootstrapIndexClass) { this.bootstrapIndexClass = bootstrapIndexClass; return this; } - public PropertyBuilder setBootstrapBasePath(String bootstrapBasePath) { + public TableBuilder setBootstrapBasePath(String bootstrapBasePath) { this.bootstrapBasePath = bootstrapBasePath; return this; } - public PropertyBuilder setBootstrapIndexEnable(Boolean bootstrapIndexEnable) { + public TableBuilder setBootstrapIndexEnable(Boolean bootstrapIndexEnable) { this.bootstrapIndexEnable = bootstrapIndexEnable; return this; } - public PropertyBuilder setPopulateMetaFields(boolean populateMetaFields) { + public TableBuilder setPopulateMetaFields(boolean populateMetaFields) { this.populateMetaFields = populateMetaFields; return this; } - public PropertyBuilder setKeyGeneratorClassProp(String keyGeneratorClassProp) { + public TableBuilder setKeyGeneratorClassProp(String keyGeneratorClassProp) { this.keyGeneratorClassProp = keyGeneratorClassProp; return this; } - public PropertyBuilder setKeyGeneratorType(String keyGeneratorType) { + public TableBuilder setKeyGeneratorType(String keyGeneratorType) { this.keyGeneratorType = keyGeneratorType; return this; } - public PropertyBuilder setHiveStylePartitioningEnable(Boolean hiveStylePartitioningEnable) { + public TableBuilder setHiveStylePartitioningEnable(Boolean hiveStylePartitioningEnable) { this.hiveStylePartitioningEnable = hiveStylePartitioningEnable; return this; } - public PropertyBuilder setUrlEncodePartitioning(Boolean urlEncodePartitioning) { + public TableBuilder setUrlEncodePartitioning(Boolean urlEncodePartitioning) { this.urlEncodePartitioning = urlEncodePartitioning; return this; } - public PropertyBuilder setCommitTimezone(HoodieTimelineTimeZone timelineTimeZone) { + public TableBuilder setCommitTimezone(HoodieTimelineTimeZone timelineTimeZone) { this.commitTimeZone = timelineTimeZone; return this; } - public PropertyBuilder setPartitionMetafileUseBaseFormat(Boolean useBaseFormat) { + public TableBuilder setPartitionMetafileUseBaseFormat(Boolean useBaseFormat) { this.partitionMetafileUseBaseFormat = useBaseFormat; return this; } - public PropertyBuilder setShouldDropPartitionColumns(Boolean shouldDropPartitionColumns) { + public TableBuilder setShouldDropPartitionColumns(Boolean shouldDropPartitionColumns) { this.shouldDropPartitionColumns = shouldDropPartitionColumns; return this; } - public PropertyBuilder setMetadataPartitions(String partitions) { + public TableBuilder setMetadataPartitions(String partitions) { this.metadataPartitions = partitions; return this; } - public PropertyBuilder setInflightMetadataPartitions(String partitions) { + public TableBuilder setInflightMetadataPartitions(String partitions) { this.inflightMetadataPartitions = partitions; return this; } - public PropertyBuilder setSecondaryIndexesMetadata(String secondaryIndexesMetadata) { + public TableBuilder setSecondaryIndexesMetadata(String secondaryIndexesMetadata) { this.secondaryIndexesMetadata = secondaryIndexesMetadata; return this; } - public PropertyBuilder setMultipleBaseFileFormatsEnabled(Boolean multipleBaseFileFormatsEnabled) { + public TableBuilder setMultipleBaseFileFormatsEnabled(Boolean multipleBaseFileFormatsEnabled) { this.multipleBaseFileFormatsEnabled = multipleBaseFileFormatsEnabled; return this; } - public PropertyBuilder setIndexDefinitionPath(String indexDefinitionPath) { + public TableBuilder setIndexDefinitionPath(String indexDefinitionPath) { this.indexDefinitionPath = indexDefinitionPath; return this; } - public PropertyBuilder set(Map props) { + public TableBuilder set(Map props) { for (ConfigProperty configProperty : HoodieTableConfig.PERSISTED_CONFIG_LIST) { if (containsConfigProperty(props, configProperty)) { String value = getStringWithAltKeys(props, configProperty); @@ -1153,16 +1105,17 @@ public PropertyBuilder set(Map props) { return this; } - public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) { + public TableBuilder fromMetaClient(HoodieTableMetaClient metaClient) { return setTableType(metaClient.getTableType()) .setTableName(metaClient.getTableConfig().getTableName()) + .setTableVersion(metaClient.getTableConfig().getTableVersion()) .setArchiveLogFolder(metaClient.getTableConfig().getArchivelogFolder()) .setRecordMergeMode(metaClient.getTableConfig().getRecordMergeMode()) .setPayloadClassName(metaClient.getTableConfig().getPayloadClass()) .setRecordMergerStrategy(metaClient.getTableConfig().getRecordMergerStrategy()); } - public PropertyBuilder fromProperties(Properties properties) { + public TableBuilder fromProperties(Properties properties) { HoodieConfig hoodieConfig = new HoodieConfig(properties); for (ConfigProperty configProperty : HoodieTableConfig.PERSISTED_CONFIG_LIST) { @@ -1180,6 +1133,11 @@ public PropertyBuilder fromProperties(Properties properties) { if (hoodieConfig.contains(HoodieTableConfig.NAME)) { setTableName(hoodieConfig.getString(HoodieTableConfig.NAME)); } + + if (hoodieConfig.contains(VERSION)) { + setTableVersion(hoodieConfig.getInt(VERSION)); + } + if (hoodieConfig.contains(HoodieTableConfig.TYPE)) { setTableType(hoodieConfig.getString(HoodieTableConfig.TYPE)); } @@ -1279,6 +1237,7 @@ public PropertyBuilder fromProperties(Properties properties) { return this; } + @VisibleForTesting public Properties build() { checkArgument(tableType != null, "tableType is null"); checkArgument(tableName != null, "tableName is null"); @@ -1292,7 +1251,14 @@ public Properties build() { } tableConfig.setValue(HoodieTableConfig.NAME, tableName); tableConfig.setValue(HoodieTableConfig.TYPE, tableType.name()); - tableConfig.setValue(HoodieTableConfig.VERSION, String.valueOf(HoodieTableVersion.current().versionCode())); + + if (null != tableVersion) { + tableConfig.setTableVersion(tableVersion); + tableConfig.setInitialVersion(tableVersion); + } else { + tableConfig.setTableVersion(HoodieTableVersion.current()); + tableConfig.setInitialVersion(HoodieTableVersion.current()); + } if (tableType == HoodieTableType.MERGE_ON_READ) { if (null != payloadClassName) { @@ -1401,15 +1367,18 @@ public Properties build() { return tableConfig.getProps(); } - /** - * Init Table with the properties build by this builder. - * - * @param configuration The storage configuration. - * @param basePath The base path for hoodie table. - */ - public HoodieTableMetaClient initTable(StorageConfiguration configuration, String basePath) - throws IOException { - return HoodieTableMetaClient.initTableAndGetMetaClient(configuration, basePath, build()); + public HoodieTableMetaClient initTable(StorageConfiguration configuration, String basePath) throws IOException { + return initTable(configuration, new StoragePath(basePath)); + } + + public HoodieTableMetaClient initTable(StorageConfiguration storageConf, StoragePath basePath) throws IOException { + Properties props = build(); + createTableLayoutOnStorage(storageConf, basePath, props); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(storageConf).setBasePath(basePath) + .setMetaserverConfig(props) + .build(); + LOG.info("Finished initializing Table of type {} from {}", metaClient.getTableConfig().getTableType(), basePath); + return metaClient; } private void inferRecordMergeMode() { @@ -1497,9 +1466,5 @@ private String constructMergeConfigErrorMessage() { stringBuilder.append(recordMergeMode); return stringBuilder.toString(); } - - public HoodieTableMetaClient initTable(StorageConfiguration configuration, StoragePath basePath) throws IOException { - return HoodieTableMetaClient.initTableAndGetMetaClient(configuration, basePath, build()); - } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java index 2fa321080f2cf..d6c0f93c7431d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java @@ -18,9 +18,11 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.exception.HoodieException; import java.util.Arrays; +import java.util.List; /** * Table's version that controls what version of writer/readers can actually read/write @@ -28,28 +30,31 @@ */ public enum HoodieTableVersion { // < 0.6.0 versions - ZERO(0), + ZERO(0, CollectionUtils.createImmutableList("0.3.0")), // 0.6.0 onwards - ONE(1), + ONE(1, CollectionUtils.createImmutableList("0.6.0")), // 0.9.0 onwards - TWO(2), + TWO(2, CollectionUtils.createImmutableList("0.9.0")), // 0.10.0 onwards - THREE(3), + THREE(3, CollectionUtils.createImmutableList("0.10.0")), // 0.11.0 onwards - FOUR(4), + FOUR(4, CollectionUtils.createImmutableList("0.11.0")), // 0.12.0 onwards - FIVE(5), + FIVE(5, CollectionUtils.createImmutableList("0.12.0", "0.13.0")), // 0.14.0 onwards - SIX(6), + SIX(6, CollectionUtils.createImmutableList("0.14.0")), // 0.16.0 - SEVEN(7), - // 1.0 beta - EIGHT(8); + SEVEN(7, CollectionUtils.createImmutableList("0.16.0")), + // 1.0 + EIGHT(8, CollectionUtils.createImmutableList("1.0.0")); private final int versionCode; - HoodieTableVersion(int versionCode) { + private final List releaseVersions; + + HoodieTableVersion(int versionCode, List releaseVersions) { this.versionCode = versionCode; + this.releaseVersions = releaseVersions; } public int versionCode() { @@ -60,10 +65,16 @@ public static HoodieTableVersion current() { return EIGHT; } - public static HoodieTableVersion versionFromCode(int versionCode) { + public static HoodieTableVersion fromVersionCode(int versionCode) { return Arrays.stream(HoodieTableVersion.values()) .filter(v -> v.versionCode == versionCode).findAny() - .orElseThrow(() -> new HoodieException("Unknown versionCode:" + versionCode)); + .orElseThrow(() -> new HoodieException("Unknown table versionCode:" + versionCode)); + } + + public static HoodieTableVersion fromReleaseVersion(String releaseVersion) { + return Arrays.stream(HoodieTableVersion.values()) + .filter(v -> v.releaseVersions.contains(releaseVersion)).findAny() + .orElseThrow(() -> new HoodieException("Unknown table firstReleaseVersion:" + releaseVersion)); } public boolean greaterThan(HoodieTableVersion other) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 2c9da90eb96a8..5e3e423933c0b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -311,7 +311,7 @@ public Schema readSchemaFromLastCompaction(Option lastCompactionC } private Schema readSchemaFromLogFile(StoragePath path) throws IOException { - return readSchemaFromLogFile(metaClient.getRawHoodieStorage(), path); + return readSchemaFromLogFile(metaClient.getRawStorage(), path); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTableConfigUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTableConfigUtils.java new file mode 100644 index 0000000000000..b15c97164fa2e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieTableConfigUtils.java @@ -0,0 +1,87 @@ +/* + * 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.common.util; + +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.keygen.BaseKeyGenerator; + +import java.util.Arrays; +import java.util.stream.Collectors; + +public class HoodieTableConfigUtils { + + /** + * This function returns the partition fields joined by BaseKeyGenerator.FIELD_SEPARATOR. It will also + * include the key generator partition type with the field. The key generator partition type is used for + * Custom Key Generator. + */ + public static Option getPartitionFieldPropForKeyGenerator(HoodieConfig config) { + return Option.ofNullable(config.getString(HoodieTableConfig.PARTITION_FIELDS)); + } + + /** + * This function returns the partition fields joined by BaseKeyGenerator.FIELD_SEPARATOR. It will + * strip the partition key generator related info from the fields. + */ + public static Option getPartitionFieldProp(HoodieConfig config) { + if (getTableVersion(config).greaterThan(HoodieTableVersion.SEVEN)) { + // With table version eight, the table config org.apache.hudi.common.table.HoodieTableConfig.PARTITION_FIELDS + // stores the corresponding partition type as well. This partition type is useful for CustomKeyGenerator + // and CustomAvroKeyGenerator. + return getPartitionFields(config).map(fields -> String.join(BaseKeyGenerator.FIELD_SEPARATOR, fields)); + } else { + return Option.ofNullable(config.getString(HoodieTableConfig.PARTITION_FIELDS)); + } + } + + /** + * This function returns the partition fields only. This method strips the key generator related + * partition key types from the configured fields. + */ + public static Option getPartitionFields(HoodieConfig config) { + if (HoodieConfig.contains(HoodieTableConfig.PARTITION_FIELDS, config)) { + return Option.of(Arrays.stream(config.getString(HoodieTableConfig.PARTITION_FIELDS).split(",")) + .filter(p -> !p.isEmpty()) + .map(p -> getPartitionFieldWithoutKeyGenPartitionType(p, config)) + .collect(Collectors.toList()).toArray(new String[] {})); + } + return Option.empty(); + } + + /** + * This function returns the partition fields only. The input partition field would contain partition + * type corresponding to the custom key generator if table version is eight and if custom key + * generator is configured. This function would strip the partition type and return the partition field. + */ + public static String getPartitionFieldWithoutKeyGenPartitionType(String partitionField, HoodieConfig config) { + return getTableVersion(config).greaterThan(HoodieTableVersion.SEVEN) + ? partitionField.split(BaseKeyGenerator.CUSTOM_KEY_GENERATOR_SPLIT_REGEX)[0] + : partitionField; + } + + /** + * This function returns the hoodie.table.version from hoodie.properties file. + */ + public static HoodieTableVersion getTableVersion(HoodieConfig config) { + return HoodieConfig.contains(HoodieTableConfig.VERSION, config) + ? HoodieTableVersion.fromVersionCode(config.getInt(HoodieTableConfig.VERSION)) + : HoodieTableConfig.VERSION.defaultValue(); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index dfdedf8965859..acee42be05552 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieIOException; @@ -131,6 +132,12 @@ public static HoodieTableMetaClient init(String basePath, HoodieFileFormat baseF return init(getDefaultStorageConf(), basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat); } + public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, HoodieTableVersion version) throws IOException { + Properties properties = new Properties(); + properties.setProperty(HoodieTableConfig.VERSION.key(), String.valueOf(version.versionCode())); + return init(getDefaultStorageConf(), basePath, tableType, properties); + } + public static HoodieTableMetaClient init(StorageConfiguration storageConf, String basePath) throws IOException { return init(storageConf, basePath, HoodieTableType.COPY_ON_WRITE); } @@ -181,8 +188,8 @@ public static HoodieTableMetaClient init(StorageConfiguration storageConf, St public static HoodieTableMetaClient init(StorageConfiguration storageConf, String basePath, HoodieTableType tableType, Properties properties, String databaseName) throws IOException { - HoodieTableMetaClient.PropertyBuilder builder = - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.TableBuilder builder = + HoodieTableMetaClient.newTableBuilder() .setDatabaseName(databaseName) .setTableName(RAW_TRIPS_TEST_NAME) .setTableType(tableType) @@ -198,9 +205,8 @@ public static HoodieTableMetaClient init(StorageConfiguration storageConf, St builder.setPartitionFields("some_nonexistent_field"); } - Properties processedProperties = builder.fromProperties(properties).build(); - - return HoodieTableMetaClient.initTableAndGetMetaClient(storageConf.newInstance(), basePath, processedProperties); + return builder.fromProperties(properties) + .initTable(storageConf.newInstance(), basePath); } public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath, HoodieFileFormat baseFileFormat, String keyGenerator) throws IOException { diff --git a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java index cdaf4dfc32f26..24783f3e2fd46 100644 --- a/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java +++ b/hudi-examples/hudi-examples-java/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -82,7 +82,7 @@ public static void main(String[] args) throws Exception { Path path = new Path(tablePath); FileSystem fs = HadoopFSUtils.getFs(tablePath, storageConf); if (!fs.exists(path)) { - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(tableType) .setTableName(tableName) .setPayloadClassName(HoodieAvroPayload.class.getName()) diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java index 3f1b598d11a11..b8df6161ca454 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java @@ -86,7 +86,7 @@ public static void main(String[] args) throws Exception { Path path = new Path(tablePath); FileSystem fs = HadoopFSUtils.getFs(tablePath, jsc.hadoopConfiguration()); if (!fs.exists(path)) { - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(tableType) .setTableName(tableName) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 7ad04317322e6..cc7f5f6783ad8 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -397,6 +397,13 @@ private FlinkOptions() { .defaultValue(WriteOperationType.UPSERT.value()) .withDescription("The write operation, that this write should do"); + @AdvancedConfig + public static final ConfigOption WRITE_TABLE_VERSION = ConfigOptions + .key(HoodieWriteConfig.WRITE_TABLE_VERSION.key()) + .intType() + .defaultValue(HoodieWriteConfig.WRITE_TABLE_VERSION.defaultValue()) + .withDescription("Table version produced by this writer."); + /** * Flag to indicate whether to drop duplicates before insert/upsert. * By default false to gain extra performance. diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index 25ba73f97d3db..4a1ef2e720af2 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hive.MultiPartKeysValueExtractor; @@ -117,6 +118,9 @@ public class FlinkStreamerConfig extends Configuration { + " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record.") public String sourceOrderingField = "ts"; + @Parameter(names = {"--write-table-version"}, description = "Version of table written") + public Integer writeTableVersion = HoodieTableVersion.current().versionCode(); + @Parameter(names = {"--payload-class"}, description = "Subclass of HoodieRecordPayload, that works off " + "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value.") public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); @@ -422,6 +426,7 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setBoolean(FlinkOptions.INSERT_CLUSTER, config.insertCluster); conf.setString(FlinkOptions.OPERATION, config.operation.value()); conf.setString(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField); + conf.setInteger(FlinkOptions.WRITE_TABLE_VERSION, config.writeTableVersion); conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, config.payloadClassName); conf.setString(FlinkOptions.RECORD_MERGER_IMPLS, config.recordMergerImpls); conf.setString(FlinkOptions.RECORD_MERGER_STRATEGY, config.recordMergerStrategy); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java index ffb989b53fdfa..e95c9637fe224 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java @@ -718,7 +718,7 @@ public void renameTable(ObjectPath tablePath, String newTableName, boolean ignor HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(location) .setConf(HadoopFSUtils.getStorageConfWithCopy(hiveConf)).build(); //Init table with new name - HoodieTableMetaClient.withPropertyBuilder().fromProperties(metaClient.getTableConfig().getProps()) + HoodieTableMetaClient.newTableBuilder().fromProperties(metaClient.getTableConfig().getProps()) .setTableName(newTableName) .initTable(HadoopFSUtils.getStorageConfWithCopy(hiveConf), location); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java index 091290801f47d..413a6a1cfac64 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.table.HoodieFlinkTable; @@ -36,6 +37,12 @@ public class FlinkTables { private FlinkTables() { } + private static HoodieFlinkTable createTableInternal(HoodieWriteConfig writeConfig, HoodieEngineContext context) { + HoodieFlinkTable table = HoodieFlinkTable.create(writeConfig, context); + CommonClientUtils.validateTableVersion(table.getMetaClient().getTableConfig(), writeConfig); + return table; + } + /** * Creates the hoodie flink table. * @@ -46,7 +53,7 @@ public static HoodieFlinkTable createTable(Configuration conf, RuntimeContext HadoopFSUtils.getStorageConf(getHadoopConf(conf)), new FlinkTaskContextSupplier(runtimeContext)); HoodieWriteConfig writeConfig = FlinkWriteClients.getHoodieClientConfig(conf, true); - return HoodieFlinkTable.create(writeConfig, context); + return createTableInternal(writeConfig, context); } /** @@ -61,7 +68,7 @@ public static HoodieFlinkTable createTable( HoodieFlinkEngineContext context = new HoodieFlinkEngineContext( HadoopFSUtils.getStorageConfWithCopy(hadoopConf), new FlinkTaskContextSupplier(runtimeContext)); - return HoodieFlinkTable.create(writeConfig, context); + return createTableInternal(writeConfig, context); } /** @@ -71,6 +78,6 @@ public static HoodieFlinkTable createTable( */ public static HoodieFlinkTable createTable(Configuration conf) { HoodieWriteConfig writeConfig = FlinkWriteClients.getHoodieClientConfig(conf, true, false); - return HoodieFlinkTable.create(writeConfig, HoodieFlinkEngineContext.DEFAULT); + return createTableInternal(writeConfig, HoodieFlinkEngineContext.DEFAULT); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java index 5f36cdae42fd6..e9d0310d4756d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java @@ -160,6 +160,7 @@ public static HoodieWriteConfig getHoodieClientConfig( .withEngineType(EngineType.FLINK) .withPath(conf.getString(FlinkOptions.PATH)) .combineInput(conf.getBoolean(FlinkOptions.PRE_COMBINE), true) + .withWriteTableVersion(conf.getInteger(FlinkOptions.WRITE_TABLE_VERSION)) .withMergeAllowDuplicateOnInserts(OptionsResolver.insertClustering(conf)) .withClusteringConfig( HoodieClusteringConfig.newBuilder() diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 883065e9ba80b..9610075f5d4e4 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -249,10 +249,11 @@ public static HoodieTableMetaClient initTableIfNotExists( org.apache.hadoop.conf.Configuration hadoopConf) throws IOException { final String basePath = conf.getString(FlinkOptions.PATH); if (!tableExists(basePath, hadoopConf)) { - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableCreateSchema(conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA)) .setTableType(conf.getString(FlinkOptions.TABLE_TYPE)) .setTableName(conf.getString(FlinkOptions.TABLE_NAME)) + .setTableVersion(conf.getInteger(FlinkOptions.WRITE_TABLE_VERSION)) .setDatabaseName(conf.getString(FlinkOptions.DATABASE_NAME)) .setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD, null)) .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index 825a82d983a0b..baa494c8422d9 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -190,8 +190,7 @@ public void testHoodieFlinkCompactorWithUpgradeAndDowngrade(boolean upgrade) thr // Create hoodie table and insert into data. EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); TableEnvironment tableEnv = TableEnvironmentImpl.create(settings); - tableEnv.getConfig().getConfiguration() - .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 4); + tableEnv.getConfig().getConfiguration().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 4); Map options = new HashMap<>(); options.put(FlinkOptions.COMPACTION_SCHEDULE_ENABLED.key(), "false"); options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false"); @@ -215,7 +214,7 @@ public void testHoodieFlinkCompactorWithUpgradeAndDowngrade(boolean upgrade) thr HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); // set the table name - conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + conf.set(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); // set table schema CompactionUtil.setAvroSchema(conf, metaClient); @@ -230,11 +229,11 @@ public void testHoodieFlinkCompactorWithUpgradeAndDowngrade(boolean upgrade) thr // try to upgrade or downgrade if (upgrade) { - metaClient.getTableConfig().setTableVersion(HoodieTableVersion.FIVE); - new UpgradeDowngrade(metaClient, writeClient.getConfig(), writeClient.getEngineContext(), FlinkUpgradeDowngradeHelper.getInstance()).run(HoodieTableVersion.SIX, "none"); - } else { metaClient.getTableConfig().setTableVersion(HoodieTableVersion.SIX); - new UpgradeDowngrade(metaClient, writeClient.getConfig(), writeClient.getEngineContext(), FlinkUpgradeDowngradeHelper.getInstance()).run(HoodieTableVersion.FIVE, "none"); + new UpgradeDowngrade(metaClient, writeClient.getConfig(), writeClient.getEngineContext(), FlinkUpgradeDowngradeHelper.getInstance()).run(HoodieTableVersion.EIGHT, "none"); + } else { + metaClient.getTableConfig().setTableVersion(HoodieTableVersion.EIGHT); + new UpgradeDowngrade(metaClient, writeClient.getConfig(), writeClient.getEngineContext(), FlinkUpgradeDowngradeHelper.getInstance()).run(HoodieTableVersion.SIX, "none"); } // generate compaction plan @@ -246,6 +245,7 @@ public void testHoodieFlinkCompactorWithUpgradeAndDowngrade(boolean upgrade) thr // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); + conf.set(FlinkOptions.WRITE_TABLE_VERSION, upgrade ? HoodieTableVersion.EIGHT.versionCode() : HoodieTableVersion.SIX.versionCode()); env.addSource(new CompactionPlanSourceFunction(Collections.singletonList(Pair.of(compactionInstantTime, compactionPlan)), conf)) .name("compaction_source") .uid("uid_compaction_source") diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java index 99ea23b7bca91..73e76b8189bdd 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.configuration.FlinkOptions; @@ -42,6 +43,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -53,13 +55,29 @@ public class TestStreamerUtil { @TempDir File tempFile; + @Test + void testInitTableWithSpecificVersion() throws IOException { + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); + + // Test for partitioned table. + conf.set(FlinkOptions.PARTITION_PATH_FIELD, "p0,p1"); + conf.set(FlinkOptions.WRITE_TABLE_VERSION, HoodieTableVersion.SIX.versionCode()); + StreamerUtil.initTableIfNotExists(conf); + + // Validate the partition fields & preCombineField in hoodie.properties. + HoodieTableMetaClient metaClient1 = HoodieTestUtils.createMetaClient(tempFile.getAbsolutePath()); + assertArrayEquals(metaClient1.getTableConfig().getPartitionFields().get(), new String[] {"p0", "p1"}); + assertNull(metaClient1.getTableConfig().getKeyGeneratorClassName()); + assertEquals(HoodieTableVersion.SIX, metaClient1.getTableConfig().getTableVersion()); + } + @Test void testInitTableIfNotExists() throws IOException { Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); // Test for partitioned table. - conf.setString(FlinkOptions.PRECOMBINE_FIELD, "ts"); - conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "p0,p1"); + conf.set(FlinkOptions.PRECOMBINE_FIELD, "ts"); + conf.set(FlinkOptions.PARTITION_PATH_FIELD, "p0,p1"); StreamerUtil.initTableIfNotExists(conf); // Validate the partition fields & preCombineField in hoodie.properties. @@ -69,6 +87,7 @@ void testInitTableIfNotExists() throws IOException { assertArrayEquals(metaClient1.getTableConfig().getPartitionFields().get(), new String[] {"p0", "p1"}); assertEquals(metaClient1.getTableConfig().getPreCombineField(), "ts"); assertEquals(metaClient1.getTableConfig().getKeyGeneratorClassName(), SimpleAvroKeyGenerator.class.getName()); + assertEquals(HoodieTableVersion.EIGHT, metaClient1.getTableConfig().getTableVersion()); // Test for non-partitioned table. conf.removeConfig(FlinkOptions.PARTITION_PATH_FIELD); diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java index f1166c531cc80..29c88b32b87f5 100644 --- a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestHoodieBigQuerySyncClient.java @@ -78,7 +78,7 @@ public class TestHoodieBigQuerySyncClient { @BeforeAll static void setupOnce() throws Exception { basePath = tempDir.toString(); - metaClient = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(TEST_TABLE) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java index 89bb52a0765c0..3026394011d4f 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -78,7 +78,7 @@ public void setUp() throws IOException { HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); HoodieStorage storage = new HoodieHadoopStorage(fs); - metaClient.setHoodieStorage(storage); + metaClient.setStorage(storage); } // Test the scenario that fs keeps retrying until it fails. @@ -92,7 +92,7 @@ public void testProcessFilesWithExceptions() throws Exception { HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); HoodieStorage storage = new HoodieHadoopStorage(fs); - metaClient.setHoodieStorage(storage); + metaClient.setStorage(storage); List folders = Arrays.asList("2016/04/15", ".hoodie/.temp/2/2016/04/15"); folders.forEach(f -> assertThrows(RuntimeException.class, () -> metaClient.getStorage() diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java index 1503628d14f54..53ab70c4ee19a 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.CollectionUtils; @@ -37,6 +39,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Arrays; +import java.util.List; import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -44,11 +47,13 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGE_MODE; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_CHECKSUM; import static org.apache.hudi.common.util.ConfigUtils.recoverIfNeeded; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -236,4 +241,43 @@ public void testPartitionFieldAPIs(String partitionFields) { assertArrayEquals(new String[] {"p1", "p2"}, HoodieTableConfig.getPartitionFields(config).get()); assertEquals("p1", HoodieTableConfig.getPartitionFieldWithoutKeyGenPartitionType(partitionFields.split(",")[0], config)); } + + @Test + public void testValidateConfigVersion() { + assertTrue(HoodieTableConfig.validateConfigVersion(HoodieTableConfig.INITIAL_VERSION, HoodieTableVersion.EIGHT)); + assertTrue(HoodieTableConfig.validateConfigVersion(ConfigProperty.key("").noDefaultValue().withDocumentation(""), + HoodieTableVersion.SIX)); + assertFalse(HoodieTableConfig.validateConfigVersion(HoodieTableConfig.INITIAL_VERSION, HoodieTableVersion.SIX)); + } + + @Test + public void testDropInvalidConfigs() { + // test invalid configs are dropped + HoodieConfig config = new HoodieConfig(); + config.setValue(HoodieTableConfig.VERSION, String.valueOf(HoodieTableVersion.SIX.versionCode())); + config.setValue(HoodieTableConfig.INITIAL_VERSION, String.valueOf(HoodieTableVersion.EIGHT.versionCode())); + config.setValue(RECORD_MERGE_MODE, RECORD_MERGE_MODE.defaultValue()); + + HoodieTableConfig.dropInvalidConfigs(config); + assertTrue(config.contains(HoodieTableConfig.VERSION)); + assertFalse(config.contains(HoodieTableConfig.INITIAL_VERSION)); + assertFalse(config.contains(RECORD_MERGE_MODE)); + + // test valid ones are not dropped + config = new HoodieConfig(); + config.setValue(HoodieTableConfig.VERSION, String.valueOf(HoodieTableVersion.EIGHT.versionCode())); + config.setValue(RECORD_MERGE_MODE, RECORD_MERGE_MODE.defaultValue()); + HoodieTableConfig.dropInvalidConfigs(config); + assertTrue(config.contains(RECORD_MERGE_MODE)); + } + + @Test + public void testDefinedTableConfigs() { + List> configProperties = HoodieTableConfig.definedTableConfigs(); + assertEquals(37, configProperties.size()); + configProperties.forEach(c -> { + assertNotNull(c); + assertFalse(c.doc().isEmpty()); + }); + } } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java index a85dfbb1fe0f6..4569b3eb9dd6d 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java @@ -30,7 +30,9 @@ import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.storage.StoragePath; +import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -39,6 +41,7 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; +import java.util.Properties; import java.util.stream.Stream; import static org.apache.hudi.common.model.HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; @@ -81,7 +84,7 @@ public void checkMetadata() { } @Test - public void checkSerDe() { + public void testSerDe() { // check if this object is serialized and de-serialized, we are able to read from the file system HoodieTableMetaClient deserializedMetaClient = HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class); @@ -99,7 +102,7 @@ public void checkSerDe() { } @Test - public void checkCommitTimeline() { + public void testCommitTimeline() { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieTimeline activeCommitTimeline = activeTimeline.getCommitAndReplaceTimeline(); assertTrue(activeCommitTimeline.empty(), "Should be empty commit timeline"); @@ -186,7 +189,7 @@ public void testInferRecordMergeMode(Option payloadClassName, Option payloadType, Option recordMergerStrategy, RecordMergeMode expectedRecordMergeMode) { - HoodieTableMetaClient.PropertyBuilder builder = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.TableBuilder builder = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.MERGE_ON_READ.name()) .setTableName("table_name"); if (payloadClassName.isPresent()) { @@ -235,7 +238,7 @@ public void testValidationFailureOnMergeConfigs(Option payloadClassName, Option recordMergerStrategy, RecordMergeMode recordMergeMode, String expectedErrorMessage) { - HoodieTableMetaClient.PropertyBuilder builder = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.TableBuilder builder = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.MERGE_ON_READ.name()) .setTableName("table_name") .setRecordMergeMode(recordMergeMode); @@ -257,7 +260,6 @@ public void testValidationFailureOnMergeConfigs(Option payloadClassName, public void testEquals() throws IOException { HoodieTableMetaClient metaClient1 = HoodieTestUtils.init(tempDir.toAbsolutePath().toString(), getTableType()); HoodieTableMetaClient metaClient2 = HoodieTestUtils.init(tempDir.toAbsolutePath().toString(), getTableType()); - assertEquals(metaClient1, metaClient1); assertEquals(metaClient1, metaClient2); assertNotEquals(metaClient1, null); assertNotEquals(metaClient1, new Object()); @@ -270,4 +272,102 @@ public void testToString() throws IOException { assertEquals(metaClient1.toString(), metaClient2.toString()); assertNotEquals(metaClient1.toString(), new Object().toString()); } + + @Test + public void testTableVersion() throws IOException { + final String basePath = tempDir.toAbsolutePath() + Path.SEPARATOR + "t1"; + HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.newTableBuilder() + .setTableType(HoodieTableType.MERGE_ON_READ.name()) + .setTableName("table-version-test") + .setTableVersion(HoodieTableVersion.SIX.versionCode()) + .initTable(this.metaClient.getStorageConf(), basePath); + assertEquals(HoodieTableVersion.SIX, metaClient1.getTableConfig().getTableVersion()); + + HoodieTableMetaClient metaClient2 = HoodieTableMetaClient.builder() + .setConf(this.metaClient.getStorageConf()) + .setBasePath(basePath) + .build(); + assertEquals(HoodieTableVersion.SIX, metaClient2.getTableConfig().getTableVersion()); + } + + @Test + public void testGenerateFromAnotherMetaClient() throws IOException { + final String basePath1 = tempDir.toAbsolutePath().toString() + Path.SEPARATOR + "t2A"; + final String basePath2 = tempDir.toAbsolutePath().toString() + Path.SEPARATOR + "t2B"; + + HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.newTableBuilder() + .setTableType(HoodieTableType.MERGE_ON_READ.name()) + .setTableName("table-version-test") + .setTableVersion(HoodieTableVersion.SIX.versionCode()) + .initTable(this.metaClient.getStorageConf(), basePath1); + + HoodieTableMetaClient metaClient2 = HoodieTableMetaClient.newTableBuilder() + .fromMetaClient(metaClient1) + .initTable(this.metaClient.getStorageConf(), basePath2); + + assertEquals(metaClient1.getTableConfig().getTableType(), metaClient2.getTableConfig().getTableType()); + assertEquals(metaClient1.getTableConfig().getTableVersion(), metaClient2.getTableConfig().getTableVersion()); + assertEquals(metaClient1.getTableConfig().getTableName(), metaClient2.getTableConfig().getTableName()); + } + + @Test + public void testTableBuilderRequiresTableNameAndType() { + assertThrows(IllegalArgumentException.class, () -> { + HoodieTableMetaClient.builder() + .setConf(this.metaClient.getStorageConf()) + .build(); + }); + assertThrows(IllegalArgumentException.class, () -> { + HoodieTableMetaClient.newTableBuilder() + .setTableName("test-table") + .initTable(this.metaClient.getStorageConf(), tempDir.toAbsolutePath().toString() + Path.SEPARATOR + "failing2"); + }); + assertThrows(IllegalArgumentException.class, () -> { + HoodieTableMetaClient.newTableBuilder() + .setTableType(HoodieTableType.COPY_ON_WRITE.name()) + .initTable(this.metaClient.getStorageConf(), tempDir.toAbsolutePath().toString() + Path.SEPARATOR + "failing3"); + }); + } + + @Test + public void testCreateMetaClientFromProperties() throws IOException { + final String basePath = tempDir.toAbsolutePath().toString() + Path.SEPARATOR + "t5"; + Properties props = new Properties(); + props.setProperty(HoodieTableConfig.NAME.key(), "test-table"); + props.setProperty(HoodieTableConfig.TYPE.key(), HoodieTableType.COPY_ON_WRITE.name()); + props.setProperty(HoodieTableConfig.PRECOMBINE_FIELD.key(), "timestamp"); + + HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.newTableBuilder() + .fromProperties(props) + .initTable(this.metaClient.getStorageConf(),basePath); + + HoodieTableMetaClient metaClient2 = HoodieTableMetaClient.builder() + .setConf(this.metaClient.getStorageConf()) + .setBasePath(basePath) + .build(); + + // test table name and type and precombine field also match + assertEquals(metaClient1.getTableConfig().getTableName(), metaClient2.getTableConfig().getTableName()); + assertEquals(metaClient1.getTableConfig().getTableType(), metaClient2.getTableConfig().getTableType()); + assertEquals(metaClient1.getTableConfig().getPreCombineField(), metaClient2.getTableConfig().getPreCombineField()); + // default table version should be current version + assertEquals(HoodieTableVersion.current(), metaClient2.getTableConfig().getTableVersion()); + } + + @Test + public void testCreateLayoutInStorage() throws IOException { + final String basePath = tempDir.toAbsolutePath().toString() + Path.SEPARATOR + "t6"; + HoodieTableMetaClient metaClient1 = HoodieTableMetaClient.newTableBuilder() + .setTableType(HoodieTableType.COPY_ON_WRITE.name()) + .setTableName("table-layout-test") + .initTable(this.metaClient.getStorageConf(), basePath); + + // test the folder structure + this.metaClient.getRawStorage().exists(new StoragePath(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME)); + this.metaClient.getRawStorage().exists(new StoragePath(basePath, HoodieTableMetaClient.METAFOLDER_NAME)); + this.metaClient.getRawStorage().exists(new StoragePath(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)); + this.metaClient.getRawStorage().exists(new StoragePath(basePath, HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue())); + this.metaClient.getRawStorage().exists(new StoragePath(basePath, HoodieTableMetaClient.METAFOLDER_NAME + + Path.SEPARATOR + "hoodie.properties")); + } } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 1e68f399cc435..f1645cd061709 100755 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -138,7 +138,7 @@ public void testLoadingInstantsFromFiles() throws IOException { "Check the instants stream"); // Backwards compatibility testing for reading compaction plans - metaClient = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) .setTimelineLayoutVersion(VERSION_0) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); @@ -773,11 +773,11 @@ private void shouldAllowTempCommit(boolean allowTempCommit, Consumer context) { ValidationUtils.checkArgument(Objects.equals(basePath, metaClient.getBasePath().toString())); ValidationUtils.checkArgument(Objects.equals( - storage.getFileSystem(), metaClient.getRawHoodieStorage().getFileSystem())); + storage.getFileSystem(), metaClient.getRawStorage().getFileSystem())); this.basePath = basePath; this.storage = storage; this.fs = (FileSystem) storage.getFileSystem(); @@ -180,7 +180,7 @@ protected HoodieTestTable(String basePath, HoodieStorage storage, public static HoodieTestTable of(HoodieTableMetaClient metaClient) { testTableState = HoodieTestTableState.of(); - return new HoodieTestTable(metaClient.getBasePath().toString(), metaClient.getRawHoodieStorage(), metaClient); + return new HoodieTestTable(metaClient.getBasePath().toString(), metaClient.getRawStorage(), metaClient); } public void setNonPartitioned() { diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java index e97869d2f04c4..7f009306a6d4f 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java @@ -163,8 +163,9 @@ static HoodieTableMetaClient initTableType(Configuration hadoopConf, String base properties.setProperty(HoodieTableConfig.TYPE.key(), tableType.name()); properties.setProperty(HoodieTableConfig.PAYLOAD_CLASS_NAME.key(), HoodieAvroPayload.class.getName()); properties.setProperty(HoodieTableConfig.RECORD_MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID); - return HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConfWithCopy(hadoopConf), basePath, properties); + return HoodieTableMetaClient.newTableBuilder() + .fromProperties(properties) + .initTable(HadoopFSUtils.getStorageConfWithCopy(hadoopConf), basePath); } static List generatePartitions(DistributedFileSystem dfs, String basePath) diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index 70910357d7d7e..38549b10cd399 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -121,7 +121,7 @@ public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc, boole (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); if (!fs.exists(new Path(cfg.targetBasePath))) { - metaClient = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(cfg.tableType) .setTableName(cfg.targetTableName) .setRecordKeyFields(this.props.getString(DataSourceWriteOptions.RECORDKEY_FIELD().key())) diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java index 4ab2c5eae5305..082016ea6a432 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java @@ -94,12 +94,13 @@ public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throw LOG.info(String.format("Setting record key %s and partition fields %s for table %s", recordKeyFields, partitionColumns, tableBasePath + tableName)); - tableMetaClient = Option.of(HoodieTableMetaClient.withPropertyBuilder() + tableMetaClient = Option.of(HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE.name()) .setTableName(tableName) .setPayloadClassName(HoodieAvroPayload.class.getName()) .setRecordKeyFields(recordKeyFields) .setPartitionFields(partitionColumns) + .setTableVersion(writeConfig.getWriteVersion()) .setKeyGeneratorClassProp(writeConfig.getKeyGeneratorClass()) .fromProperties(connectConfigs.getProps()) .initTable(storageConf.newInstance(), tableBasePath)); diff --git a/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestKafkaConnectTransactionServices.java b/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestKafkaConnectTransactionServices.java new file mode 100644 index 0000000000000..b561104c6a96d --- /dev/null +++ b/hudi-kafka-connect/src/test/java/org/apache/hudi/writers/TestKafkaConnectTransactionServices.java @@ -0,0 +1,61 @@ +/* + * + * * 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.writers; + +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.connect.writers.KafkaConnectConfigs; +import org.apache.hudi.connect.writers.KafkaConnectTransactionServices; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestKafkaConnectTransactionServices { + + @TempDir + Path path; + + @Test + public void testTableCreation() { + Properties props = new Properties(); + props.put("hoodie.table.name", "test"); + props.put("hoodie.base.path", path); + props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition_path"); + props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "key"); + props.put("hoodie.table.version", "6"); + KafkaConnectConfigs configs = KafkaConnectConfigs.newBuilder() + .withProperties(props) + .build(); + new KafkaConnectTransactionServices(configs); + + HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(path.toFile().getPath()); + assertEquals(HoodieTableVersion.SIX, metaClient.getTableConfig().getTableVersion()); + assertEquals(HoodieTableType.COPY_ON_WRITE, metaClient.getTableConfig().getTableType()); + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index cb427aebedbd9..d37dd7f7c2ae6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -70,7 +70,7 @@ public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writ this.metaClient = HoodieTableMetaClient.builder() .setConf(storageConf.newInstance()).setBasePath(writeConfig.getBasePath()).build(); - this.metaClient.validateTableProperties(writeConfig.getProps()); + this.writeClient.validateAgainstTableProperties(this.metaClient.getTableConfig(), writeConfig); this.writeClient.preWrite(instantTime, WriteOperationType.BULK_INSERT, metaClient); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 48d8dec474ab9..3ac47a76e5983 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -48,7 +48,7 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, T import org.apache.hudi.common.util.ConfigUtils.getAllConfigKeys import org.apache.hudi.common.util.{CommitUtils, StringUtils, Option => HOption} import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} -import org.apache.hudi.config.HoodieWriteConfig.SPARK_SQL_MERGE_INTO_PREPPED_KEY +import org.apache.hudi.config.HoodieWriteConfig.{SPARK_SQL_MERGE_INTO_PREPPED_KEY, WRITE_TABLE_VERSION} import org.apache.hudi.config.{HoodieCompactionConfig, HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.exception.{HoodieException, HoodieRecordCreationException, HoodieWriteConflictException} import org.apache.hudi.hadoop.fs.HadoopFSUtils @@ -301,7 +301,7 @@ class HoodieSparkSqlWriterInternal { if (StringUtils.nonEmpty(hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME))) hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME) else KeyGeneratorType.getKeyGeneratorClassName(hoodieConfig) - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(tableType) .setDatabaseName(databaseName) .setTableName(tblName) @@ -746,10 +746,11 @@ class HoodieSparkSqlWriterInternal { String.valueOf(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue()) )) - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.valueOf(tableType)) .setTableName(tableName) .setRecordKeyFields(recordKeyFields) + .setTableVersion(hoodieConfig.getIntOrDefault(WRITE_TABLE_VERSION)) .setArchiveLogFolder(archiveLogFolder) .setPayloadClassName(payloadClass) .setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null)) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index 29c06eed64717..71580250d7da5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -205,7 +205,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten "The database names from this hoodie path and this catalog table is not same.") val recordName = AvroSchemaUtils.getAvroRecordQualifiedName(table.identifier.table) // just persist hoodie.table.create.schema - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .fromProperties(properties) .setDatabaseName(catalogDatabaseName) .setTableCreateSchema(SchemaConverters.toAvroType(dataSchema, recordName = recordName).toString()) @@ -221,7 +221,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten table.partitionColumnNames.mkString(",") } - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .fromProperties(properties) .setDatabaseName(catalogDatabaseName) .setTableName(table.identifier.table) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala index 84ca4c3c00b18..af41e81258b18 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala @@ -38,7 +38,7 @@ case class AlterHoodieTableRenameCommand( val hoodieCatalogTable = HoodieCatalogTable(sparkSession, oldName) // Init table with new name. - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .fromProperties(hoodieCatalogTable.tableConfig.getProps) .setTableName(newName.table) .initTable( diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala index 06450e60ce31a..48abc91e80338 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala @@ -72,7 +72,7 @@ case class TruncateHoodieTableCommand( FSUtils.deleteDir(engineContext, storage, targetPath, sparkSession.sparkContext.defaultParallelism) // ReInit hoodie.properties - val metaClient = HoodieTableMetaClient.withPropertyBuilder() + val metaClient = HoodieTableMetaClient.newTableBuilder() .fromProperties(properties) .initTable( HadoopFSUtils.getStorageConf(sparkSession.sessionState.newHadoopConf), diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java index 09afc65019152..8ffb2991386c2 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java @@ -27,6 +27,7 @@ import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.util.CommonClientUtils; import org.apache.spark.api.java.JavaSparkContext; import org.slf4j.Logger; @@ -44,11 +45,11 @@ private ArchiveExecutorUtils() { } public static int archive(JavaSparkContext jsc, - int minCommits, - int maxCommits, - int commitsRetained, - boolean enableMetadata, - String basePath) throws IOException { + int minCommits, + int maxCommits, + int commitsRetained, + boolean enableMetadata, + String basePath) throws IOException { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(minCommits, maxCommits).build()) .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(commitsRetained).build()) @@ -57,6 +58,7 @@ public static int archive(JavaSparkContext jsc, .build(); HoodieEngineContext context = new HoodieSparkEngineContext(jsc); HoodieSparkTable table = HoodieSparkTable.create(config, context); + CommonClientUtils.validateTableVersion(table.getMetaClient().getTableConfig(), config); try { HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table); archiver.archiveIfRequired(context, true); diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java index a886aedc61c92..cf970319f09a4 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java @@ -234,11 +234,12 @@ private void initializeTable() throws IOException { Map timestampKeyGeneratorConfigs = extractConfigsRelatedToTimestampBasedKeyGenerator(keyGenClassAndParColsForKeyGenerator.getLeft(), props); - HoodieTableMetaClient.PropertyBuilder builder = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.TableBuilder builder = HoodieTableMetaClient.newTableBuilder() .fromProperties(props) .setTableType(cfg.tableType) .setDatabaseName(cfg.database) .setTableName(cfg.tableName) + .setTableVersion(bootstrapConfig.getWriteVersion()) .setRecordKeyFields(props.getString(RECORDKEY_FIELD_NAME.key())) .setPreCombineField(props.getString(PRECOMBINE_FIELD_NAME.key(), null)) .setPopulateMetaFields(props.getBoolean( diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/HDFSParquetImporterUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/HDFSParquetImporterUtils.java index 839b5e0d9de5f..0c9c12298cdc1 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/HDFSParquetImporterUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/HDFSParquetImporterUtils.java @@ -68,7 +68,6 @@ import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.List; -import java.util.Properties; import scala.Tuple2; @@ -156,12 +155,10 @@ public int dataImport(JavaSparkContext jsc, FileSystem fs) { if (!fs.exists(new Path(this.targetPath))) { // Initialize target hoodie table. - Properties properties = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableName(this.tableName) .setTableType(this.tableType) - .build(); - HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), this.targetPath, properties); + .initTable(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), this.targetPath); } // Get schema. diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java index 112ff1721f8b6..a963a1d70a3a7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java @@ -24,6 +24,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; @@ -91,7 +92,10 @@ public void setUp() throws Exception { Properties props = new Properties(); props.putAll(options); try { - metaClient = HoodieTableMetaClient.initTableAndGetMetaClient(storageConf.newInstance(), basePath.toString(), props); + metaClient = HoodieTableMetaClient.newTableBuilder() + .fromProperties(props) + .setTableType(HoodieTableType.MERGE_ON_READ.name()) + .initTable(storageConf.newInstance(), basePath.toString()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 6de7a53e137fa..fc52ff685f2e8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -157,11 +157,12 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS @Test def testPartitionSchemaWithoutKeyGenerator(): Unit = { val metaClient = HoodieTestUtils.init( - storageConf, basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableMetaClient.withPropertyBuilder() + storageConf, basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableMetaClient.newTableBuilder() .fromMetaClient(this.metaClient) .setRecordKeyFields("_row_key") .setPartitionFields("partition_path") - .setTableName("hoodie_test").build()) + .setTableName("hoodie_test") + .build()) val props = Map( "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4", diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index 07b2236f303fd..7fc31e6cf7b5d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -573,7 +573,7 @@ def testBulkInsertForDropPartitionColumn(): Unit = { // when metadata is enabled, directly instantiating write client using DataSourceUtils.createHoodieClient // will hit a code which tries to instantiate meta client for data table. if table does not exist, it fails. // hence doing an explicit instantiation here. - val tableMetaClientBuilder = HoodieTableMetaClient.withPropertyBuilder() + val tableMetaClientBuilder = HoodieTableMetaClient.newTableBuilder() .setTableType(tableType) .setTableName(hoodieFooTableName) .setRecordKeyFields(fooTableParams(DataSourceWriteOptions.RECORDKEY_FIELD.key)) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestMultipleTableVersionWriting.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestMultipleTableVersionWriting.scala new file mode 100644 index 0000000000000..15988d214d79d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestMultipleTableVersionWriting.scala @@ -0,0 +1,72 @@ +/* + * + * * 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 + +import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.table.HoodieTableVersion +import org.apache.hudi.common.testutils.HoodieTestUtils +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.exception.{HoodieException, HoodieNotSupportedException} +import org.apache.spark.sql.SaveMode +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test +import org.scalatest.Assertions.assertThrows + +class TestMultipleTableVersionWriting extends HoodieSparkWriterTestBase { + + @Test + def testTableVersionAndWriteVersionMatching(): Unit = { + val basePath = s"${tempBasePath}/tbl_1"; + val df = spark.range(1).selectExpr("1 as id", "1 as name", "1 as partition") + + // write table with current version + df.write.format("hudi") + .option(HoodieWriteConfig.TBL_NAME.key, "tbl_1") + .mode(SaveMode.Overwrite) + .save(basePath) + + val metaClient = HoodieTestUtils.createMetaClient(basePath); + assertEquals(HoodieTableVersion.current().versionCode(), + metaClient.getTableConfig.getTableVersion.versionCode()) + + // should error out when writing with lower write version. + assertThrows[HoodieException] { + df.write.format("hudi") + .option(HoodieWriteConfig.AUTO_UPGRADE_VERSION.key(), "false") + .option(HoodieWriteConfig.WRITE_TABLE_VERSION.key, HoodieTableVersion.SIX.versionCode()) + .mode(SaveMode.Append) + .save(basePath) + } + } + + @Test + def testThrowsExceptionForIncompatibleTableVersion(): Unit = { + val basePath = s"${tempBasePath}/tbl_2"; + HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableVersion.SIX); + + val df = spark.range(1).selectExpr("1 as id", "1 as name", "1 as partition") + assertThrows[HoodieNotSupportedException] { + df.write.format("hudi") + .mode(SaveMode.Append) + .save(basePath) + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala index 5e28ea830d3d7..392a5cb12661c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamSourceReadByStateTransitionTime.scala @@ -44,7 +44,7 @@ class TestStreamSourceReadByStateTransitionTime extends TestStreamingSource { HoodieTableType.values().foreach { tableType => withTempDir { inputDir => val tablePath = s"${inputDir.getCanonicalPath}/test_stream_${tableType.name()}" - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(tableType) .setTableName(s"test_stream_${tableType.name()}") .setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala index b9ac64f2e10f0..ab60e64e58d10 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala @@ -60,7 +60,7 @@ class TestStreamingSource extends StreamTest { test("test cow stream source") { withTempDir { inputDir => val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream" - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(COPY_ON_WRITE) .setTableName(getTableName(tablePath)) .setRecordKeyFields("id") @@ -113,7 +113,7 @@ class TestStreamingSource extends StreamTest { test("test mor stream source") { withTempDir { inputDir => val tablePath = s"${inputDir.getCanonicalPath}/test_mor_stream" - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(MERGE_ON_READ) .setTableName(getTableName(tablePath)) .setRecordKeyFields("id") @@ -160,7 +160,7 @@ class TestStreamingSource extends StreamTest { test("Test cow from latest offset") { withTempDir { inputDir => val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream" - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(COPY_ON_WRITE) .setTableName(getTableName(tablePath)) .setRecordKeyFields("id") @@ -193,7 +193,7 @@ class TestStreamingSource extends StreamTest { test("Test cow from specified offset") { withTempDir { inputDir => val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream" - val metaClient = HoodieTableMetaClient.withPropertyBuilder() + val metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(COPY_ON_WRITE) .setTableName(getTableName(tablePath)) .setRecordKeyFields("id") @@ -230,7 +230,7 @@ class TestStreamingSource extends StreamTest { test("test mor stream source with compaction") { withTempDir { inputDir => val tablePath = s"${inputDir.getCanonicalPath}/test_mor_stream" - val metaClient = HoodieTableMetaClient.withPropertyBuilder() + val metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(MERGE_ON_READ) .setTableName(getTableName(tablePath)) .setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue) diff --git a/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/TestDataHubSyncClient.java b/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/TestDataHubSyncClient.java index bbc998d1c9fee..a19e035ad1d0a 100644 --- a/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/TestDataHubSyncClient.java +++ b/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/TestDataHubSyncClient.java @@ -19,6 +19,7 @@ package org.apache.hudi.sync.datahub; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.sync.datahub.config.DataHubSyncConfig; @@ -71,8 +72,10 @@ public static void beforeAll() throws IOException { Properties props = new Properties(); props.put("hoodie.table.name", "some_table"); tableBasePath = Paths.get(tmpDir.toString(), "some_table").toString(); - HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConf(new Configuration()), tableBasePath, props); + HoodieTableMetaClient.newTableBuilder() + .fromProperties(props) + .setTableType(HoodieTableType.MERGE_ON_READ.name()) + .initTable(HadoopFSUtils.getStorageConf(new Configuration()), tableBasePath); } @BeforeEach diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java index 009401dca951e..7fbf61ba376e5 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java @@ -108,7 +108,7 @@ public HiveSyncConfig hiveSyncConf() throws IOException { } public HoodieHiveSyncClient hiveClient(HiveSyncConfig hiveSyncConfig) throws IOException { - HoodieTableMetaClient metaClient = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(hiveSyncConfig.getString(META_SYNC_TABLE_NAME)) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java index 79b37845a13d6..fbf4ff522d55c 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java @@ -161,7 +161,7 @@ public void createCOWTable(String commitTime, int numberOfPartitions, String dbN String tablePathStr = tablePath(dbName, tableName); Path path = new Path(tablePathStr); FileIOUtils.deleteDirectory(new File(path.toString())); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(tableName) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 38892abf9598f..309c55236cdf0 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -185,7 +185,7 @@ public static void setUp(Option hiveSyncProperties, boolean sho public static void clear() throws IOException, HiveException, MetaException { fileSystem.delete(new Path(basePath), true); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) @@ -281,7 +281,7 @@ public static void createCOWTable(String instantTime, int numberOfPartitions, bo if (fileSystem.exists(path)) { fileSystem.delete(path, true); } - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(tableName) .setPayloadClass(HoodieAvroPayload.class) @@ -381,7 +381,7 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil throws IOException, URISyntaxException { Path path = new Path(basePath); FileIOUtils.deleteDirectory(new File(basePath)); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) @@ -418,7 +418,7 @@ public static void createMORTable(String commitTime, String deltaCommitTime, int throws IOException, URISyntaxException, InterruptedException { Path path = new Path(basePath); FileIOUtils.deleteDirectory(new File(basePath)); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index 1dc24fd31b8ba..05b902087a660 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -60,7 +61,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Properties; import scala.Tuple2; @@ -139,12 +139,11 @@ protected int dataImport(JavaSparkContext jsc) throws IOException { if (!fs.exists(new Path(cfg.targetPath))) { // Initialize target hoodie table. - Properties properties = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableName(cfg.tableName) .setTableType(cfg.tableType) - .build(); - HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), cfg.targetPath, properties); + .setTableVersion(cfg.tableVersion) + .initTable(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), cfg.targetPath); } // Get schema. @@ -267,6 +266,8 @@ public static class Config implements Serializable { public String tableName = null; @Parameter(names = {"--table-type", "-tt"}, description = "Table type", required = true) public String tableType = null; + @Parameter(names = {"--table-version", "-tv"}, description = "Table version") + public int tableVersion = HoodieTableVersion.current().versionCode(); @Parameter(names = {"--row-key-field", "-rk"}, description = "Row key field name", required = true) public String rowKey = null; @Parameter(names = {"--partition-key-field", "-pk"}, description = "Partition key field name", required = true) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 82acb4524c5e8..99ce6e9e60c78 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -261,6 +261,7 @@ private int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { if (cfg.retryLastFailedClusteringJob) { HoodieSparkTable table = HoodieSparkTable.create(client.getConfig(), client.getEngineContext()); + client.validateAgainstTableProperties(table.getMetaClient().getTableConfig(), client.getConfig()); Option lastClusterOpt = table.getActiveTimeline().getLastPendingClusterInstant(); if (lastClusterOpt.isPresent()) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java index 8de01545354e9..d8efea97bff15 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java @@ -323,6 +323,7 @@ public void run() { public void dryRun() { try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, Option.empty(), props)) { HoodieSparkTable table = HoodieSparkTable.create(client.getConfig(), client.getEngineContext()); + client.validateAgainstTableProperties(table.getMetaClient().getTableConfig(), client.getConfig()); List parts = Arrays.asList(cfg.partitions.split(",")); Map> partitionToReplaceFileIds = jsc.parallelize(parts, parts.size()).distinct() .mapToPair(partitionPath -> new Tuple2<>(partitionPath, table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()))) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java index dd1386c593e16..51a43eb2db4f3 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java @@ -59,6 +59,7 @@ import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_TIMEZONE; import static org.apache.hudi.config.HoodieWriteConfig.PRECOMBINE_FIELD_NAME; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_TABLE_VERSION; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC; import static org.apache.hudi.keygen.constant.KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE; @@ -204,13 +205,14 @@ private void initializeTable() throws IOException { throw new IllegalArgumentException("Bootstrap source base path and Hudi table base path must be different"); } - HoodieTableMetaClient.PropertyBuilder builder = HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.TableBuilder builder = HoodieTableMetaClient.newTableBuilder() .fromProperties(props) .setTableType(cfg.tableType) .setTableName(cfg.targetTableName) .setRecordKeyFields(props.getString(RECORDKEY_FIELD_NAME.key())) .setPreCombineField(props.getString( PRECOMBINE_FIELD_NAME.key(), PRECOMBINE_FIELD_NAME.defaultValue())) + .setTableVersion(props.getInteger(WRITE_TABLE_VERSION.key(), WRITE_TABLE_VERSION.defaultValue())) .setPopulateMetaFields(props.getBoolean( POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue())) .setArchiveLogFolder(props.getString( diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java index 1bf0d259c5f7c..6bd48474c846a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java @@ -93,7 +93,7 @@ public static Option getWriteConfigWithRecordSizeEstimate(Jav private static Pair doSampleWrites(JavaSparkContext jsc, Option> recordsOpt, HoodieWriteConfig writeConfig, String instantTime) throws IOException { final String sampleWritesBasePath = getSampleWritesBasePath(jsc, writeConfig, instantTime); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(String.format("%s_samples_%s", writeConfig.getTableName(), instantTime)) .setCDCEnabled(false) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java index 919bdd35dec55..199a0ceaf8c5d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java @@ -75,6 +75,7 @@ import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hudi.sync.common.util.SyncUtilHelpers; @@ -395,16 +396,23 @@ private HoodieTableMetaClient getMetaClient() { } private void initializeEmptyTable() throws IOException { + initializeEmptyTable(HoodieTableMetaClient.newTableBuilder(), + SparkKeyGenUtils.getPartitionColumnsForKeyGenerator(props), + HadoopFSUtils.getStorageConfWithCopy(hoodieSparkContext.hadoopConfiguration())); + } + + void initializeEmptyTable(HoodieTableMetaClient.TableBuilder tableBuilder, String partitionColumns, + StorageConfiguration storageConf) throws IOException { this.commitsTimelineOpt = Option.empty(); this.allCommitsTimelineOpt = Option.empty(); - String partitionColumns = SparkKeyGenUtils.getPartitionColumnsForKeyGenerator(props); - HoodieTableMetaClient.withPropertyBuilder() - .setTableType(cfg.tableType) + tableBuilder.setTableType(cfg.tableType) .setTableName(cfg.targetTableName) .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName(cfg.payloadClassName) .setBaseFileFormat(cfg.baseFileFormat) .setPartitionFields(partitionColumns) + .setTableVersion(props.getInteger(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), + HoodieWriteConfig.WRITE_TABLE_VERSION.defaultValue())) .setRecordKeyFields(props.getProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key())) .setPopulateMetaFields(props.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())) @@ -421,8 +429,7 @@ private void initializeEmptyTable() throws IOException { Boolean.parseBoolean(HIVE_STYLE_PARTITIONING_ENABLE.defaultValue()))) .setUrlEncodePartitioning(props.getBoolean(URL_ENCODE_PARTITIONING.key(), Boolean.parseBoolean(URL_ENCODE_PARTITIONING.defaultValue()))) - .initTable(HadoopFSUtils.getStorageConfWithCopy(hoodieSparkContext.hadoopConfiguration()), - cfg.targetBasePath); + .initTable(storageConf, cfg.targetBasePath); } /** diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index 5b1e1de975fbc..fa96d4429ca45 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -99,6 +99,7 @@ import org.apache.hudi.utilities.sources.TestParquetDFSSourceEmptyBatch; import org.apache.hudi.utilities.streamer.HoodieStreamer; import org.apache.hudi.utilities.streamer.NoNewDataTerminationStrategy; +import org.apache.hudi.utilities.streamer.StreamSync; import org.apache.hudi.utilities.testutils.JdbcTestUtils; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; import org.apache.hudi.utilities.testutils.sources.DistributedTestDataSource; @@ -2618,7 +2619,7 @@ public void testFetchingCheckpointFromPreviousCommits() throws IOException { TypedProperties properties = new TypedProperties(); properties.setProperty("hoodie.datasource.write.recordkey.field", "key"); properties.setProperty("hoodie.datasource.write.partitionpath.field", "pp"); - TestStreamSync testDeltaSync = new TestStreamSync(cfg, sparkSession, null, properties, + DummyStreamSync testDeltaSync = new DummyStreamSync(cfg, sparkSession, null, properties, jsc, fs, jsc.hadoopConfiguration(), null); properties.put(HoodieTableConfig.NAME.key(), "sample_tbl"); @@ -2960,11 +2961,11 @@ private Set getAllFileIDsInTable(String tableBasePath, Option pa return baseFileStream.map(HoodieBaseFile::getFileId).collect(Collectors.toSet()); } - class TestStreamSync extends DeltaSync { + static class DummyStreamSync extends StreamSync { - public TestStreamSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider, TypedProperties props, - JavaSparkContext jssc, FileSystem fs, Configuration conf, - Function onInitializingHoodieWriteClient) throws IOException { + public DummyStreamSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider, TypedProperties props, + JavaSparkContext jssc, FileSystem fs, Configuration conf, + Function onInitializingHoodieWriteClient) throws IOException { super(cfg, sparkSession, schemaProvider, props, jssc, fs, conf, onInitializingHoodieWriteClient); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java index d6b74682fae21..58d3d4d5ee834 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java @@ -91,7 +91,7 @@ public void init() throws Exception { targetPath = Paths.get(basePath(), "target").toString(); storage = HoodieStorageUtils.getStorage(basePath(), HadoopFSUtils.getStorageConf(jsc().hadoopConfiguration())); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java index ca285257efab5..9f313f0926c51 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java @@ -58,14 +58,12 @@ public void testHoodieClusteringJobWithClean(boolean skipClean) throws Exception Properties props = getPropertiesForKeyGen(true); HoodieWriteConfig config = getWriteConfig(tableBasePath); props.putAll(config.getProps()); - Properties metaClientProps = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); + .initTable(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), tableBasePath); - metaClient = HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), tableBasePath, metaClientProps); client = new SparkRDDWriteClient(context, config); writeData(false, client.createNewInstantTime(), 100, true); @@ -97,14 +95,11 @@ public void testPurgePendingInstants() throws Exception { Properties props = getPropertiesForKeyGen(true); HoodieWriteConfig config = getWriteConfig(tableBasePath); props.putAll(config.getProps()); - Properties metaClientProps = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); - - metaClient = HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), tableBasePath, metaClientProps); + .initTable(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), tableBasePath); client = new SparkRDDWriteClient(context, config); writeData(false, client.createNewInstantTime(), 100, true); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieCompactorJob.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieCompactorJob.java index bd0000efd005f..4ba071e26764b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieCompactorJob.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieCompactorJob.java @@ -73,14 +73,12 @@ public void testHoodieCompactorWithOptionalClean(boolean skipClean) throws Excep .withIndexConfig(HoodieIndexConfig.newBuilder().fromProperties(props).withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("1").build()) .build(); props.putAll(config.getProps()); - Properties metaClientProps = HoodieTableMetaClient.withPropertyBuilder() + metaClient = HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); + .initTable(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), tableBasePath); - metaClient = HoodieTableMetaClient.initTableAndGetMetaClient( - HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()), tableBasePath, metaClientProps); client = new SparkRDDWriteClient(context, config); writeData(true, client.createNewInstantTime(), 100, true); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java index 6626d86c2406f..874a5c6c4ce3c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java @@ -106,13 +106,12 @@ public void setUp() throws IOException { @Override public HoodieTableMetaClient getHoodieMetaClient(StorageConfiguration storageConf, String basePath, Properties props) throws IOException { - props = HoodieTableMetaClient.withPropertyBuilder() + return HoodieTableMetaClient.newTableBuilder() .setTableName(RAW_TRIPS_TEST_NAME) .setTableType(tableType) .setPayloadClass(HoodieAvroPayload.class) .fromProperties(props) - .build(); - return HoodieTableMetaClient.initTableAndGetMetaClient(storageConf.newInstance(), basePath, props); + .initTable(storageConf.newInstance(), basePath); } @Test diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSyncUnitTests.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSync.java similarity index 89% rename from hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSyncUnitTests.java rename to hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSync.java index fee063a136014..df8a9c42eaf0b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSyncUnitTests.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSync.java @@ -24,11 +24,13 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieErrorTableConfig; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -44,6 +46,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -62,6 +65,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -70,7 +74,8 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -public class TestStreamSyncUnitTests { +public class TestStreamSync { + @ParameterizedTest @MethodSource("testCasesFetchNextBatchFromSource") void testFetchNextBatchFromSource(Boolean useRowWriter, Boolean hasTransformer, Boolean hasSchemaProvider, @@ -280,4 +285,37 @@ static Stream testCasesFetchNextBatchFromSource() { } return b.build(); } + + @Test + public void testInitializeEmptyTable() throws IOException { + // given + HoodieStreamer.Config cfg = new HoodieStreamer.Config(); + cfg.targetTableName = "testTableName"; + cfg.targetBasePath = "/fake/table/name"; + cfg.tableType = "MERGE_ON_READ"; + SchemaProvider schemaProvider = getSchemaProvider("InputBatch", false); + TypedProperties props = new TypedProperties(); + props.put(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), HoodieTableVersion.SIX.versionCode()); + + // setup + HoodieSparkEngineContext hoodieSparkEngineContext = mock(HoodieSparkEngineContext.class); + HoodieStorage storage = new HoodieHadoopStorage(mock(FileSystem.class)); + SparkSession sparkSession = mock(SparkSession.class); + Configuration configuration = mock(Configuration.class); + SourceFormatAdapter sourceFormatAdapter = mock(SourceFormatAdapter.class); + TypedProperties propsSpy = spy(props); + HoodieTableMetaClient.TableBuilder tableBuilder = spy(HoodieTableMetaClient.newTableBuilder() + .fromProperties(propsSpy)); + doReturn(null).when(tableBuilder).initTable(any(), anyString()); + + StreamSync streamSync = new StreamSync(cfg, sparkSession, propsSpy, hoodieSparkEngineContext, + storage, configuration, client -> true, schemaProvider, Option.empty(), sourceFormatAdapter, Option.empty(), false); + StreamSync spy = spy(streamSync); + + // when + spy.initializeEmptyTable(tableBuilder, "", null); + + // then + verify(tableBuilder, times(1)).setTableVersion(HoodieTableVersion.SIX); + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 57d1ed241e8cf..5beb41a869594 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -309,7 +309,7 @@ private static void clearHiveDb(String tempWriteablePath) throws Exception { // Create Dummy hive sync config HiveSyncConfig hiveSyncConfig = getHiveSyncConfig(tempWriteablePath, "dummy"); hiveSyncConfig.setHadoopConf(hiveTestService.getHiveConf()); - HoodieTableMetaClient.withPropertyBuilder() + HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) .setTableName(hiveSyncConfig.getString(META_SYNC_TABLE_NAME)) .initTable(storage.getConf().newInstance(), hiveSyncConfig.getString(META_SYNC_BASE_PATH));