diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml index 76172203866b0..8b82415982f90 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml @@ -74,10 +74,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: true delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: second_hive_sync last_validate: config: execute_itr_count: 50 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml index dc1e99a431209..031664cd15c99 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml @@ -62,10 +62,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 30 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml index eca4eac1c710a..c23775b2ce546 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml @@ -62,10 +62,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 50 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml b/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml index 81c21a7be67c8..2fc68596d84a4 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml @@ -64,10 +64,12 @@ dag_content: config: validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 20 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml index a2d85a7a4d0f5..db7edb8f8f28c 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml @@ -65,10 +65,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: false + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 20 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml index 1c2f44b060036..102807ec435be 100644 --- a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml +++ b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml @@ -62,10 +62,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 50 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml b/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml index dfbfba0a15700..947bbdab86b43 100644 --- a/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml +++ b/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml @@ -45,10 +45,12 @@ dag_content: config: validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 6 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/spark-long-running.yaml b/docker/demo/config/test-suite/spark-long-running.yaml index 00fea43f4578e..2ffef557815c7 100644 --- a/docker/demo/config/test-suite/spark-long-running.yaml +++ b/docker/demo/config/test-suite/spark-long-running.yaml @@ -46,10 +46,12 @@ dag_content: validate_once_every_itr : 5 validate_hive: false delete_input_data: true + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateDatasetNode deps: first_delete last_validate: config: execute_itr_count: 30 + max_wait_time_for_deltastreamer_catch_up_ms: 600000 type: ValidateAsyncOperations deps: second_validate diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java index 8ab6c0ca4f992..637f1393f51ad 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -364,7 +364,7 @@ private HoodieWriteConfig getWriteConfig() { private void initJavaSparkContext(Option userDefinedMaster) { if (jsc == null) { - jsc = SparkUtil.initJavaSparkConf(SparkUtil.getDefaultConf("HoodieCLI", userDefinedMaster)); + jsc = SparkUtil.initJavaSparkContext(SparkUtil.getDefaultConf("HoodieCLI", userDefinedMaster)); } } } \ No newline at end of file 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 323c7bb5c36e0..9fe83f1995c2b 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 @@ -95,7 +95,7 @@ public static void main(String[] args) throws Exception { LOG.info("Invoking SparkMain: " + commandString); final SparkCommand cmd = SparkCommand.valueOf(commandString); - JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + commandString, + JavaSparkContext jsc = SparkUtil.initJavaSparkContext("hoodie-cli-" + commandString, Option.of(args[1]), Option.of(args[2])); int returnCode = 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 a5e513c614939..2ddb88792c0fd 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 @@ -56,7 +56,7 @@ public String upgradeHoodieTable( if (exitCode != 0) { return String.format("Failed: Could not Upgrade/Downgrade Hoodie table to \"%s\".", toVersion); } - return String.format("Hoodie table upgraded/downgraded to ", toVersion); + return String.format("Hoodie table upgraded/downgraded to %s", toVersion); } @CliCommand(value = "downgrade table", help = "Downgrades a table") @@ -78,6 +78,6 @@ public String downgradeHoodieTable( if (exitCode != 0) { return String.format("Failed: Could not Upgrade/Downgrade Hoodie table to \"%s\".", toVersion); } - return String.format("Hoodie table upgraded/downgraded to ", toVersion); + return String.format("Hoodie table upgraded/downgraded to %s", toVersion); } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java index ae99b0b8240d7..bcccb66b3716c 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java @@ -32,8 +32,8 @@ import java.io.File; import java.net.URISyntaxException; +import java.util.Arrays; import java.util.Map; -import java.util.Objects; import java.util.Properties; /** @@ -56,9 +56,12 @@ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntax if (!StringUtils.isNullOrEmpty(propertiesFile)) { sparkLauncher.setPropertiesFile(propertiesFile); } + File libDirectory = new File(new File(currentJar).getParent(), "lib"); - for (String library : Objects.requireNonNull(libDirectory.list())) { - sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath()); + // This lib directory may be not required, such as providing libraries through a bundle jar + if (libDirectory.exists()) { + Arrays.stream(libDirectory.list()).forEach(library -> + sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath())); } return sparkLauncher; } @@ -99,20 +102,20 @@ public static SparkConf getDefaultConf(final String appName, final Option master, Option executorMemory) { + public static JavaSparkContext initJavaSparkContext(String name, Option master, Option executorMemory) { SparkConf sparkConf = getDefaultConf(name, master); if (executorMemory.isPresent()) { sparkConf.set(HoodieCliSparkConfig.CLI_EXECUTOR_MEMORY, executorMemory.get()); } - return initJavaSparkConf(sparkConf); + return initJavaSparkContext(sparkConf); } - public static JavaSparkContext initJavaSparkConf(SparkConf sparkConf) { + public static JavaSparkContext initJavaSparkContext(SparkConf sparkConf) { SparkRDDWriteClient.registerClasses(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf); jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java index 1d2872edf267f..d5c535ebfe00c 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java @@ -88,11 +88,11 @@ public void init() throws IOException { // Write date files and log file String testWriteToken = "1-0-1"; Files.createFile(Paths.get(fullPartitionPath, FSUtils - .makeDataFileName(commitTime1, testWriteToken, fileId1))); + .makeBaseFileName(commitTime1, testWriteToken, fileId1))); Files.createFile(Paths.get(fullPartitionPath, FSUtils .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, testWriteToken))); Files.createFile(Paths.get(fullPartitionPath, FSUtils - .makeDataFileName(commitTime2, testWriteToken, fileId1))); + .makeBaseFileName(commitTime2, testWriteToken, fileId1))); Files.createFile(Paths.get(fullPartitionPath, FSUtils .makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, testWriteToken))); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java index 4966438292949..a470ee1c2a37d 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/SparkUtilTest.java @@ -22,11 +22,22 @@ import org.apache.hudi.cli.utils.SparkUtil; import org.apache.spark.SparkConf; +import org.apache.spark.launcher.SparkLauncher; import org.junit.jupiter.api.Test; +import java.net.URISyntaxException; + import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; public class SparkUtilTest { + + @Test + public void testInitSparkLauncher() throws URISyntaxException { + SparkLauncher sparkLauncher = SparkUtil.initLauncher(null); + assertNotNull(sparkLauncher); + } + @Test public void testGetDefaultSparkConf() { SparkConf sparkConf = SparkUtil.getDefaultConf("test-spark-app", Option.of("")); 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 270027df18053..251ff97799ffa 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 @@ -1379,7 +1379,7 @@ protected Option inlineScheduleClustering(Option> ex return scheduleClustering(extraMetadata); } - protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { + public void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { Option pendingRollbackInstantInfo = getPendingRollbackInfo(table.getMetaClient(), inflightInstant.getTimestamp(), false); String commitTime = pendingRollbackInstantInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 2974cc2ef6d6f..f111bb70ef007 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -72,7 +72,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -507,10 +506,16 @@ private Stream getInstantsToArchive() { List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), HoodieInstant.getComparableAction(hoodieInstant.getAction()))); if (instantsToStream != null) { - return instantsToStream.stream(); + // sorts the instants in natural order to make sure the metadata files be removed + // in HoodieInstant.State sequence: requested -> inflight -> completed, + // this is important because when a COMPLETED metadata file is removed first, + // other monitors on the timeline(such as the compaction or clustering services) would + // mistakenly recognize the pending file as a pending operation, + // then all kinds of weird bugs occur. + return instantsToStream.stream().sorted(); } else { // if a concurrent writer archived the instant - return Collections.EMPTY_LIST.stream(); + return Stream.empty(); } }); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index eee6f4f4927e0..1180845a6ed8a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -51,6 +51,8 @@ public class HoodieClusteringConfig extends HoodieConfig { public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy."; public static final String SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy"; + public static final String FLINK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = + "org.apache.hudi.client.clustering.plan.strategy.FlinkSizeBasedClusteringPlanStrategy"; public static final String JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.JavaSizeBasedClusteringPlanStrategy"; public static final String SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY = diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 7fc46e8b9bbc4..426e20f83b034 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -180,7 +180,7 @@ private void init(HoodieRecord record) { // base file to denote some log appends happened on a slice. writeToken will still fence concurrent // writers. // https://issues.apache.org/jira/browse/HUDI-1517 - createMarkerFile(partitionPath, FSUtils.makeDataFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension())); this.writer = createLogWriter(fileSlice, baseInstantTime); } catch (Exception e) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 43a8c12324136..738e2d6b48d13 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -100,7 +100,7 @@ public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTa new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath), hoodieTable.getPartitionMetafileFormat()); partitionMetadata.trySave(getPartitionId()); - createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension())); this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writeSchemaWithMetaFields, this.taskContextSupplier); } catch (IOException e) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index b999cc6906406..92fa5c28394a5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -174,7 +174,7 @@ private void init(String fileId, String partitionPath, HoodieBaseFile baseFileTo hoodieTable.getPartitionMetafileFormat()); partitionMetadata.trySave(getPartitionId()); - String newFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()); + String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()); makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName); LOG.info(String.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 5d5760961a461..b7fdbecfd56d1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -164,7 +164,7 @@ public Path makeNewPath(String partitionPath) { throw new HoodieIOException("Failed to make dir " + path, e); } - return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId, + return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, writeToken, fileId, hoodieTable.getMetaClient().getTableConfig().getBaseFileFormat().getFileExtension())); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java index 63b502531a896..89360c247403d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java @@ -25,20 +25,20 @@ import java.io.Serializable; /** - * Repartition input records into at least expected number of output spark partitions. It should give below guarantees - - * Output spark partition will have records from only one hoodie partition. - Average records per output spark - * partitions should be almost equal to (#inputRecords / #outputSparkPartitions) to avoid possible skews. + * Repartition input records into at least expected number of output partitions. It should give below guarantees - + * Output partition will have records from only one hoodie partition. - Average records per output + * partitions should be almost equal to (#inputRecords / #outputPartitions) to avoid possible skews. */ public interface BulkInsertPartitioner extends Serializable { /** - * Repartitions the input records into at least expected number of output spark partitions. + * Repartitions the input records into at least expected number of output partitions. * - * @param records Input Hoodie records - * @param outputSparkPartitions Expected number of output partitions + * @param records Input Hoodie records + * @param outputPartitions Expected number of output partitions * @return */ - I repartitionRecords(I records, int outputSparkPartitions); + I repartitionRecords(I records, int outputPartitions); /** * @return {@code true} if the records within a partition are sorted; {@code false} otherwise. @@ -48,6 +48,7 @@ public interface BulkInsertPartitioner extends Serializable { /** * Return file group id prefix for the given data partition. * By defauult, return a new file group id prefix, so that incoming records will route to a fresh new file group + * * @param partitionId data partition * @return */ @@ -57,6 +58,7 @@ default String getFileIdPfx(int partitionId) { /** * Return write handle factory for the given partition. + * * @param partitionId data partition * @return */ 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 807865dae2416..56526d23db006 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 @@ -885,24 +885,22 @@ private boolean shouldExecuteMetadataTableDeletion() { // partitions are ready to use return !HoodieTableMetadata.isMetadataTable(metaClient.getBasePath()) && !config.isMetadataTableEnabled() - && (!metaClient.getTableConfig().contains(TABLE_METADATA_PARTITIONS) - || !metaClient.getTableConfig().getMetadataPartitions().isEmpty()); + && !metaClient.getTableConfig().getMetadataPartitions().isEmpty(); } /** * Clears hoodie.table.metadata.partitions in hoodie.properties */ private void clearMetadataTablePartitionsConfig(Option partitionType, boolean clearAll) { - if (clearAll) { + Set partitions = getCompletedMetadataPartitions(metaClient.getTableConfig()); + if (clearAll && partitions.size() > 0) { LOG.info("Clear hoodie.table.metadata.partitions in hoodie.properties"); metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), EMPTY_STRING); HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); - return; + } else if (partitions.remove(partitionType.get().getPartitionPath())) { + metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", partitions)); + HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } - Set completedPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig()); - completedPartitions.remove(partitionType.get().getPartitionPath()); - metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); - HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } public HoodieTableMetadata getMetadataTable() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index 479f63932c5b3..a96ff73947cdb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -70,6 +70,9 @@ public static String checkAndGetClusteringPlanStrategy(HoodieWriteConfig config) String sparkSizeBasedClassName = HoodieClusteringConfig.SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; String sparkSelectedPartitionsClassName = "org.apache.hudi.client.clustering.plan.strategy.SparkSelectedPartitionsClusteringPlanStrategy"; String sparkRecentDaysClassName = "org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy"; + String flinkSizeBasedClassName = HoodieClusteringConfig.FLINK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; + String flinkSelectedPartitionsClassName = "org.apache.hudi.client.clustering.plan.strategy.FlinkSelectedPartitionsClusteringPlanStrategy"; + String flinkRecentDaysClassName = "org.apache.hudi.client.clustering.plan.strategy.FlinkRecentDaysClusteringPlanStrategy"; String javaSelectedPartitionClassName = "org.apache.hudi.client.clustering.plan.strategy.JavaRecentDaysClusteringPlanStrategy"; String javaSizeBasedClassName = HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY; @@ -82,6 +85,14 @@ public static String checkAndGetClusteringPlanStrategy(HoodieWriteConfig config) config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name()); LOG.warn(String.format(logStr, className, sparkSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name())); return sparkSizeBasedClassName; + } else if (flinkRecentDaysClassName.equals(className)) { + config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.RECENT_DAYS.name()); + LOG.warn(String.format(logStr, className, sparkSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.RECENT_DAYS.name())); + return flinkSizeBasedClassName; + } else if (flinkSelectedPartitionsClassName.equals(className)) { + config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name()); + LOG.warn(String.format(logStr, className, sparkSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name())); + return flinkSizeBasedClassName; } else if (javaSelectedPartitionClassName.equals(className)) { config.setValue(HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME, ClusteringPlanPartitionFilterMode.RECENT_DAYS.name()); LOG.warn(String.format(logStr, className, javaSizeBasedClassName, HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key(), ClusteringPlanPartitionFilterMode.SELECTED_PARTITIONS.name())); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java index 5c184e77dfaa2..fc4ae986e6d55 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; @@ -99,6 +100,7 @@ public HoodieWriteMetadata> execute() { metadata.addMetadata(SerDeHelper.LATEST_SCHEMA, schemaPair.getLeft().get()); metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaPair.getRight().get()); } + metadata.setOperationType(WriteOperationType.COMPACT); compactionMetadata.setWriteStatuses(statuses); compactionMetadata.setCommitted(false); compactionMetadata.setCommitMetadata(Option.of(metadata)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 42add690f29ea..95f22bba27d5f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -137,6 +137,6 @@ private static String getFileNameForMarkerFromLogFile(String logFilePath, Hoodie String baseInstant = FSUtils.getBaseCommitTimeFromLogPath(logPath); String writeToken = FSUtils.getWriteTokenFromLogPath(logPath); - return FSUtils.makeDataFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension()); + return FSUtils.makeBaseFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension()); } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java index 0c4a971ac8055..3146c9d6b4928 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java @@ -103,7 +103,7 @@ public static void setupTimelineInFS( } public static String getBaseFilename(String instantTime, String fileId) { - return FSUtils.makeDataFileName(instantTime, BASE_FILE_WRITE_TOKEN, fileId); + return FSUtils.makeBaseFileName(instantTime, BASE_FILE_WRITE_TOKEN, fileId); } public static String getLogFilename(String instantTime, String fileId) { 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 2d23c3afb7f14..ddfbabaf36ae9 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 @@ -29,8 +29,10 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; 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.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; @@ -39,6 +41,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.FlinkHoodieIndexFactory; @@ -53,7 +56,6 @@ import org.apache.hudi.io.MiniBatchHandle; import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; @@ -69,6 +71,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.HashMap; import java.util.Iterator; @@ -365,8 +369,7 @@ public void completeCompaction( // commit to data table after committing to metadata table. // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. - table.getMetadataWriter(compactionInstant.getTimestamp()).ifPresent( - w -> ((HoodieTableMetadataWriter) w).update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); + writeTableMetadata(table, compactionCommitTime, compactionInstant.getAction(), metadata); LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); } finally { @@ -401,6 +404,52 @@ public HoodieWriteMetadata> cluster(final String clusteringIns throw new HoodieNotSupportedException("Clustering is not supported yet"); } + private void completeClustering( + HoodieReplaceCommitMetadata metadata, + HoodieTable>, List, List> table, + String clusteringCommitTime) { + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect clustering write status and commit clustering"); + HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime); + List writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e -> + e.getValue().stream()).collect(Collectors.toList()); + if (writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum() > 0) { + throw new HoodieClusteringException("Clustering failed to write to files:" + + writeStats.stream().filter(s -> s.getTotalWriteErrors() > 0L).map(HoodieWriteStat::getFileId).collect(Collectors.joining(","))); + } + + try { + this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); + finalizeWrite(table, clusteringCommitTime, writeStats); + // commit to data table after committing to metadata table. + // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a + // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. + writeTableMetadata(table, clusteringCommitTime, clusteringInstant.getAction(), metadata); + LOG.info("Committing Clustering {} finished with result {}.", clusteringCommitTime, metadata); + table.getActiveTimeline().transitionReplaceInflightToComplete( + HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieClusteringException( + "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + clusteringCommitTime, e); + } finally { + this.txnManager.endTransaction(Option.of(clusteringInstant)); + } + + WriteMarkersFactory.get(config.getMarkersType(), table, clusteringCommitTime) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); + if (clusteringTimer != null) { + long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); + try { + metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(clusteringCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.REPLACE_COMMIT_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " + + config.getBasePath() + " at time " + clusteringCommitTime, e); + } + } + LOG.info("Clustering successfully on commit " + clusteringCommitTime); + } + @Override protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { // Create a Hoodie table which encapsulated the commits and files visible @@ -414,6 +463,23 @@ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option insta // no need to execute the upgrade/downgrade on each write in streaming. } + public void completeTableService( + TableServiceType tableServiceType, + HoodieCommitMetadata metadata, + HoodieTable>, List, List> table, + String commitInstant) { + switch (tableServiceType) { + case CLUSTER: + completeClustering((HoodieReplaceCommitMetadata) metadata, table, commitInstant); + break; + case COMPACT: + completeCompaction(metadata, table, commitInstant); + break; + default: + throw new IllegalArgumentException("This table service is not valid " + tableServiceType); + } + } + /** * Upgrade downgrade the Hoodie table. * diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java new file mode 100644 index 0000000000000..0109aaa60ffb9 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkRecentDaysClusteringPlanStrategy.java @@ -0,0 +1,65 @@ +/* + * 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.client.clustering.plan.strategy; + +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable; +import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Clustering Strategy based on following. + * 1) Only looks at latest 'daybased.lookback.partitions' partitions. + * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. + */ +public class FlinkRecentDaysClusteringPlanStrategy> + extends FlinkSizeBasedClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(FlinkRecentDaysClusteringPlanStrategy.class); + + public FlinkRecentDaysClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public FlinkRecentDaysClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + int targetPartitionsForClustering = getWriteConfig().getTargetPartitionsForClustering(); + int skipPartitionsFromLatestForClustering = getWriteConfig().getSkipPartitionsFromLatestForClustering(); + return partitionPaths.stream() + .sorted(Comparator.reverseOrder()) + .skip(Math.max(skipPartitionsFromLatestForClustering, 0)) + .limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitionPaths.size()) + .collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java new file mode 100644 index 0000000000000..ae5726bb4a46e --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSelectedPartitionsClusteringPlanStrategy.java @@ -0,0 +1,67 @@ +/* + * 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.client.clustering.plan.strategy; + +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable; +import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX; + +/** + * Clustering Strategy to filter just specified partitions from [begin, end]. Note both begin and end are inclusive. + */ +public class FlinkSelectedPartitionsClusteringPlanStrategy> + extends FlinkSizeBasedClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(FlinkSelectedPartitionsClusteringPlanStrategy.class); + + public static final String CONF_BEGIN_PARTITION = CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.begin.partition"; + public static final String CONF_END_PARTITION = CLUSTERING_STRATEGY_PARAM_PREFIX + "cluster.end.partition"; + + public FlinkSelectedPartitionsClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public FlinkSelectedPartitionsClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + String beginPartition = getWriteConfig().getProps().getProperty(CONF_BEGIN_PARTITION); + String endPartition = getWriteConfig().getProps().getProperty(CONF_END_PARTITION); + List filteredPartitions = partitionPaths.stream() + .filter(path -> path.compareTo(beginPartition) >= 0 && path.compareTo(endPartition) <= 0) + .collect(Collectors.toList()); + LOG.info("Filtered to the following partitions: " + filteredPartitions); + return filteredPartitions; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java new file mode 100644 index 0000000000000..8347da6014af8 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java @@ -0,0 +1,129 @@ +/* + * 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.client.clustering.plan.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable; +import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; +import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS; + +/** + * Clustering Strategy based on following. + * 1) Creates clustering groups based on max size allowed per group. + * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. + */ +public class FlinkSizeBasedClusteringPlanStrategy> + extends PartitionAwareClusteringPlanStrategy>, List, List> { + private static final Logger LOG = LogManager.getLogger(FlinkSizeBasedClusteringPlanStrategy.class); + + public FlinkSizeBasedClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public FlinkSizeBasedClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, + HoodieFlinkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + HoodieWriteConfig writeConfig = getWriteConfig(); + + List, Integer>> fileSliceGroups = new ArrayList<>(); + List currentGroup = new ArrayList<>(); + long totalSizeSoFar = 0; + + for (FileSlice currentSlice : fileSlices) { + // check if max size is reached and create new group, if needed. + // in now, every clustering group out put is 1 file group. + if (totalSizeSoFar >= writeConfig.getClusteringTargetFileMaxBytes() && !currentGroup.isEmpty()) { + LOG.info("Adding one clustering group " + totalSizeSoFar + " max bytes: " + + writeConfig.getClusteringMaxBytesInGroup() + " num input slices: " + currentGroup.size()); + fileSliceGroups.add(Pair.of(currentGroup, 1)); + currentGroup = new ArrayList<>(); + totalSizeSoFar = 0; + } + + // Add to the current file-group + currentGroup.add(currentSlice); + // assume each file group size is ~= parquet.max.file.size + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize(); + } + + if (!currentGroup.isEmpty()) { + fileSliceGroups.add(Pair.of(currentGroup, 1)); + } + + return fileSliceGroups.stream().map(fileSliceGroup -> + HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) + .setNumOutputFileGroups(fileSliceGroup.getRight()) + .setMetrics(buildMetrics(fileSliceGroup.getLeft())) + .build()); + } + + @Override + protected Map getStrategyParams() { + Map params = new HashMap<>(); + if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) { + params.put(PLAN_STRATEGY_SORT_COLUMNS.key(), getWriteConfig().getClusteringSortColumns()); + } + return params; + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + return partitionPaths; + } + + @Override + protected Stream getFileSlicesEligibleForClustering(final String partition) { + return super.getFileSlicesEligibleForClustering(partition) + // Only files that have basefile size smaller than small file size are eligible. + .filter(slice -> slice.getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) < getWriteConfig().getClusteringSmallFileLimit()); + } + + private int getNumberOfOutputFileGroups(long groupSize, long targetFileSize) { + return (int) Math.ceil(groupSize / (double) targetFileSize); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java index 3005f40352afb..777e228c9510d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkCreateHandle.java @@ -88,7 +88,7 @@ public FlinkCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTab */ private void deleteInvalidDataFile(long lastAttemptId) { final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId); - final String lastDataFileName = FSUtils.makeDataFileName(instantTime, + final String lastDataFileName = FSUtils.makeBaseFileName(instantTime, lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension()); final Path path = makeNewFilePath(partitionPath, lastDataFileName); try { @@ -136,7 +136,7 @@ public boolean canWrite(HoodieRecord record) { * Use the writeToken + "-" + rollNumber as the new writeToken of a mini-batch write. */ private Path newFilePathWithRollover(int rollNumber) { - final String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, fileId, + final String dataFileName = FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, fileId, hoodieTable.getBaseFileExtension()); return makeNewFilePath(partitionPath, dataFileName); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java index a16e9cc738633..24da25b20be1e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeAndReplaceHandle.java @@ -90,7 +90,7 @@ public FlinkMergeAndReplaceHandle(HoodieWriteConfig config, String instantTime, */ private void deleteInvalidDataFile(long lastAttemptId) { final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId); - final String lastDataFileName = FSUtils.makeDataFileName(instantTime, + final String lastDataFileName = FSUtils.makeBaseFileName(instantTime, lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension()); final Path path = makeNewFilePath(partitionPath, lastDataFileName); try { @@ -139,7 +139,7 @@ protected void makeOldAndNewFilePaths(String partitionPath, String oldFileName, protected String newFileNameWithRollover(int rollNumber) { // make the intermediate file as hidden final String fileID = "." + this.fileId; - return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, + return FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, fileID, hoodieTable.getBaseFileExtension()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java index fbb7dd7b5a91a..e1117712634cf 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/FlinkMergeHandle.java @@ -94,7 +94,7 @@ public FlinkMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTabl */ private void deleteInvalidDataFile(long lastAttemptId) { final String lastWriteToken = FSUtils.makeWriteToken(getPartitionId(), getStageId(), lastAttemptId); - final String lastDataFileName = FSUtils.makeDataFileName(instantTime, + final String lastDataFileName = FSUtils.makeBaseFileName(instantTime, lastWriteToken, this.fileId, hoodieTable.getBaseFileExtension()); final Path path = makeNewFilePath(partitionPath, lastDataFileName); if (path.equals(oldFilePath)) { @@ -159,7 +159,7 @@ protected void makeOldAndNewFilePaths(String partitionPath, String oldFileName, */ protected String newFileNameWithRollover(int rollNumber) { // make the intermediate file as hidden - return FSUtils.makeDataFileName(instantTime, writeToken + "-" + rollNumber, + return FSUtils.makeBaseFileName(instantTime, writeToken + "-" + rollNumber, this.fileId, hoodieTable.getBaseFileExtension()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java index 486a5cc54b69a..f65e6cf215480 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataCreateHandle.java @@ -97,7 +97,7 @@ public HoodieRowDataCreateHandle(HoodieTable table, HoodieWriteConfig writeConfi FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), table.getPartitionMetafileFormat()); partitionMetadata.trySave(taskPartitionId); - createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); this.fileWriter = createNewFileWriter(path, table, writeConfig, rowType); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize file writer for path " + path, e); @@ -180,7 +180,7 @@ private Path makeNewPath(String partitionPath) { throw new HoodieIOException("Failed to make dir " + path, e); } HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); - return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId, + return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId, tableConfig.getBaseFileFormat().getFileExtension())); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 9ab633f9e3b37..0e5f1c26e32f4 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -55,6 +55,7 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.CleanActionExecutor; import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; +import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor; import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.FlinkInsertOverwriteCommitActionExecutor; @@ -286,7 +287,7 @@ public HoodieWriteMetadata> compact( @Override public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { - throw new HoodieNotSupportedException("Clustering is not supported on a Flink CopyOnWrite table"); + return new ClusteringPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute(); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index f1e43b9d30d42..6eae15e7e1aff 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -105,13 +105,9 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con public Option getMetadataWriter(String triggeringInstantTimestamp, Option actionMetadata) { if (config.isMetadataTableEnabled()) { - // even with metadata enabled, some index could have been disabled - // delete metadata partitions corresponding to such indexes - deleteMetadataIndexIfNecessary(); return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp))); } else { - maybeDeleteMetadataTable(); return Option.empty(); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java index eb3d4ef312e99..b9e466485f209 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java @@ -49,7 +49,7 @@ public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema, boo @Override public List> repartitionRecords( - List> records, int outputSparkPartitions) { + List> records, int outputPartitions) { return records.stream().sorted((o1, o2) -> { Object values1 = HoodieAvroUtils.getRecordColumnValues(o1, sortColumnNames, schema, consistentLogicalTimestampEnabled); Object values2 = HoodieAvroUtils.getRecordColumnValues(o2, sortColumnNames, schema, consistentLogicalTimestampEnabled); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java index fded0ffab51bd..d272849a19f28 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaGlobalSortPartitioner.java @@ -37,7 +37,7 @@ public class JavaGlobalSortPartitioner @Override public List> repartitionRecords(List> records, - int outputSparkPartitions) { + int outputPartitions) { // Now, sort the records and line them up nicely for loading. records.sort(new Comparator() { @Override diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 7b0c4dbdf2a96..28d3ac6e9bd6e 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -103,7 +103,7 @@ public void testMakeNewPath() { }).collect(Collectors.toList()).get(0); assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath, - FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); + FSUtils.makeBaseFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); } private HoodieWriteConfig makeHoodieClientConfig() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index 4db7eb26e64ba..916b31d2931e5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -96,7 +96,7 @@ public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, S FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath), table.getPartitionMetafileFormat()); partitionMetadata.trySave(taskPartitionId); - createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); + createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension())); this.fileWriter = createNewFileWriter(path, table, writeConfig, structType); } catch (IOException e) { throw new HoodieInsertException("Failed to initialize file writer for path " + path, e); @@ -178,7 +178,7 @@ private Path makeNewPath(String partitionPath) { throw new HoodieIOException("Failed to make dir " + path, e); } HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); - return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId, + return new Path(path.toString(), FSUtils.makeBaseFileName(instantTime, getWriteToken(), fileId, tableConfig.getBaseFileFormat().getFileExtension())); } 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 a6a37030e8a69..bf3063c5d4733 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 @@ -2496,7 +2496,7 @@ private Pair> testConsistencyCheck(HoodieTableMetaCli Option markerFilePath = WriteMarkersFactory.get( cfg.getMarkersType(), getHoodieTable(metaClient, cfg), instantTime) .create(partitionPath, - FSUtils.makeDataFileName(instantTime, "1-0-1", UUID.randomUUID().toString()), + FSUtils.makeBaseFileName(instantTime, "1-0-1", UUID.randomUUID().toString()), IOType.MERGE); LOG.info("Created a dummy marker path=" + markerFilePath.get()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 30f7ad66543d1..7471d26cdfb56 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -127,7 +127,7 @@ public void testMakeNewPath() { }).collect().get(0); assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath, - FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); + FSUtils.makeBaseFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString()); } private HoodieWriteConfig makeHoodieClientConfig() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index aa0cadf5b9354..cfc143e3d0caa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -159,12 +159,12 @@ public static String makeWriteToken(int taskPartitionId, int stageId, long taskA } // TODO: this should be removed - public static String makeDataFileName(String instantTime, String writeToken, String fileId) { + public static String makeBaseFileName(String instantTime, String writeToken, String fileId) { return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension()); } - public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) { + public static String makeBaseFileName(String instantTime, String writeToken, String fileId, String fileExtension) { return String.format("%s_%s_%s%s", fileId, writeToken, instantTime, fileExtension); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java index 58b9f7475a35f..3eb8f784dbab7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCleaningPolicy.java @@ -22,5 +22,5 @@ * Hoodie cleaning policies. */ public enum HoodieCleaningPolicy { - KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_COMMITS, KEEP_LATEST_BY_HOURS; + KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_COMMITS, KEEP_LATEST_BY_HOURS } 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 edc6caa5bcbdf..886911466b95f 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 @@ -272,8 +272,8 @@ private static Properties getOrderedPropertiesWithTableChecksum(Properties props * @throws IOException */ private static String storeProperties(Properties props, FSDataOutputStream outputStream) throws IOException { - String checksum; - if (props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props)) { + final String checksum; + if (isValidChecksum(props)) { checksum = props.getProperty(TABLE_CHECKSUM.key()); props.store(outputStream, "Updated at " + Instant.now()); } else { @@ -285,8 +285,8 @@ private static String storeProperties(Properties props, FSDataOutputStream outpu return checksum; } - private boolean isValidChecksum() { - return contains(TABLE_CHECKSUM) && validateChecksum(props); + private static boolean isValidChecksum(Properties props) { + return props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props); } /** @@ -298,20 +298,13 @@ public HoodieTableConfig() { private void fetchConfigs(FileSystem fs, String metaPath) throws IOException { Path cfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE); - Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP); try (FSDataInputStream is = fs.open(cfgPath)) { props.load(is); - // validate checksum for latest table version - if (getTableVersion().versionCode() >= HoodieTableVersion.FOUR.versionCode() && !isValidChecksum()) { - LOG.warn("Checksum validation failed. Falling back to backed up configs."); - try (FSDataInputStream fsDataInputStream = fs.open(backupCfgPath)) { - props.load(fsDataInputStream); - } - } } catch (IOException ioe) { if (!fs.exists(cfgPath)) { LOG.warn("Run `table recover-configs` if config update/delete failed midway. Falling back to backed up configs."); // try the backup. this way no query ever fails if update fails midway. + Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP); try (FSDataInputStream is = fs.open(backupCfgPath)) { props.load(is); } @@ -631,7 +624,7 @@ public List getMetadataPartitions() { CONFIG_VALUES_DELIMITER ); } - + /** * Returns the format to use for partition meta files. */ diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java index f51702a447258..7506e659c9254 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java @@ -84,7 +84,7 @@ public void setUp() throws IOException { public void testMakeDataFileName() { String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); - assertEquals(FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION); + assertEquals(FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + BASE_FILE_EXTENSION); } @Test @@ -159,7 +159,7 @@ public void testProcessFiles() throws Exception { public void testGetCommitTime() { String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); - String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName); + String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName); assertEquals(instantTime, FSUtils.getCommitTime(fullFileName)); // test log file name fullFileName = FSUtils.makeLogFileName(fileName, HOODIE_LOG.getFileExtension(), instantTime, 1, TEST_WRITE_TOKEN); @@ -170,7 +170,7 @@ public void testGetCommitTime() { public void testGetFileNameWithoutMeta() { String instantTime = HoodieActiveTimeline.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); - String fullFileName = FSUtils.makeDataFileName(instantTime, TEST_WRITE_TOKEN, fileName); + String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName); assertEquals(fileName, FSUtils.getFileId(fullFileName)); } @@ -304,7 +304,7 @@ public void testFileNameRelatedFunctions() throws Exception { final String LOG_EXTENTION = "." + LOG_STR; // data file name - String dataFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId); + String dataFileName = FSUtils.makeBaseFileName(instantTime, writeToken, fileId); assertEquals(instantTime, FSUtils.getCommitTime(dataFileName)); assertEquals(fileId, FSUtils.getFileId(dataFileName)); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java index f8995ab4c07ac..631c7cd41a385 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java @@ -45,7 +45,7 @@ public void testSetPaths() { Path basePath = new Path(basePathString); Path partitionPath = new Path(basePath, partitionPathString); - Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(instantTime, writeToken, fileName)); + Path finalizeFilePath = new Path(partitionPath, FSUtils.makeBaseFileName(instantTime, writeToken, fileName)); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setPath(basePath, finalizeFilePath); assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath())); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index 54bc138fc8f84..b63b9df9746a3 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -378,7 +378,7 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData String dataFileName = null; if (!skipCreatingDataFile) { - dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId); + dataFileName = FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + dataFileName).createNewFile(); } String fileName1 = @@ -417,7 +417,7 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap); } String compactionRequestedTime = "4"; - String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); + String compactDataFileName = FSUtils.makeBaseFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); List> partitionFileSlicesPairs = new ArrayList<>(); partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0))); HoodieCompactionPlan compactionPlan = @@ -552,12 +552,12 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData final String orphanFileId2 = UUID.randomUUID().toString(); final String invalidInstantId = "INVALIDTIME"; String inflightDeltaInstantTime = "7"; - String orphanDataFileName = FSUtils.makeDataFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1); + String orphanDataFileName = FSUtils.makeBaseFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1); new File(basePath + "/" + partitionPath + "/" + orphanDataFileName).createNewFile(); String orphanLogFileName = FSUtils.makeLogFileName(orphanFileId2, HoodieLogFile.DELTA_EXTENSION, invalidInstantId, 0, TEST_WRITE_TOKEN); new File(basePath + "/" + partitionPath + "/" + orphanLogFileName).createNewFile(); - String inflightDataFileName = FSUtils.makeDataFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1); + String inflightDataFileName = FSUtils.makeBaseFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1); new File(basePath + "/" + partitionPath + "/" + inflightDataFileName).createNewFile(); String inflightLogFileName = FSUtils.makeLogFileName(inflightFileId2, HoodieLogFile.DELTA_EXTENSION, inflightDeltaInstantTime, 0, TEST_WRITE_TOKEN); @@ -712,7 +712,7 @@ public void testGetLatestDataFilesForFileId() throws IOException { // Only one commit, but is not safe String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); refreshFsView(); assertFalse(roView.getLatestBaseFiles(partitionPath).anyMatch(dfile -> dfile.getFileId().equals(fileId)), @@ -728,7 +728,7 @@ public void testGetLatestDataFilesForFileId() throws IOException { // Do another commit, but not safe String commitTime2 = "2"; - String fileName2 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId); + String fileName2 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); refreshFsView(); assertEquals(fileName1, roView.getLatestBaseFiles(partitionPath) @@ -762,22 +762,22 @@ public void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly) th String fileId3 = UUID.randomUUID().toString(); String fileId4 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)) .createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)) .createNewFile(); @@ -827,9 +827,9 @@ private void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly, S for (HoodieBaseFile status : dataFileList) { filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); filenames = new HashSet<>(); List logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4, true) @@ -856,12 +856,12 @@ private void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly, S } if (!isLatestFileSliceOnly) { assertEquals(3, dataFiles.size()); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); } else { assertEquals(1, dataFiles.size()); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); } logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3, true) @@ -887,13 +887,13 @@ protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -918,22 +918,22 @@ protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) Set expFileNames = new HashSet<>(); if (fileId.equals(fileId1)) { if (!isLatestFileSliceOnly) { - expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)); } - expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)); assertEquals(expFileNames, filenames); } else if (fileId.equals(fileId2)) { if (!isLatestFileSliceOnly) { - expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)); - expFileNames.add(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)); } - expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)); assertEquals(expFileNames, filenames); } else { if (!isLatestFileSliceOnly) { - expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)); } - expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)); + expFileNames.add(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)); assertEquals(expFileNames, filenames); } } @@ -956,21 +956,21 @@ protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) thr String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -993,10 +993,10 @@ protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) thr filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); if (!isLatestFileSliceOnly) { - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3))); } List slices = @@ -1037,13 +1037,13 @@ protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) thr String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); - new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); @@ -1063,8 +1063,8 @@ protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) thr for (HoodieBaseFile status : dataFiles) { filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2))); } else { assertEquals(0, dataFiles.size()); } @@ -1088,30 +1088,30 @@ protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IO String fileId2 = UUID.randomUUID().toString(); String fileId3 = UUID.randomUUID().toString(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)) .createNewFile(); new File(fullPartitionPath + "/" + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)) .createNewFile(); new File(fullPartitionPath + "/" + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)) .createNewFile(); new File(fullPartitionPath + "/" + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, TEST_WRITE_TOKEN)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)) .createNewFile(); - new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)) + new File(fullPartitionPath + "/" + FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)) .createNewFile(); new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); @@ -1158,9 +1158,9 @@ protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IO for (HoodieBaseFile status : statuses1) { filenames.add(status.getFileName()); } - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); - assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId1))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId2))); + assertTrue(filenames.contains(FSUtils.makeBaseFileName(commitTime4, TEST_WRITE_TOKEN, fileId3))); } @Test @@ -1181,15 +1181,15 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E String deltaInstantTime2 = "3"; String fileId = UUID.randomUUID().toString(); - String dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId); + String dataFileName = FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId); new File(fullPartitionPath1 + dataFileName).createNewFile(); String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0, TEST_WRITE_TOKEN); new File(fullPartitionPath1 + fileName1).createNewFile(); - new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); + new File(fullPartitionPath2 + FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); new File(fullPartitionPath2 + fileName1).createNewFile(); - new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); + new File(fullPartitionPath3 + FSUtils.makeBaseFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile(); new File(fullPartitionPath3 + fileName1).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); @@ -1228,7 +1228,7 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E partitionFileSlicesPairs.add(Pair.of(partitionPath3, fileSlices.get(0))); String compactionRequestedTime = "2"; - String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); + String compactDataFileName = FSUtils.makeBaseFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId); HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, Option.empty(), Option.empty()); @@ -1345,8 +1345,8 @@ public void testReplaceWithTimeTravel() throws IOException { "No commit, should not find any data file"); // Only one commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); @@ -1362,8 +1362,8 @@ public void testReplaceWithTimeTravel() throws IOException { // create commit2 - fileId1 is replaced. new file groups fileId3,fileId4 are created. String fileId3 = UUID.randomUUID().toString(); String fileId4 = UUID.randomUUID().toString(); - String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); - String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); + String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile(); @@ -1440,10 +1440,10 @@ public void testReplaceFileIdIsExcludedInView() throws IOException { // Only one commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); - String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); - String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); @@ -1500,9 +1500,9 @@ public void testPendingClusteringOperations() throws IOException { "No commit, should not find any data file"); // Only one commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); - String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName3 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); @@ -1614,8 +1614,8 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept // first insert commit String commitTime1 = "1"; - String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); - String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName1 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); @@ -1636,7 +1636,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept // replace commit String commitTime2 = "2"; - String fileName3 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId3); + String fileName3 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId3); new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile(); HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2); @@ -1658,7 +1658,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept // another insert commit String commitTime3 = "3"; - String fileName4 = FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId4); + String fileName4 = FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId4); new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile(); HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime3); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index a9c9db303f328..1c59558c94ce7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -826,7 +826,7 @@ private List> generateDataForInstant(String baseIn File file = new File(basePath + "/" + p + "/" + (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN) - : FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f))); + : FSUtils.makeBaseFileName(instant, TEST_WRITE_TOKEN, f))); file.createNewFile(); HoodieWriteStat w = new HoodieWriteStat(); w.setFileId(f); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 231915072914d..290753ef52006 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -80,7 +80,7 @@ public static String baseFileName(String instantTime, String fileId) { } public static String baseFileName(String instantTime, String fileId, String fileExtension) { - return FSUtils.makeDataFileName(instantTime, WRITE_TOKEN, fileId, fileExtension); + return FSUtils.makeBaseFileName(instantTime, WRITE_TOKEN, fileId, fileExtension); } public static String logFileName(String instantTime, String fileId, int version) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java index 54ca072651e07..a5d45d1184f9b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java @@ -147,7 +147,7 @@ private HoodieInstant createRequestedReplaceInstant(String partitionPath1, Strin private FileSlice generateFileSlice(String partitionPath, String fileId, String baseInstant) { FileSlice fs = new FileSlice(new HoodieFileGroupId(partitionPath, fileId), baseInstant); - fs.setBaseFile(new HoodieBaseFile(FSUtils.makeDataFileName(baseInstant, "1-0-1", fileId))); + fs.setBaseFile(new HoodieBaseFile(FSUtils.makeBaseFileName(baseInstant, "1-0-1", fileId))); return fs; } 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 729f0147b5940..3de4bd4f757b8 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 @@ -18,6 +18,7 @@ package org.apache.hudi.configuration; +import org.apache.hudi.client.clustering.plan.strategy.FlinkRecentDaysClusteringPlanStrategy; import org.apache.hudi.common.config.ConfigClassProperty; import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.HoodieConfig; @@ -583,6 +584,72 @@ private FlinkOptions() { .defaultValue(40)// default min 40 commits .withDescription("Min number of commits to keep before archiving older commits into a sequential log, default 40"); + // ------------------------------------------------------------------------ + // Clustering Options + // ------------------------------------------------------------------------ + + public static final ConfigOption CLUSTERING_SCHEDULE_ENABLED = ConfigOptions + .key("clustering.schedule.enabled") + .booleanType() + .defaultValue(false) // default false for pipeline + .withDescription("Schedule the cluster plan, default false"); + + public static final ConfigOption CLUSTERING_DELTA_COMMITS = ConfigOptions + .key("clustering.delta_commits") + .intType() + .defaultValue(4) + .withDescription("Max delta commits needed to trigger clustering, default 4 commits"); + + public static final ConfigOption CLUSTERING_TASKS = ConfigOptions + .key("clustering.tasks") + .intType() + .defaultValue(4) + .withDescription("Parallelism of tasks that do actual clustering, default is 4"); + + public static final ConfigOption CLUSTERING_TARGET_PARTITIONS = ConfigOptions + .key("clustering.plan.strategy.daybased.lookback.partitions") + .intType() + .defaultValue(2) + .withDescription("Number of partitions to list to create ClusteringPlan, default is 2"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_CLASS = ConfigOptions + .key("clustering.plan.strategy.class") + .stringType() + .defaultValue(FlinkRecentDaysClusteringPlanStrategy.class.getName()) + .withDescription("Config to provide a strategy class (subclass of ClusteringPlanStrategy) to create clustering plan " + + "i.e select what file groups are being clustered. Default strategy, looks at the last N (determined by " + + CLUSTERING_TARGET_PARTITIONS.key() + ") day based partitions picks the small file slices within those partitions."); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES = ConfigOptions + .key("clustering.plan.strategy.target.file.max.bytes") + .intType() + .defaultValue(1024 * 1024 * 1024) // default 1 GB + .withDescription("Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT = ConfigOptions + .key("clustering.plan.strategy.small.file.limit") + .intType() + .defaultValue(600) // default 600 MB + .withDescription("Files smaller than the size specified here are candidates for clustering, default 600 MB"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigOptions + .key("clustering.plan.strategy.daybased.skipfromlatest.partitions") + .intType() + .defaultValue(0) + .withDescription("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan"); + + public static final ConfigOption CLUSTERING_SORT_COLUMNS = ConfigOptions + .key("clustering.plan.strategy.sort.columns") + .stringType() + .noDefaultValue() + .withDescription("Columns to sort the data by when clustering"); + + public static final ConfigOption CLUSTERING_MAX_NUM_GROUPS = ConfigOptions + .key("clustering.plan.strategy.max.num.groups") + .intType() + .defaultValue(30) + .withDescription("Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism, default is 30"); + // ------------------------------------------------------------------------ // Hive Sync Options // ------------------------------------------------------------------------ diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java index 4d3fc08efe197..b5599886a9d0b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java @@ -48,7 +48,7 @@ public OneInputStreamOperator createSortOperator() { codeGen.generateRecordComparator("SortComparator")); } - private SortCodeGenerator createSortCodeGenerator() { + public SortCodeGenerator createSortCodeGenerator() { SortSpec.SortSpecBuilder builder = SortSpec.builder(); IntStream.range(0, sortIndices.length).forEach(i -> builder.addField(i, true, true)); return new SortCodeGenerator(tableConfig, rowType, builder.build()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java new file mode 100644 index 0000000000000..30a8fbed3fafd --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitEvent.java @@ -0,0 +1,77 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.client.WriteStatus; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a commit event from the clustering task {@link ClusteringFunction}. + */ +public class ClusteringCommitEvent implements Serializable { + private static final long serialVersionUID = 1L; + + /** + * The clustering commit instant time. + */ + private String instant; + /** + * The write statuses. + */ + private List writeStatuses; + /** + * The clustering task identifier. + */ + private int taskID; + + public ClusteringCommitEvent() { + } + + public ClusteringCommitEvent(String instant, List writeStatuses, int taskID) { + this.instant = instant; + this.writeStatuses = writeStatuses; + this.taskID = taskID; + } + + public void setInstant(String instant) { + this.instant = instant; + } + + public void setWriteStatuses(List writeStatuses) { + this.writeStatuses = writeStatuses; + } + + public void setTaskID(int taskID) { + this.taskID = taskID; + } + + public String getInstant() { + return instant; + } + + public List getWriteStatuses() { + return writeStatuses; + } + + public int getTaskID() { + return taskID; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java new file mode 100644 index 0000000000000..bc87270a49f1b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java @@ -0,0 +1,174 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.TableServiceType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.sink.CleanFunction; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Function to check and commit the clustering action. + * + *

Each time after receiving a clustering commit event {@link ClusteringCommitEvent}, + * it loads and checks the clustering plan {@link org.apache.hudi.avro.model.HoodieClusteringPlan}, + * if all the clustering operations {@link org.apache.hudi.common.model.ClusteringOperation} + * of the plan are finished, tries to commit the clustering action. + * + *

It also inherits the {@link CleanFunction} cleaning ability. This is needed because + * the SQL API does not allow multiple sinks in one table sink provider. + */ +public class ClusteringCommitSink extends CleanFunction { + private static final Logger LOG = LoggerFactory.getLogger(ClusteringCommitSink.class); + + /** + * Config options. + */ + private final Configuration conf; + + private transient HoodieFlinkTable table; + + /** + * Buffer to collect the event from each clustering task {@code ClusteringFunction}. + * The key is the instant time. + */ + private transient Map> commitBuffer; + + public ClusteringCommitSink(Configuration conf) { + super(conf); + this.conf = conf; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (writeClient == null) { + this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); + } + this.commitBuffer = new HashMap<>(); + this.table = writeClient.getHoodieTable(); + } + + @Override + public void invoke(ClusteringCommitEvent event, Context context) throws Exception { + final String instant = event.getInstant(); + commitBuffer.computeIfAbsent(instant, k -> new ArrayList<>()) + .add(event); + commitIfNecessary(instant, commitBuffer.get(instant)); + } + + /** + * Condition to commit: the commit buffer has equal size with the clustering plan operations + * and all the clustering commit event {@link ClusteringCommitEvent} has the same clustering instant time. + * + * @param instant Clustering commit instant time + * @param events Commit events ever received for the instant + */ + private void commitIfNecessary(String instant, List events) { + HoodieInstant clusteringInstant = HoodieTimeline.getReplaceCommitInflightInstant(instant); + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + StreamerUtil.createMetaClient(this.conf), clusteringInstant); + HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + boolean isReady = clusteringPlan.getInputGroups().size() == events.size(); + if (!isReady) { + return; + } + List statuses = events.stream() + .map(ClusteringCommitEvent::getWriteStatuses) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + + HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); + writeMetadata.setWriteStatuses(statuses); + writeMetadata.setWriteStats(statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList())); + writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(clusteringPlan, writeMetadata)); + validateWriteResult(clusteringPlan, instant, writeMetadata); + if (!writeMetadata.getCommitMetadata().isPresent()) { + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata( + writeMetadata.getWriteStats().get(), + writeMetadata.getPartitionToReplaceFileIds(), + Option.empty(), + WriteOperationType.CLUSTER, + this.writeClient.getConfig().getSchema(), + HoodieTimeline.REPLACE_COMMIT_ACTION); + writeMetadata.setCommitMetadata(Option.of(commitMetadata)); + } + // commit the clustering + this.table.getMetaClient().reloadActiveTimeline(); + this.writeClient.completeTableService( + TableServiceType.CLUSTER, writeMetadata.getCommitMetadata().get(), table, instant); + + // reset the status + reset(instant); + } + + private void reset(String instant) { + this.commitBuffer.remove(instant); + } + + /** + * Validate actions taken by clustering. In the first implementation, we validate at least one new file is written. + * But we can extend this to add more validation. E.g. number of records read = number of records written etc. + * We can also make these validations in BaseCommitActionExecutor to reuse pre-commit hooks for multiple actions. + */ + private static void validateWriteResult(HoodieClusteringPlan clusteringPlan, String instantTime, HoodieWriteMetadata> writeMetadata) { + if (writeMetadata.getWriteStatuses().isEmpty()) { + throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime + + " #groups: " + clusteringPlan.getInputGroups().size() + " expected at least " + + clusteringPlan.getInputGroups().stream().mapToInt(HoodieClusteringGroup::getNumOutputFileGroups).sum() + + " write statuses"); + } + } + + private static Map> getPartitionToReplacedFileIds( + HoodieClusteringPlan clusteringPlan, + HoodieWriteMetadata> writeMetadata) { + Set newFilesWritten = writeMetadata.getWriteStats().get().stream() + .map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())).collect(Collectors.toSet()); + return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan) + .filter(fg -> !newFilesWritten.contains(fg)) + .collect(Collectors.groupingBy(HoodieFileGroupId::getPartitionPath, Collectors.mapping(HoodieFileGroupId::getFileId, Collectors.toList()))); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java new file mode 100644 index 0000000000000..a415ac9d46165 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java @@ -0,0 +1,318 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.apache.hudi.common.model.ClusteringGroupInfo; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.log.HoodieFileSliceReader; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.IOUtils; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.sink.bulk.BulkInsertWriterHelper; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.AvroToRowDataConverters; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.avro.generic.IndexedRecord; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.NormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.RecordComparator; +import org.apache.flink.table.runtime.operators.TableStreamOperator; +import org.apache.flink.table.runtime.operators.sort.BinaryExternalSorter; +import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.runtime.util.StreamRecordCollector; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.hudi.table.format.FormatUtils.buildAvroRecordBySchema; + +/** + * Operator to execute the actual clustering task assigned by the clustering plan task. + * In order to execute scalable, the input should shuffle by the clustering event {@link ClusteringPlanEvent}. + */ +public class ClusteringOperator extends TableStreamOperator implements + OneInputStreamOperator, BoundedOneInput { + private static final Logger LOG = LoggerFactory.getLogger(ClusteringOperator.class); + + private final Configuration conf; + private final RowType rowType; + private int taskID; + private transient HoodieWriteConfig writeConfig; + private transient HoodieFlinkTable table; + private transient Schema schema; + private transient Schema readerSchema; + private transient int[] requiredPos; + private transient AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter; + private transient HoodieFlinkWriteClient writeClient; + private transient BulkInsertWriterHelper writerHelper; + private transient String instantTime; + + private transient BinaryExternalSorter sorter; + private transient StreamRecordCollector collector; + private transient BinaryRowDataSerializer binarySerializer; + + public ClusteringOperator(Configuration conf, RowType rowType) { + this.conf = conf; + this.rowType = rowType; + } + + @Override + public void open() throws Exception { + super.open(); + + this.taskID = getRuntimeContext().getIndexOfThisSubtask(); + this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf); + this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); + this.table = writeClient.getHoodieTable(); + + this.schema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + this.readerSchema = HoodieAvroUtils.addMetadataFields(this.schema); + this.requiredPos = getRequiredPositions(); + + this.avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(rowType); + + ClassLoader cl = getContainingTask().getUserCodeClassLoader(); + + AbstractRowDataSerializer inputSerializer = new BinaryRowDataSerializer(rowType.getFieldCount()); + this.binarySerializer = new BinaryRowDataSerializer(inputSerializer.getArity()); + + NormalizedKeyComputer computer = createSortCodeGenerator().generateNormalizedKeyComputer("SortComputer").newInstance(cl); + RecordComparator comparator = createSortCodeGenerator().generateRecordComparator("SortComparator").newInstance(cl); + + MemoryManager memManager = getContainingTask().getEnvironment().getMemoryManager(); + this.sorter = + new BinaryExternalSorter( + this.getContainingTask(), + memManager, + computeMemorySize(), + this.getContainingTask().getEnvironment().getIOManager(), + inputSerializer, + binarySerializer, + computer, + comparator, + getContainingTask().getJobConfiguration()); + this.sorter.startThreads(); + + collector = new StreamRecordCollector<>(output); + + // register the metrics. + getMetricGroup().gauge("memoryUsedSizeInBytes", (Gauge) sorter::getUsedMemoryInBytes); + getMetricGroup().gauge("numSpillFiles", (Gauge) sorter::getNumSpillFiles); + getMetricGroup().gauge("spillInBytes", (Gauge) sorter::getSpillInBytes); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + ClusteringPlanEvent event = element.getValue(); + final String instantTime = event.getClusteringInstantTime(); + final ClusteringGroupInfo clusteringGroupInfo = event.getClusteringGroupInfo(); + + initWriterHelper(instantTime); + + List clusteringOps = clusteringGroupInfo.getOperations(); + boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); + + Iterator iterator; + if (hasLogFiles) { + // if there are log files, we read all records into memory for a file group and apply updates. + iterator = readRecordsForGroupWithLogs(clusteringOps, instantTime); + } else { + // We want to optimize reading records for case there are no log files. + iterator = readRecordsForGroupBaseFiles(clusteringOps); + } + + RowDataSerializer rowDataSerializer = new RowDataSerializer(rowType); + while (iterator.hasNext()) { + RowData rowData = iterator.next(); + BinaryRowData binaryRowData = rowDataSerializer.toBinaryRow(rowData).copy(); + this.sorter.write(binaryRowData); + } + + BinaryRowData row = binarySerializer.createInstance(); + while ((row = sorter.getIterator().next(row)) != null) { + this.writerHelper.write(row); + } + } + + @Override + public void close() { + if (this.writeClient != null) { + this.writeClient.cleanHandlesGracefully(); + this.writeClient.close(); + } + } + + /** + * End input action for batch source. + */ + public void endInput() { + List writeStatuses = this.writerHelper.getWriteStatuses(this.taskID); + collector.collect(new ClusteringCommitEvent(instantTime, writeStatuses, this.taskID)); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + + private void initWriterHelper(String clusteringInstantTime) { + if (this.writerHelper == null) { + this.writerHelper = new BulkInsertWriterHelper(this.conf, this.table, this.writeConfig, + clusteringInstantTime, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), getRuntimeContext().getAttemptNumber(), + this.rowType); + this.instantTime = clusteringInstantTime; + } + } + + /** + * Read records from baseFiles, apply updates and convert to Iterator. + */ + @SuppressWarnings("unchecked") + private Iterator readRecordsForGroupWithLogs(List clusteringOps, String instantTime) { + List> recordIterators = new ArrayList<>(); + + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new FlinkTaskContextSupplier(null), writeConfig); + LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); + + for (ClusteringOperation clusteringOp : clusteringOps) { + try { + Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) + ? Option.empty() + : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(table.getMetaClient().getFs()) + .withBasePath(table.getMetaClient().getBasePath()) + .withLogFilePaths(clusteringOp.getDeltaFilePaths()) + .withReaderSchema(readerSchema) + .withLatestInstantTime(instantTime) + .withMaxMemorySizeInBytes(maxMemoryPerCompaction) + .withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled()) + .withReverseReader(writeConfig.getCompactionReverseLogReadEnabled()) + .withBufferSize(writeConfig.getMaxDFSStreamBufferSize()) + .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath()) + .build(); + + HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + HoodieFileSliceReader hoodieFileSliceReader = HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema, + tableConfig.getPayloadClass(), + tableConfig.getPreCombineField(), + tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp()))); + + recordIterators.add(StreamSupport.stream(Spliterators.spliteratorUnknownSize(hoodieFileSliceReader, Spliterator.NONNULL), false).map(hoodieRecord -> { + try { + return this.transform((IndexedRecord) hoodieRecord.getData().getInsertValue(readerSchema).get()); + } catch (IOException e) { + throw new HoodieIOException("Failed to read next record", e); + } + }).iterator()); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + } + + return new ConcatenatingIterator<>(recordIterators); + } + + /** + * Read records from baseFiles and get iterator. + */ + private Iterator readRecordsForGroupBaseFiles(List clusteringOps) { + List> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> { + Iterable indexedRecords = () -> { + try { + return HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())).getRecordIterator(readerSchema); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }; + + return StreamSupport.stream(indexedRecords.spliterator(), false).map(this::transform).iterator(); + }).collect(Collectors.toList()); + + return new ConcatenatingIterator<>(iteratorsForPartition); + } + + /** + * Transform IndexedRecord into HoodieRecord. + */ + private RowData transform(IndexedRecord indexedRecord) { + GenericRecord record = buildAvroRecordBySchema(indexedRecord, schema, requiredPos, new GenericRecordBuilder(schema)); + return (RowData) avroToRowDataConverter.convert(record); + } + + private int[] getRequiredPositions() { + final List fieldNames = readerSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList()); + return schema.getFields().stream() + .map(field -> fieldNames.indexOf(field.name())) + .mapToInt(i -> i) + .toArray(); + } + + private SortCodeGenerator createSortCodeGenerator() { + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, + conf.getString(FlinkOptions.CLUSTERING_SORT_COLUMNS).split(",")); + return sortOperatorGen.createSortCodeGenerator(); + } + + @Override + public void setKeyContextElement(StreamRecord record) throws Exception { + OneInputStreamOperator.super.setKeyContextElement(record); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.java new file mode 100644 index 0000000000000..c82075877bcf3 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanEvent.java @@ -0,0 +1,73 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.common.model.ClusteringGroupInfo; + +import java.io.Serializable; +import java.util.Map; + +/** + * Represents a cluster command from the clustering plan task {@link ClusteringPlanSourceFunction}. + */ +public class ClusteringPlanEvent implements Serializable { + private static final long serialVersionUID = 1L; + + private String clusteringInstantTime; + + private ClusteringGroupInfo clusteringGroupInfo; + + private Map strategyParams; + + public ClusteringPlanEvent() { + } + + public ClusteringPlanEvent( + String instantTime, + ClusteringGroupInfo clusteringGroupInfo, + Map strategyParams) { + this.clusteringInstantTime = instantTime; + this.clusteringGroupInfo = clusteringGroupInfo; + this.strategyParams = strategyParams; + } + + public void setClusteringInstantTime(String clusteringInstantTime) { + this.clusteringInstantTime = clusteringInstantTime; + } + + public void setClusteringGroupInfo(ClusteringGroupInfo clusteringGroupInfo) { + this.clusteringGroupInfo = clusteringGroupInfo; + } + + public void setStrategyParams(Map strategyParams) { + this.strategyParams = strategyParams; + } + + public String getClusteringInstantTime() { + return clusteringInstantTime; + } + + public ClusteringGroupInfo getClusteringGroupInfo() { + return clusteringGroupInfo; + } + + public Map getStrategyParams() { + return strategyParams; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java new file mode 100644 index 0000000000000..a3db2d41c8371 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanSourceFunction.java @@ -0,0 +1,91 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.common.model.ClusteringGroupInfo; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink hudi clustering source function. + * + *

This function read the clustering plan as {@link ClusteringOperation}s then assign the clustering task + * event {@link ClusteringPlanEvent} to downstream operators. + * + *

The clustering instant time is specified explicitly with strategies: + * + *

    + *
  • If the timeline has no inflight instants, + * use {@link org.apache.hudi.common.table.timeline.HoodieActiveTimeline#createNewInstantTime()} + * as the instant time;
  • + *
  • If the timeline has inflight instants, + * use the median instant time between [last complete instant time, earliest inflight instant time] + * as the instant time.
  • + *
+ */ +public class ClusteringPlanSourceFunction extends AbstractRichFunction implements SourceFunction { + + protected static final Logger LOG = LoggerFactory.getLogger(ClusteringPlanSourceFunction.class); + + /** + * The clustering plan. + */ + private final HoodieClusteringPlan clusteringPlan; + + /** + * Hoodie instant. + */ + private final HoodieInstant instant; + + public ClusteringPlanSourceFunction(HoodieInstant instant, HoodieClusteringPlan clusteringPlan) { + this.instant = instant; + this.clusteringPlan = clusteringPlan; + } + + @Override + public void open(Configuration parameters) throws Exception { + // no operation + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + for (HoodieClusteringGroup clusteringGroup : clusteringPlan.getInputGroups()) { + LOG.info("ClusteringPlanSourceFunction cluster " + clusteringGroup + " files"); + sourceContext.collect(new ClusteringPlanEvent(this.instant.getTimestamp(), ClusteringGroupInfo.create(clusteringGroup), clusteringPlan.getStrategy().getStrategyParams())); + } + } + + @Override + public void close() throws Exception { + // no operation + } + + @Override + public void cancel() { + // no operation + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java new file mode 100644 index 0000000000000..e87a7d6752b6e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java @@ -0,0 +1,148 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.configuration.FlinkOptions; + +import com.beust.jcommander.Parameter; +import org.apache.flink.configuration.Configuration; + +/** + * Configurations for Hoodie Flink clustering. + */ +public class FlinkClusteringConfig extends Configuration { + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + // ------------------------------------------------------------------------ + // Hudi Write Options + // ------------------------------------------------------------------------ + + @Parameter(names = {"--path"}, description = "Base path for the target hoodie table.", required = true) + public String path; + + // ------------------------------------------------------------------------ + // Clustering Options + // ------------------------------------------------------------------------ + @Parameter(names = {"--clustering-delta-commits"}, description = "Max delta commits needed to trigger clustering, default 4 commits", required = false) + public Integer clusteringDeltaCommits = 1; + + @Parameter(names = {"--clustering-tasks"}, description = "Parallelism of tasks that do actual clustering, default is -1", required = false) + public Integer clusteringTasks = -1; + + @Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB.", required = false) + public Integer compactionMaxMemory = 100; + + @Parameter(names = {"--clean-retain-commits"}, + description = "Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n" + + "This also directly translates into how much you can incrementally pull on this table, default 10", + required = false) + public Integer cleanRetainCommits = 10; + + @Parameter(names = {"--archive-min-commits"}, + description = "Min number of commits to keep before archiving older commits into a sequential log, default 20.", + required = false) + public Integer archiveMinCommits = 20; + + @Parameter(names = {"--archive-max-commits"}, + description = "Max number of commits to keep before archiving older commits into a sequential log, default 30.", + required = false) + public Integer archiveMaxCommits = 30; + + @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n" + + "There is a risk of losing data when scheduling clustering outside the writer job.\n" + + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n" + + "Default is true", required = false) + public Boolean schedule = true; + + @Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default", required = false) + public Boolean cleanAsyncEnable = false; + + @Parameter(names = {"--plan-strategy-class"}, description = "Config to provide a strategy class to generator clustering plan", required = false) + public String planStrategyClass = "org.apache.hudi.client.clustering.plan.strategy.FlinkRecentDaysClusteringPlanStrategy"; + + @Parameter(names = {"--target-file-max-bytes"}, description = "Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB", required = false) + public Integer targetFileMaxBytes = 1024 * 1024 * 1024; + + @Parameter(names = {"--small-file-limit"}, description = "Files smaller than the size specified here are candidates for clustering, default 600 MB", required = false) + public Integer smallFileLimit = 600; + + @Parameter(names = {"--skip-from-latest-partitions"}, description = "Number of partitions to skip from latest when choosing partitions to create ClusteringPlan, default 0", required = false) + public Integer skipFromLatestPartitions = 0; + + @Parameter(names = {"--sort-columns"}, description = "Columns to sort the data by when clustering.", required = false) + public String sortColumns = ""; + + @Parameter(names = {"--max-num-groups"}, description = "Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism. default 30", required = false) + public Integer maxNumGroups = 30; + + @Parameter(names = {"--target-partitions"}, description = "Number of partitions to list to create ClusteringPlan, default 2", required = false) + public Integer targetPartitions = 2; + + public static final String SEQ_FIFO = "FIFO"; + public static final String SEQ_LIFO = "LIFO"; + @Parameter(names = {"--seq"}, description = "Clustering plan execution sequence, two options are supported:\n" + + "1). FIFO: execute the oldest plan first;\n" + + "2). LIFO: execute the latest plan first, by default LIFO", required = false) + public String clusteringSeq = SEQ_LIFO; + + @Parameter(names = {"--write-partition-url-encode"}, description = "Whether to encode the partition path url, default false") + public Boolean writePartitionUrlEncode = false; + + @Parameter(names = {"--hive-style-partitioning"}, description = "Whether to use Hive style partitioning.\n" + + "If set true, the names of partition folders follow = format.\n" + + "By default false (the names of partition folders are only partition values)") + public Boolean hiveStylePartitioning = false; + + /** + * Transforms a {@code FlinkClusteringConfig.config} into {@code Configuration}. + * The latter is more suitable for the table APIs. It reads all the properties + * in the properties file (set by `--props` option) and cmd line options + * (set by `--hoodie-conf` option). + */ + public static Configuration toFlinkConfig(FlinkClusteringConfig config) { + Configuration conf = new Configuration(); + + conf.setString(FlinkOptions.PATH, config.path); + conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits); + conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, config.archiveMinCommits); + conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits); + conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory); + conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, config.clusteringDeltaCommits); + conf.setInteger(FlinkOptions.CLUSTERING_TASKS, config.clusteringTasks); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS, config.planStrategyClass); + conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, config.targetFileMaxBytes); + conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT, config.smallFileLimit); + conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, config.skipFromLatestPartitions); + conf.setString(FlinkOptions.CLUSTERING_SORT_COLUMNS, config.sortColumns); + conf.setInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS, config.maxNumGroups); + conf.setInteger(FlinkOptions.CLUSTERING_TARGET_PARTITIONS, config.targetPartitions); + conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnable); + + // use synchronous clustering always + conf.setBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED, config.schedule); + + // bulk insert conf + conf.setBoolean(FlinkOptions.URL_ENCODE_PARTITIONING, config.writePartitionUrlEncode); + conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, config.hiveStylePartitioning); + + return conf; + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java new file mode 100644 index 0000000000000..f7c361533a0d9 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java @@ -0,0 +1,191 @@ +/* + * 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.sink.clustering; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.CompactionUtil; +import org.apache.hudi.util.StreamerUtil; + +import com.beust.jcommander.JCommander; +import org.apache.avro.Schema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink hudi clustering program that can be executed manually. + */ +public class HoodieFlinkClusteringJob { + + protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkClusteringJob.class); + + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + FlinkClusteringConfig cfg = new FlinkClusteringConfig(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + + // create metaClient + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + + // set table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + + // set table type + conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name()); + + // set record key field + conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp()); + + // set partition field + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp()); + + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); + + HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + // judge whether have operation + // to compute the clustering instant time and do cluster. + if (cfg.schedule) { + String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime(); + boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty()); + if (!scheduled) { + // do nothing. + LOG.info("No clustering plan for this job "); + return; + } + } + + table.getMetaClient().reloadActiveTimeline(); + + // fetch the instant based on the configured execution sequence + HoodieTimeline timeline = table.getActiveTimeline().filterPendingReplaceTimeline() + .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); + Option requested = CompactionUtil.isLIFO(cfg.clusteringSeq) ? timeline.lastInstant() : timeline.firstInstant(); + if (!requested.isPresent()) { + // do nothing. + LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option"); + return; + } + + HoodieInstant clusteringInstant = requested.get(); + + HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant.getTimestamp()); + if (timeline.containsInstant(inflightInstant)) { + LOG.info("Rollback inflight clustering instant: [" + clusteringInstant + "]"); + writeClient.rollbackInflightClustering(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + + // generate clustering plan + // should support configurable commit metadata + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), clusteringInstant); + + if (!clusteringPlanOption.isPresent()) { + // do nothing. + LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option"); + return; + } + + HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + + if (clusteringPlan == null || (clusteringPlan.getInputGroups() == null) + || (clusteringPlan.getInputGroups().isEmpty())) { + // No clustering plan, do nothing and return. + LOG.info("No clustering plan for instant " + clusteringInstant.getTimestamp()); + return; + } + + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(clusteringInstant.getTimestamp()); + HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline(); + if (!pendingClusteringTimeline.containsInstant(instant)) { + // this means that the clustering plan was written to auxiliary path(.tmp) + // but not the meta path(.hoodie), this usually happens when the job crush + // exceptionally. + + // clean the clustering plan in auxiliary path and cancels the clustering. + + LOG.warn("The clustering plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n" + + "Clean the clustering plan in auxiliary path and cancels the clustering"); + CompactionUtil.cleanInstant(table.getMetaClient(), instant); + return; + } + + // get clusteringParallelism. + int clusteringParallelism = conf.getInteger(FlinkOptions.CLUSTERING_TASKS) == -1 + ? clusteringPlan.getInputGroups().size() : conf.getInteger(FlinkOptions.CLUSTERING_TASKS); + + // Mark instant as clustering inflight + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + + final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); + final RowType rowType = (RowType) rowDataType.getLogicalType(); + + // setup configuration + long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); + conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); + + DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(timeline.lastInstant().get(), clusteringPlan)) + .name("clustering_source") + .uid("uid_clustering_source") + .rebalance() + .transform("clustering_task", + TypeInformation.of(ClusteringCommitEvent.class), + new ClusteringOperator(conf, rowType)) + .setParallelism(clusteringPlan.getInputGroups().size()); + + ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(), + conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); + + dataStream + .addSink(new ClusteringCommitSink(conf)) + .name("clustering_commit") + .uid("uid_clustering_commit") + .setParallelism(1); + + env.execute("flink_hudi_clustering"); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java index fdb8152b00577..db13a6c2ae584 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java @@ -256,7 +256,11 @@ public synchronized void reload(long checkpointId) { return; } this.metaClient.reloadActiveTimeline(); - this.fsView.sync(); + // release the old fs view and create a new one + SyncableFileSystemView oldFsView = this.fsView; + this.fsView = getFileSystemView(); + oldFsView.close(); + recordProfile(); cleanMetadataCache(this.metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()); this.smallFilesMap.clear(); 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 f82712bca2c2a..e9574dd52bedd 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 @@ -192,7 +192,7 @@ public class FlinkStreamerConfig extends Configuration { public Boolean indexGlobalEnabled = true; @Parameter(names = {"--index-partition-regex"}, - description = "Whether to load partitions in state if partition path matching, default *") + description = "Whether to load partitions in state if partition path matching, default *") public String indexPartitionRegex = ".*"; @Parameter(names = {"--source-avro-schema-path"}, description = "Source avro schema file path, the parsed schema is used for deserialization") 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 b977dfd7c5343..fcffbed54b48f 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 @@ -29,6 +29,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -37,6 +38,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.config.HoodiePayloadConfig; @@ -162,6 +164,17 @@ public static HoodieWriteConfig getHoodieClientConfig( .withPath(conf.getString(FlinkOptions.PATH)) .combineInput(conf.getBoolean(FlinkOptions.PRE_COMBINE), true) .withMergeAllowDuplicateOnInserts(OptionsResolver.insertClustering(conf)) + .withClusteringConfig( + HoodieClusteringConfig.newBuilder() + .withAsyncClustering(conf.getBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED)) + .withClusteringPlanStrategyClass(conf.getString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS)) + .withClusteringTargetPartitions(conf.getInteger(FlinkOptions.CLUSTERING_TARGET_PARTITIONS)) + .withClusteringMaxNumGroups(conf.getInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS)) + .withClusteringTargetFileMaxBytes(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES)) + .withClusteringPlanSmallFileLimit(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT) * 1024 * 1024L) + .withClusteringSkipPartitionsFromLatest(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST)) + .withAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS)) + .build()) .withCompactionConfig( HoodieCompactionConfig.newBuilder() .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) @@ -505,6 +518,11 @@ public static boolean haveSuccessfulCommits(HoodieTableMetaClient metaClient) { * Returns the max compaction memory in bytes with given conf. */ public static long getMaxCompactionMemoryInBytes(Configuration conf) { - return conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024; + return (long) conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024; + } + + public static Schema getTableAvroSchema(HoodieTableMetaClient metaClient, boolean includeMetadataFields) throws Exception { + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + return schemaUtil.getTableAvroSchema(includeMetadataFields); } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java new file mode 100644 index 0000000000000..ac2ee0be374ea --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java @@ -0,0 +1,184 @@ +/* + * 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.sink.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.clustering.ClusteringCommitEvent; +import org.apache.hudi.sink.clustering.ClusteringCommitSink; +import org.apache.hudi.sink.clustering.ClusteringOperator; +import org.apache.hudi.sink.clustering.ClusteringPlanSourceFunction; +import org.apache.hudi.sink.clustering.FlinkClusteringConfig; +import org.apache.hudi.sink.clustering.HoodieFlinkClusteringJob; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.CompactionUtil; +import org.apache.hudi.util.StreamerUtil; +import org.apache.hudi.utils.TestConfigurations; +import org.apache.hudi.utils.TestData; +import org.apache.hudi.utils.TestSQL; + +import org.apache.avro.Schema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * IT cases for {@link HoodieFlinkClusteringJob}. + */ +public class ITTestHoodieFlinkClustering { + + private static final Map EXPECTED = new HashMap<>(); + + static { + EXPECTED.put("par1", "[id1,par1,id1,Danny,23,1000,par1, id2,par1,id2,Stephen,33,2000,par1]"); + EXPECTED.put("par2", "[id3,par2,id3,Julian,53,3000,par2, id4,par2,id4,Fabian,31,4000,par2]"); + EXPECTED.put("par3", "[id5,par3,id5,Sophia,18,5000,par3, id6,par3,id6,Emma,20,6000,par3]"); + EXPECTED.put("par4", "[id7,par4,id7,Bob,44,7000,par4, id8,par4,id8,Han,56,8000,par4]"); + } + + @TempDir + File tempFile; + + @Test + public void testHoodieFlinkClustering() throws Exception { + // 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, 1); + Map options = new HashMap<>(); + options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath()); + + // use append mode + options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value()); + options.put(FlinkOptions.INSERT_CLUSTER.key(), "false"); + + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options); + tableEnv.executeSql(hoodieTableDDL); + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(3); + + // Make configuration and setAvroSchema. + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + FlinkClusteringConfig cfg = new FlinkClusteringConfig(); + cfg.path = tempFile.getAbsolutePath(); + cfg.targetPartitions = 4; + Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + + // create metaClient + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + + // set the table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name()); + + // set record key field + conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp()); + // set partition field + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp()); + + long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); + conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition"); + + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); + + // judge whether have operation + // To compute the clustering instant time and do clustering. + String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime(); + + HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty()); + + assertTrue(scheduled, "The clustering plan should be scheduled"); + + // fetch the instant based on the configured execution sequence + table.getMetaClient().reloadActiveTimeline(); + HoodieTimeline timeline = table.getActiveTimeline().filterPendingReplaceTimeline() + .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); + + // generate clustering plan + // should support configurable commit metadata + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), timeline.lastInstant().get()); + + HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + + // Mark instant as clustering inflight + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(clusteringInstantTime); + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + + final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); + final RowType rowType = (RowType) rowDataType.getLogicalType(); + + DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(timeline.lastInstant().get(), clusteringPlan)) + .name("clustering_source") + .uid("uid_clustering_source") + .rebalance() + .transform("clustering_task", + TypeInformation.of(ClusteringCommitEvent.class), + new ClusteringOperator(conf, rowType)) + .setParallelism(clusteringPlan.getInputGroups().size()); + + ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(), + conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); + + dataStream + .addSink(new ClusteringCommitSink(conf)) + .name("clustering_commit") + .uid("uid_clustering_commit") + .setParallelism(1); + + env.execute("flink_hudi_clustering"); + TestData.checkWrittenData(tempFile, EXPECTED, 4); + } +} diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index ccd85d382930a..1081e43175630 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -101,7 +101,7 @@ public static File simulateInserts(File partitionPath, String baseFileExtension, throws IOException { for (int i = 0; i < numberOfFiles; i++) { Files.createFile(partitionPath.toPath() - .resolve(FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i, baseFileExtension))); + .resolve(FSUtils.makeBaseFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i, baseFileExtension))); } return partitionPath; } @@ -118,7 +118,7 @@ public static void simulateUpdates(File directory, String baseFileExtension, fin List toUpdateList = dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size())); for (File file : toUpdateList) { String fileId = FSUtils.getFileId(file.getName()); - Files.createFile(directory.toPath().resolve(FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId, + Files.createFile(directory.toPath().resolve(FSUtils.makeBaseFileName(newCommit, TEST_WRITE_TOKEN, fileId, baseFileExtension))); } } @@ -270,7 +270,7 @@ private static void createData(Schema schema, java.nio.file.Path partitionPath, String commitNumber) throws IOException { AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { - String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); + String fileId = FSUtils.makeBaseFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema); try { for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) { @@ -286,7 +286,7 @@ private static void createSimpleData(Schema schema, java.nio.file.Path partition String commitNumber) throws Exception { AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { - String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); + String fileId = FSUtils.makeBaseFileName(commitNumber, "1", "fileid" + i, HoodieFileFormat.PARQUET.getFileExtension()); parquetWriter = new AvroParquetWriter(new Path(partitionPath.resolve(fileId).toString()), schema); try { List records = SchemaTestUtil.generateTestRecords(0, numberOfRecords); @@ -318,7 +318,7 @@ public static void simulateParquetUpdates(File directory, Schema schema, String File fileToUpdate = Objects.requireNonNull(directory.listFiles((dir, name) -> name.endsWith("parquet")))[0]; String fileId = FSUtils.getFileId(fileToUpdate.getName()); File dataFile = new File(directory, - FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension())); + FSUtils.makeBaseFileName(newCommit, TEST_WRITE_TOKEN, fileId, HoodieFileFormat.PARQUET.getFileExtension())); try (AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema)) { for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) { if (numberOfRecordsToUpdate > 0) { diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md index 5d26d03a20a89..687ad9a2a90d2 100644 --- a/hudi-integ-test/README.md +++ b/hudi-integ-test/README.md @@ -593,6 +593,56 @@ Sample spark-submit command to test one delta streamer and a spark data source w --use-hudi-data-to-generate-updates ``` +======= +### Testing async table services +We can test async table services with deltastreamer using below command. 3 additional arguments are required to test async +table services comapared to previous command. + +```shell +--continuous \ +--test-continuous-mode \ +--min-sync-interval-seconds 20 +``` + +Here is the full command: +```shell +./bin/spark-submit --packages org.apache.spark:spark-avro_2.11:2.4.4 \ + --conf spark.task.cpus=1 --conf spark.executor.cores=1 \ +--conf spark.task.maxFailures=100 \ +--conf spark.memory.fraction=0.4 \ +--conf spark.rdd.compress=true \ +--conf spark.kryoserializer.buffer.max=2000m \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.memory.storageFraction=0.1 \ +--conf spark.shuffle.service.enabled=true \ +--conf spark.sql.hive.convertMetastoreParquet=false \ +--conf spark.driver.maxResultSize=12g \ +--conf spark.executor.heartbeatInterval=120s \ +--conf spark.network.timeout=600s \ +--conf spark.yarn.max.executor.failures=10 \ +--conf spark.sql.catalogImplementation=hive \ +--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob /hudi-integ-test-bundle-0.12.0-SNAPSHOT.jar \ +--source-ordering-field test_suite_source_ordering_field \ +--use-deltastreamer \ +--target-base-path /tmp/hudi/output \ +--input-base-path /tmp/hudi/input \ +--target-table table1 \ +-props file:/tmp/test.properties \ +--schemaprovider-class org.apache.hudi.integ.testsuite.schema.TestSuiteFileBasedSchemaProvider \ +--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ +--input-file-size 125829120 \ +--workload-yaml-path file:/tmp/simple-deltastreamer.yaml \ +--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ +--table-type COPY_ON_WRITE \ +--compact-scheduling-minshare 1 \ +--clean-input \ +--clean-output \ +--continuous \ +--test-continuous-mode \ +--min-sync-interval-seconds 20 +``` + +We can use any yaml and properties file w/ above spark-submit command to test deltastreamer w/ async table services. ## Automated tests for N no of yamls in Local Docker environment diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java new file mode 100644 index 0000000000000..1bf69aaf836cc --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java @@ -0,0 +1,157 @@ +/* + * 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.integ.testsuite; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; +import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; + +/** + * Test suite Writer that assists in testing async table operations with Deltastreamer continuous mode. + * + * Sample command + * ./bin/spark-submit --packages org.apache.spark:spark-avro_2.11:2.4.4 \ + * --conf spark.task.cpus=1 --conf spark.executor.cores=1 \ + * --conf spark.task.maxFailures=100 \ + * --conf spark.memory.fraction=0.4 \ + * --conf spark.rdd.compress=true \ + * --conf spark.kryoserializer.buffer.max=2000m \ + * --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ + * --conf spark.memory.storageFraction=0.1 \ + * --conf spark.shuffle.service.enabled=true \ + * --conf spark.sql.hive.convertMetastoreParquet=false \ + * --conf spark.driver.maxResultSize=12g \ + * --conf spark.executor.heartbeatInterval=120s \ + * --conf spark.network.timeout=600s \ + * --conf spark.yarn.max.executor.failures=10 \ + * --conf spark.sql.catalogImplementation=hive \ + * --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob /hudi-integ-test-bundle-0.12.0-SNAPSHOT.jar \ + * --source-ordering-field test_suite_source_ordering_field \ + * --use-deltastreamer \ + * --target-base-path /tmp/hudi/output \ + * --input-base-path /tmp/hudi/input \ + * --target-table table1 \ + * -props file:/tmp/test.properties \ + * --schemaprovider-class org.apache.hudi.integ.testsuite.schema.TestSuiteFileBasedSchemaProvider \ + * --source-class org.apache.hudi.utilities.sources.AvroDFSSource \ + * --input-file-size 125829120 \ + * --workload-yaml-path file:/tmp/simple-deltastreamer.yaml \ + * --workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ + * --table-type COPY_ON_WRITE \ + * --compact-scheduling-minshare 1 \ + * --clean-input \ + * --clean-output \ + * --continuous \ + * --test-continuous-mode \ + * --min-sync-interval-seconds 20 + */ +public class HoodieContinousTestSuiteWriter extends HoodieTestSuiteWriter { + + private static Logger log = LoggerFactory.getLogger(HoodieContinousTestSuiteWriter.class); + + public HoodieContinousTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteJob.HoodieTestSuiteConfig cfg, String schema) throws Exception { + super(jsc, props, cfg, schema); + } + + @Override + public void shutdownResources() { + log.info("Shutting down deltastreamer gracefully "); + this.deltaStreamerWrapper.shutdownGracefully(); + } + + @Override + public RDD getNextBatch() throws Exception { + return null; + } + + @Override + public Pair>> fetchSource() throws Exception { + return null; + } + + @Override + public Option startCommit() { + return null; + } + + public JavaRDD upsert(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD insert(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD insertOverwrite(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD insertOverwriteTable(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD bulkInsert(Option instantTime) throws Exception { + return null; + } + + @Override + public JavaRDD compact(Option instantTime) throws Exception { + return null; + } + + @Override + public void inlineClustering() { + } + + @Override + public Option scheduleCompaction(Option> previousCommitExtraMetadata) throws + Exception { + return Option.empty(); + } + + @Override + public void commit(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) { + } + + @Override + public void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) throws IOException { + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java new file mode 100644 index 0000000000000..63805e71a5645 --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java @@ -0,0 +1,225 @@ +/* + * 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.integ.testsuite; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.HoodieReadClient; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodiePayloadConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; +import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.compact.CompactHelpers; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are + * {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}. + */ +public class HoodieInlineTestSuiteWriter extends HoodieTestSuiteWriter { + + private static Logger log = LoggerFactory.getLogger(HoodieInlineTestSuiteWriter.class); + + private static final String GENERATED_DATA_PATH = "generated.data.path"; + + public HoodieInlineTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws Exception { + super(jsc, props, cfg, schema); + } + + public void shutdownResources() { + // no-op for non continuous mode test suite writer. + } + + public RDD getNextBatch() throws Exception { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + JavaRDD inputRDD = nextBatch.getRight().getRight(); + return inputRDD.map(r -> (GenericRecord) ((HoodieAvroRecord) r).getData() + .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); + } + + public Pair>> fetchSource() throws Exception { + return this.deltaStreamerWrapper.fetchSource(); + } + + public Option startCommit() { + if (cfg.useDeltaStreamer) { + return Option.of(HoodieActiveTimeline.createNewInstantTime()); + } else { + return Option.of(writeClient.startCommit()); + } + } + + public JavaRDD upsert(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.upsert(WriteOperationType.UPSERT); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.upsert(nextBatch.getRight().getRight(), instantTime.get()); + } + } + + public JavaRDD insert(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.insert(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.insert(nextBatch.getRight().getRight(), instantTime.get()); + } + } + + public JavaRDD insertOverwrite(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.insertOverwrite(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.insertOverwrite(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); + } + } + + public JavaRDD insertOverwriteTable(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.insertOverwriteTable(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.insertOverwriteTable(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); + } + } + + public JavaRDD bulkInsert(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.bulkInsert(); + } else { + Pair>> nextBatch = fetchSource(); + lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); + return writeClient.bulkInsert(nextBatch.getRight().getRight(), instantTime.get()); + } + } + + public JavaRDD compact(Option instantTime) throws Exception { + if (cfg.useDeltaStreamer) { + return deltaStreamerWrapper.compact(); + } else { + if (!instantTime.isPresent()) { + Option> compactionPlanPair = Option + .fromJavaOptional(hoodieReadClient.getPendingCompactions() + .stream().findFirst()); + if (compactionPlanPair.isPresent()) { + instantTime = Option.of(compactionPlanPair.get().getLeft()); + } + } + if (instantTime.isPresent()) { + HoodieWriteMetadata> compactionMetadata = writeClient.compact(instantTime.get()); + return compactionMetadata.getWriteStatuses(); + } else { + return null; + } + } + } + + public void inlineClustering() { + if (!cfg.useDeltaStreamer) { + Option clusteringInstantOpt = writeClient.scheduleClustering(Option.empty()); + clusteringInstantOpt.ifPresent(clusteringInstant -> { + // inline cluster should auto commit as the user is never given control + log.warn("Clustering instant :: " + clusteringInstant); + writeClient.cluster(clusteringInstant, true); + }); + } else { + // TODO: fix clustering to be done async https://issues.apache.org/jira/browse/HUDI-1590 + throw new IllegalArgumentException("Clustering cannot be triggered with deltastreamer"); + } + } + + public Option scheduleCompaction(Option> previousCommitExtraMetadata) throws + Exception { + if (cfg.useDeltaStreamer) { + deltaStreamerWrapper.scheduleCompact(); + return Option.empty(); + } else { + return writeClient.scheduleCompaction(previousCommitExtraMetadata); + } + } + + public void commit(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) { + if (!cfg.useDeltaStreamer) { + Map extraMetadata = new HashMap<>(); + /** Store the checkpoint in the commit metadata just like + * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ + extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); + if (generatedDataStats != null && generatedDataStats.count() > 1) { + // Just stores the path where this batch of data is generated to + extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); + } + writeClient.commit(instantTime.get(), records, Option.of(extraMetadata)); + } + } + + public void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) throws IOException { + if (!cfg.useDeltaStreamer) { + Map extraMetadata = new HashMap<>(); + /** Store the checkpoint in the commit metadata just like + * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ + extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); + if (generatedDataStats != null && generatedDataStats.count() > 1) { + // Just stores the path where this batch of data is generated to + extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); + } + HoodieSparkTable table = HoodieSparkTable.create(writeClient.getConfig(), writeClient.getEngineContext()); + HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(table, instantTime.get(), HoodieJavaRDD.of(records), writeClient.getConfig().getSchema()); + writeClient.commitCompaction(instantTime.get(), metadata, Option.of(extraMetadata)); + } + } +} 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 2d9f841ae351c..5e2f9812ba529 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 @@ -190,11 +190,12 @@ public WorkflowDag createWorkflowDag() throws IOException { } public void runTestSuite() { + WriterContext writerContext = null; try { WorkflowDag workflowDag = createWorkflowDag(); log.info("Workflow Dag => " + DagUtils.convertDagToYaml(workflowDag)); long startTime = System.currentTimeMillis(); - WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); + writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); writerContext.initContext(jsc); startOtherServicesIfNeeded(writerContext); if (this.cfg.saferSchemaEvolution) { @@ -217,6 +218,9 @@ public void runTestSuite() { log.error("Failed to run Test Suite ", e); throw new HoodieException("Failed to run Test Suite ", e); } finally { + if (writerContext != null) { + writerContext.shutdownResources(); + } if (stopJsc) { stopQuietly(); } @@ -310,5 +314,8 @@ public static class HoodieTestSuiteConfig extends HoodieDeltaStreamer.Config { @Parameter(names = {"--use-hudi-data-to-generate-updates"}, description = "Use data from hudi to generate updates for new batches ") public Boolean useHudiToGenerateUpdates = false; + + @Parameter(names = {"--test-continuous-mode"}, description = "Tests continuous mode in deltastreamer.") + public Boolean testContinousMode = false; } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index a98c7f2aec3f0..7a9e122e86c15 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -18,37 +18,25 @@ package org.apache.hudi.integ.testsuite; -import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieAvroRecord; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; import org.apache.hudi.integ.testsuite.dag.nodes.CleanNode; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode; import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode; import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats; -import org.apache.hudi.table.HoodieSparkTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; @@ -57,38 +45,31 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Properties; import java.util.Set; -/** - * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are - * {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}. - */ -public class HoodieTestSuiteWriter implements Serializable { +public abstract class HoodieTestSuiteWriter implements Serializable { private static Logger log = LoggerFactory.getLogger(HoodieTestSuiteWriter.class); - private HoodieDeltaStreamerWrapper deltaStreamerWrapper; - private HoodieWriteConfig writeConfig; - private SparkRDDWriteClient writeClient; - protected HoodieTestSuiteConfig cfg; - private Option lastCheckpoint; - private HoodieReadClient hoodieReadClient; - private Properties props; - private String schema; - private transient Configuration configuration; - private transient JavaSparkContext sparkContext; - private static Set VALID_DAG_NODES_TO_ALLOW_WRITE_CLIENT_IN_DELTASTREAMER_MODE = new HashSet<>( + protected HoodieDeltaStreamerWrapper deltaStreamerWrapper; + protected HoodieWriteConfig writeConfig; + protected SparkRDDWriteClient writeClient; + protected HoodieTestSuiteJob.HoodieTestSuiteConfig cfg; + protected Option lastCheckpoint; + protected HoodieReadClient hoodieReadClient; + protected Properties props; + protected String schema; + protected transient Configuration configuration; + protected transient JavaSparkContext sparkContext; + protected static Set VALID_DAG_NODES_TO_ALLOW_WRITE_CLIENT_IN_DELTASTREAMER_MODE = new HashSet<>( Arrays.asList(RollbackNode.class.getName(), CleanNode.class.getName(), ScheduleCompactNode.class.getName())); - private static final String GENERATED_DATA_PATH = "generated.data.path"; - public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws Exception { + public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteJob.HoodieTestSuiteConfig cfg, String schema) throws Exception { // We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter // This does not instantiate a HoodieWriteClient until a // {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked. @@ -110,7 +91,7 @@ public HoodieWriteConfig getWriteConfig() { return this.writeConfig; } - private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteConfig cfg, Properties props, String schema) { + private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteJob.HoodieTestSuiteConfig cfg, Properties props, String schema) { HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) .withAutoCommit(false) @@ -131,159 +112,35 @@ private boolean allowWriteClientAccess(DagNode dagNode) { return false; } - public RDD getNextBatch() throws Exception { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - JavaRDD inputRDD = nextBatch.getRight().getRight(); - return inputRDD.map(r -> (GenericRecord) ((HoodieAvroRecord) r).getData() - .getInsertValue(new Schema.Parser().parse(schema)).get()).rdd(); - } + public abstract void shutdownResources(); - public void getNextBatchForDeletes() throws Exception { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - JavaRDD inputRDD = nextBatch.getRight().getRight(); - inputRDD.collect(); - } + public abstract RDD getNextBatch() throws Exception; - public Pair>> fetchSource() throws Exception { - return this.deltaStreamerWrapper.fetchSource(); - } + public abstract Pair>> fetchSource() throws Exception ; - public Option startCommit() { - if (cfg.useDeltaStreamer) { - return Option.of(HoodieActiveTimeline.createNewInstantTime()); - } else { - return Option.of(writeClient.startCommit()); - } - } + public abstract Option startCommit(); - public JavaRDD upsert(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.upsert(WriteOperationType.UPSERT); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.upsert(nextBatch.getRight().getRight(), instantTime.get()); - } - } + public abstract JavaRDD upsert(Option instantTime) throws Exception; - public JavaRDD insert(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.insert(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.insert(nextBatch.getRight().getRight(), instantTime.get()); - } - } + public abstract JavaRDD insert(Option instantTime) throws Exception; - public JavaRDD insertOverwrite(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.insertOverwrite(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.insertOverwrite(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); - } - } + public abstract JavaRDD insertOverwrite(Option instantTime) throws Exception; - public JavaRDD insertOverwriteTable(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.insertOverwriteTable(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.insertOverwriteTable(nextBatch.getRight().getRight(), instantTime.get()).getWriteStatuses(); - } - } + public abstract JavaRDD insertOverwriteTable(Option instantTime) throws Exception; - public JavaRDD bulkInsert(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.bulkInsert(); - } else { - Pair>> nextBatch = fetchSource(); - lastCheckpoint = Option.of(nextBatch.getValue().getLeft()); - return writeClient.bulkInsert(nextBatch.getRight().getRight(), instantTime.get()); - } - } + public abstract JavaRDD bulkInsert(Option instantTime) throws Exception; - public JavaRDD compact(Option instantTime) throws Exception { - if (cfg.useDeltaStreamer) { - return deltaStreamerWrapper.compact(); - } else { - if (!instantTime.isPresent()) { - Option> compactionPlanPair = Option - .fromJavaOptional(hoodieReadClient.getPendingCompactions() - .stream().findFirst()); - if (compactionPlanPair.isPresent()) { - instantTime = Option.of(compactionPlanPair.get().getLeft()); - } - } - if (instantTime.isPresent()) { - HoodieWriteMetadata> compactionMetadata = writeClient.compact(instantTime.get()); - return compactionMetadata.getWriteStatuses(); - } else { - return null; - } - } - } + public abstract JavaRDD compact(Option instantTime) throws Exception; - public void inlineClustering() { - if (!cfg.useDeltaStreamer) { - Option clusteringInstantOpt = writeClient.scheduleClustering(Option.empty()); - clusteringInstantOpt.ifPresent(clusteringInstant -> { - // inline cluster should auto commit as the user is never given control - log.warn("Clustering instant :: " + clusteringInstant); - writeClient.cluster(clusteringInstant, true); - }); - } else { - // TODO: fix clustering to be done async https://issues.apache.org/jira/browse/HUDI-1590 - throw new IllegalArgumentException("Clustering cannot be triggered with deltastreamer"); - } - } + public abstract void inlineClustering() throws Exception ; - public Option scheduleCompaction(Option> previousCommitExtraMetadata) throws - Exception { - if (cfg.useDeltaStreamer) { - deltaStreamerWrapper.scheduleCompact(); - return Option.empty(); - } else { - return writeClient.scheduleCompaction(previousCommitExtraMetadata); - } - } + public abstract Option scheduleCompaction(Option> previousCommitExtraMetadata) throws Exception; - public void commit(JavaRDD records, JavaRDD generatedDataStats, - Option instantTime) { - if (!cfg.useDeltaStreamer) { - Map extraMetadata = new HashMap<>(); - /** Store the checkpoint in the commit metadata just like - * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ - extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); - if (generatedDataStats != null && generatedDataStats.count() > 1) { - // Just stores the path where this batch of data is generated to - extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); - } - writeClient.commit(instantTime.get(), records, Option.of(extraMetadata)); - } - } + public abstract void commit(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime); - public void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, - Option instantTime) throws IOException { - if (!cfg.useDeltaStreamer) { - Map extraMetadata = new HashMap<>(); - /** Store the checkpoint in the commit metadata just like - * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/ - extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get()); - if (generatedDataStats != null && generatedDataStats.count() > 1) { - // Just stores the path where this batch of data is generated to - extraMetadata.put(GENERATED_DATA_PATH, generatedDataStats.map(s -> s.getFilePath()).collect().get(0)); - } - HoodieSparkTable table = HoodieSparkTable.create(writeClient.getConfig(), writeClient.getEngineContext()); - HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(table, instantTime.get(), HoodieJavaRDD.of(records), writeClient.getConfig().getSchema()); - writeClient.commitCompaction(instantTime.get(), metadata, Option.of(extraMetadata)); - } - } + public abstract void commitCompaction(JavaRDD records, JavaRDD generatedDataStats, + Option instantTime) throws Exception; public SparkRDDWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException { if (cfg.useDeltaStreamer & !allowWriteClientAccess(dagNode)) { @@ -301,7 +158,7 @@ public HoodieDeltaStreamerWrapper getDeltaStreamerWrapper() { return deltaStreamerWrapper; } - public HoodieTestSuiteConfig getCfg() { + public HoodieTestSuiteJob.HoodieTestSuiteConfig getCfg() { return cfg; } @@ -325,3 +182,4 @@ public String getSchema() { return schema; } } + diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java index 1578e86be47b6..a781d19cb78c5 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java @@ -103,6 +103,7 @@ public static class Config { private static String DELETE_INPUT_DATA_EXCEPT_LATEST = "delete_input_data_except_latest"; private static String PARTITIONS_TO_DELETE = "partitions_to_delete"; private static String INPUT_PARTITIONS_TO_SKIP_VALIDATE = "input_partitions_to_skip_validate"; + private static String MAX_WAIT_TIME_FOR_DELTASTREAMER_TO_CATCH_UP_MS = "max_wait_time_for_deltastreamer_catch_up_ms"; // Spark SQL Create Table private static String TABLE_TYPE = "table_type"; @@ -253,6 +254,10 @@ public boolean enableRowWriting() { return Boolean.valueOf(configsMap.getOrDefault(ENABLE_ROW_WRITING, false).toString()); } + public long maxWaitTimeForDeltastreamerToCatchupMs() { + return Long.valueOf(configsMap.getOrDefault(MAX_WAIT_TIME_FOR_DELTASTREAMER_TO_CATCH_UP_MS, 5 * 60 * 1000).toString()); + } + public Option getTableType() { return !configsMap.containsKey(TABLE_TYPE) ? Option.empty() : Option.of(configsMap.get(TABLE_TYPE).toString()); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index d31ef195ecdd5..83b5751c8646b 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -21,7 +21,9 @@ import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.integ.testsuite.HoodieContinousTestSuiteWriter; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; +import org.apache.hudi.integ.testsuite.HoodieInlineTestSuiteWriter; import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; @@ -37,6 +39,8 @@ import org.apache.spark.sql.SparkSession; import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * WriterContext wraps the delta writer/data generator related configuration needed to init/reinit. @@ -53,6 +57,7 @@ public class WriterContext { private BuiltinKeyGenerator keyGenerator; private transient SparkSession sparkSession; private transient JavaSparkContext jsc; + private ExecutorService executorService; public WriterContext(JavaSparkContext jsc, TypedProperties props, HoodieTestSuiteConfig cfg, BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) { @@ -67,7 +72,8 @@ public void initContext(JavaSparkContext jsc) throws HoodieException { try { this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc); String schemaStr = schemaProvider.getSourceSchema().toString(); - this.hoodieTestSuiteWriter = new HoodieTestSuiteWriter(jsc, props, cfg, schemaStr); + this.hoodieTestSuiteWriter = (cfg.testContinousMode && cfg.useDeltaStreamer) ? new HoodieContinousTestSuiteWriter(jsc, props, cfg, schemaStr) + : new HoodieInlineTestSuiteWriter(jsc, props, cfg, schemaStr); int inputParallelism = cfg.inputParallelism > 0 ? cfg.inputParallelism : jsc.defaultParallelism(); this.deltaGenerator = new DeltaGenerator( new DFSDeltaConfig(DeltaOutputMode.valueOf(cfg.outputTypeName), DeltaInputType.valueOf(cfg.inputFormatName), @@ -75,6 +81,10 @@ public void initContext(JavaSparkContext jsc) throws HoodieException { schemaStr, cfg.limitFileSize, inputParallelism, cfg.deleteOldInput, cfg.useHudiToGenerateUpdates), jsc, sparkSession, schemaStr, keyGenerator); log.info(String.format("Initialized writerContext with: %s", schemaStr)); + if (cfg.testContinousMode) { + executorService = Executors.newFixedThreadPool(1); + executorService.execute(new TestSuiteWriterRunnable(hoodieTestSuiteWriter)); + } } catch (Exception e) { throw new HoodieException("Failed to reinitialize writerContext", e); } @@ -113,4 +123,35 @@ public String toString() { public SparkSession getSparkSession() { return sparkSession; } + + public void shutdownResources() { + this.hoodieTestSuiteWriter.shutdownResources(); + if (executorService != null) { + executorService.shutdownNow(); + } + } + + /** + * TestSuiteWriterRunnable to spin up a thread to execute deltastreamer with async table services. + */ + class TestSuiteWriterRunnable implements Runnable { + private HoodieTestSuiteWriter hoodieTestSuiteWriter; + + TestSuiteWriterRunnable(HoodieTestSuiteWriter hoodieTestSuiteWriter) { + this.hoodieTestSuiteWriter = hoodieTestSuiteWriter; + } + + @Override + public void run() { + try { + Thread.sleep(20000); + log.info("Starting continuous sync with deltastreamer "); + hoodieTestSuiteWriter.getDeltaStreamerWrapper().sync(); + log.info("Completed continuous sync with deltastreamer "); + } catch (Exception e) { + log.error("Deltastreamer failed in continuous mode " + e.getMessage()); + throw new HoodieException("Shutting down deltastreamer in continuous mode failed ", e); + } + } + } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java index a0ebdc5754716..15c209e4752b8 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java @@ -20,10 +20,17 @@ package org.apache.hudi.integ.testsuite.dag.nodes; import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.dag.ExecutionContext; import org.apache.hudi.integ.testsuite.schema.SchemaUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -40,6 +47,9 @@ import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; @@ -47,6 +57,8 @@ import scala.collection.JavaConversions; import scala.collection.JavaConverters; +import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; + /** * This nodes validates contents from input path are in tact with Hudi. By default no configs are required for this node. But there is an * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. @@ -78,6 +90,12 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception int itrCountToExecute = config.getIterationCountToExecute(); if ((itrCountToExecute != -1 && itrCountToExecute == curItrCount) || (itrCountToExecute == -1 && ((curItrCount % validateOnceEveryItr) == 0))) { + FileSystem fs = new Path(context.getHoodieTestSuiteWriter().getCfg().inputBasePath) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + if (context.getHoodieTestSuiteWriter().getCfg().testContinousMode) { + awaitUntilDeltaStreamerCaughtUp(context, context.getHoodieTestSuiteWriter().getCfg().targetBasePath, fs, + context.getHoodieTestSuiteWriter().getCfg().inputBasePath); + } SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); // todo: Fix partitioning schemes. For now, assumes data based partitioning. String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; @@ -85,8 +103,6 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception // listing batches to be validated String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; if (log.isDebugEnabled()) { - FileSystem fs = new Path(inputPathStr) - .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); log.info("fileStatuses length: " + fileStatuses.length); for (FileStatus fileStatus : fileStatuses) { @@ -145,8 +161,6 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception if (config.isDeleteInputData()) { // clean up input data for current group of writes. inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; - FileSystem fs = new Path(inputPathStr) - .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); for (FileStatus fileStatus : fileStatuses) { log.debug("Micro batch to be deleted " + fileStatus.getPath().toString()); @@ -157,6 +171,50 @@ public void execute(ExecutionContext context, int curItrCount) throws Exception } } + private void awaitUntilDeltaStreamerCaughtUp(ExecutionContext context, String hudiTablePath, FileSystem fs, String inputPath) throws IOException, InterruptedException { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(hudiTablePath).build(); + HoodieTimeline commitTimeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + Option latestCheckpoint = getLatestCheckpoint(commitTimeline); + FileStatus[] subDirs = fs.listStatus(new Path(inputPath)); + List subDirList = Arrays.asList(subDirs); + subDirList.sort(Comparator.comparingLong(entry -> Long.parseLong(entry.getPath().getName()))); + String latestSubDir = subDirList.get(subDirList.size() -1).getPath().getName(); + log.info("Latest sub directory in input path " + latestSubDir + ", latest checkpoint from deltastreamer " + + (latestCheckpoint.isPresent() ? latestCheckpoint.get() : "none")); + long maxWaitTime = config.maxWaitTimeForDeltastreamerToCatchupMs(); + long waitedSoFar = 0; + while (!(latestCheckpoint.isPresent() && latestCheckpoint.get().equals(latestSubDir))) { + log.warn("Sleeping for 20 secs awaiting for deltastreamer to catch up with ingested data"); + Thread.sleep(20000); + meta.reloadActiveTimeline(); + commitTimeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + latestCheckpoint = getLatestCheckpoint(commitTimeline); + waitedSoFar += 20000; + if (waitedSoFar >= maxWaitTime) { + throw new AssertionError("DeltaStreamer has not caught up after 5 mins of wait time. Last known checkpoint " + + (latestCheckpoint.isPresent() ? latestCheckpoint.get() : "none") + ", expected checkpoint to have caugth up " + latestSubDir); + } + log.info("Latest sub directory in input path " + latestSubDir + ", latest checkpoint from deltastreamer " + + (latestCheckpoint.isPresent() ? latestCheckpoint.get() : "none")); + } + } + + private Option getLatestCheckpoint(HoodieTimeline timeline) { + return (Option) timeline.getReverseOrderedInstants().map(instant -> { + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_KEY))) { + return Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); + } else { + return Option.empty(); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to parse HoodieCommitMetadata for " + instant.toString(), e); + } + }).filter(Option::isPresent).findFirst().orElse(Option.empty()); + } + private Dataset getInputDf(ExecutionContext context, SparkSession session, String inputPath) { String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD().key()); String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD().key()); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index c30be2a2a5d2c..20e12e9030854 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -216,15 +216,22 @@ public JavaRDD generateDeletes(Config config) throws IOException adjustedRDD = deltaInputReader.read(config.getNumRecordsDelete()); adjustedRDD = adjustRDDToGenerateExactNumUpdates(adjustedRDD, jsc, config.getNumRecordsDelete()); } else { - deltaInputReader = - new DFSHoodieDatasetInputReader(jsc, ((DFSDeltaConfig) deltaOutputConfig).getDatasetOutputPath(), - schemaStr); - if (config.getFractionUpsertPerFile() > 0) { - adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), - config.getFractionUpsertPerFile()); + if (((DFSDeltaConfig) deltaOutputConfig).shouldUseHudiToGenerateUpdates()) { + deltaInputReader = + new DFSHoodieDatasetInputReader(jsc, ((DFSDeltaConfig) deltaOutputConfig).getDatasetOutputPath(), + schemaStr); + if (config.getFractionUpsertPerFile() > 0) { + adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), + config.getFractionUpsertPerFile()); + } else { + adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), config + .getNumRecordsDelete()); + } } else { - adjustedRDD = deltaInputReader.read(config.getNumDeletePartitions(), config.getNumUpsertFiles(), config - .getNumRecordsDelete()); + deltaInputReader = new DFSAvroDeltaInputReader(sparkSession, schemaStr, + ((DFSDeltaConfig) deltaOutputConfig).getDeltaBasePath(), Option.empty(), Option.empty()); + adjustedRDD = deltaInputReader.read(config.getNumRecordsDelete()); + adjustedRDD = adjustRDDToGenerateExactNumUpdates(adjustedRDD, jsc, config.getNumRecordsDelete()); } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java index 4b44ae4385ad3..c68bd60ba6344 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/sql/InsertMode.java @@ -38,8 +38,7 @@ public enum InsertMode { * In non-strict mode for insert into, we use insert operation * to write data which allow writing the duplicate record. */ - NON_STRICT("non-strict") - ; + NON_STRICT("non-strict"); private String value; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 24c6e21df93a9..8a9486ea45dd5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.avro.HoodieAvroUtils.getRootLevelFieldName import org.apache.hudi.common.model.DefaultHoodieRecordPayload import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.ValidationUtils @@ -198,14 +199,14 @@ object HoodieOptionConfig { .map(_.split(",").filter(_.length > 0)) ValidationUtils.checkArgument(primaryKeys.nonEmpty, "No `primaryKey` is specified.") primaryKeys.get.foreach { primaryKey => - ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, primaryKey)), + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, getRootLevelFieldName(primaryKey))), s"Can't find primaryKey `$primaryKey` in ${schema.treeString}.") } // validate preCombine key val preCombineKey = sqlOptions.get(SQL_KEY_PRECOMBINE_FIELD.sqlKeyName) if (preCombineKey.isDefined && preCombineKey.get.nonEmpty) { - ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, preCombineKey.get)), + ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, getRootLevelFieldName(preCombineKey.get))), s"Can't find preCombineKey `${preCombineKey.get}` in ${schema.treeString}.") } diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql index e19dd1eb6b8ba..3e92d31e3a3b4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql @@ -1,3 +1,18 @@ +# 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. # SET OPTION set hoodie.insert.shuffle.parallelism = 1; diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala index 48bb46f81b1b0..6f13dbc82f4d9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala @@ -26,14 +26,14 @@ import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.keygen.TimestampBasedKeyGenerator import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config -import org.apache.hudi.keygen.{ComplexKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} -import org.junit.jupiter.api.{Disabled, Tag} +import org.junit.jupiter.api.Tag import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{CsvSource, ValueSource} @@ -51,31 +51,33 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp", - HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "false" ) val verificationCol: String = "driver" val updatedVerificationVal: String = "driver_update" @ParameterizedTest - @CsvSource(Array( - "true,org.apache.hudi.keygen.SimpleKeyGenerator", - "true,org.apache.hudi.keygen.ComplexKeyGenerator", - "true,org.apache.hudi.keygen.TimestampBasedKeyGenerator", - "false,org.apache.hudi.keygen.SimpleKeyGenerator", - "false,org.apache.hudi.keygen.ComplexKeyGenerator", - "false,org.apache.hudi.keygen.TimestampBasedKeyGenerator" - )) - def testCopyOnWriteStorage(isMetadataEnabled: Boolean, keyGenClass: String): Unit = { - commonOpts += DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> keyGenClass - if (classOf[ComplexKeyGenerator].getName.equals(keyGenClass)) { - commonOpts += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key, pii_col" - } - if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { - commonOpts += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key" - commonOpts += DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "current_ts" - commonOpts += Config.TIMESTAMP_TYPE_FIELD_PROP -> "EPOCHMILLISECONDS" - commonOpts += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd" + @CsvSource(value = Array( + "true|org.apache.hudi.keygen.SimpleKeyGenerator|_row_key", + "true|org.apache.hudi.keygen.ComplexKeyGenerator|_row_key,nation.bytes", + "true|org.apache.hudi.keygen.TimestampBasedKeyGenerator|_row_key", + "false|org.apache.hudi.keygen.SimpleKeyGenerator|_row_key", + "false|org.apache.hudi.keygen.ComplexKeyGenerator|_row_key,nation.bytes", + "false|org.apache.hudi.keygen.TimestampBasedKeyGenerator|_row_key" + ), delimiter = '|') + def testCopyOnWriteStorage(isMetadataEnabled: Boolean, keyGenClass: String, recordKeys: String): Unit = { + var options: Map[String, String] = commonOpts + + (HoodieMetadataConfig.ENABLE.key -> String.valueOf(isMetadataEnabled)) + + (DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> keyGenClass) + + (DataSourceWriteOptions.RECORDKEY_FIELD.key() -> recordKeys) + val isTimestampBasedKeyGen: Boolean = classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass) + if (isTimestampBasedKeyGen) { + options += DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key" + options += Config.TIMESTAMP_TYPE_FIELD_PROP -> "DATE_STRING" + options += Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd" + options += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd" } val dataGen = new HoodieTestDataGenerator(0xDEED) val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration) @@ -83,14 +85,12 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val records0 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF0 = spark.read.json(spark.sparkContext.parallelize(records0, 2)) inputDF0.write.format("org.apache.hudi") - .options(commonOpts) + .options(options) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) .mode(SaveMode.Overwrite) .save(basePath) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) - val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath) // Snapshot query val snapshotDF1 = spark.read.format("org.apache.hudi") @@ -102,7 +102,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) val verificationRowKey = inputDF1.limit(1).select("_row_key").first.getString(0) var updateDf: DataFrame = null - if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + if (isTimestampBasedKeyGen) { // update current_ts to be same as original record so that partition path does not change with timestamp based key gen val originalRow = snapshotDF1.filter(col("_row_key") === verificationRowKey).collectAsList().get(0) updateDf = inputDF1.filter(col("_row_key") === verificationRowKey) @@ -116,8 +116,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { } updateDf.write.format("org.apache.hudi") - .options(commonOpts) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) + .options(options) .mode(SaveMode.Append) .save(basePath) val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath) @@ -132,7 +131,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val records2 = recordsToStrings(dataGen.generateUpdates("002", 100)).toList var inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) - if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) { + if (isTimestampBasedKeyGen) { // incase of Timestamp based key gen, current_ts should not be updated. but dataGen.generateUpdates() would have updated // the value of current_ts. So, we need to revert it back to original value. // here is what we are going to do. Copy values to temp columns, join with original df and update the current_ts @@ -152,8 +151,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() inputDF2.write.format("org.apache.hudi") - .options(commonOpts) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) + .options(options) .mode(SaveMode.Append) .save(basePath) @@ -191,8 +189,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val emptyRecords = recordsToStrings(dataGen.generateUpdates("003", 0)).toList val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1)) emptyDF.write.format("org.apache.hudi") - .options(commonOpts) - .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled) + .options(options) .mode(SaveMode.Append) .save(basePath) @@ -211,9 +208,10 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val hoodieIncViewDF3 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commitInstantTime2) - .option(DataSourceReadOptions.INCR_PATH_GLOB.key, "/2016/*/*/*") + .option(DataSourceReadOptions.INCR_PATH_GLOB.key, if (isTimestampBasedKeyGen) "/2016*/*" else "/2016/*/*/*") .load(basePath) - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count()) + assertEquals(hoodieIncViewDF2 + .filter(col("_hoodie_partition_path").startsWith("2016")).count(), hoodieIncViewDF3.count()) val timeTravelDF = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala index 5435aad05e88a..cad30eca24469 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala @@ -763,4 +763,22 @@ class TestCreateTable extends HoodieSparkSqlTestBase { assertResult(true)(shown.contains("COMMENT 'This is a simple hudi table'")) } } + + test("Test CTAS using an illegal definition -- a COW table with compaction enabled.") { + val tableName = generateTableName + checkExceptionContain( + s""" + | create table $tableName using hudi + | tblproperties( + | primaryKey = 'id', + | type = 'cow', + | hoodie.compact.inline='true' + | ) + | AS + | select 1 as id, 'a1' as name, 10 as price, 1000 as ts + |""".stripMargin)("Compaction is not supported on a CopyOnWrite table") + val dbPath = spark.sessionState.catalog.getDatabaseMetadata("default").locationUri.getPath + val tablePath = s"${dbPath}/${tableName}" + assertResult(false)(existsPath(tablePath)) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index ab75ef563f229..760d1269c2e42 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -663,4 +663,37 @@ class TestInsertTable extends HoodieSparkSqlTestBase { } } } + + test("Test nested field as primaryKey and preCombineField") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | name string, + | price double, + | ts long, + | nestedcol struct>> + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | options ( + | type = '$tableType', + | primaryKey = 'nestedcol.a1', + | preCombineField = 'nestedcol.a2.b2.c2' + | ) + """.stripMargin) + // insert data to table + spark.sql( + s"""insert into $tableName values + |('name_1', 10, 1000, struct('a', struct('b', struct('c', 999)))), + |('name_2', 20, 2000, struct('a', struct('b', struct('c', 333)))) + |""".stripMargin) + checkAnswer(s"select name, price, ts, nestedcol.a1, nestedcol.a2.b2.c2 from $tableName")( + Seq("name_1", 10.0, 1000, "a", 999) + ) + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml index 679579ae9a5e6..3d7f61c290f46 100644 --- a/hudi-spark-datasource/hudi-spark2/pom.xml +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -203,19 +203,6 @@ true - - io.netty - netty - 3.9.9.Final - true - - - io.netty - netty-all - 4.1.17.Final - true - - org.apache.hudi diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala index f30976f58ea26..67012c7723db2 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, ProvidesHoodieConfig} import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{Dataset, SaveMode, SparkSession, _} +import java.net.URI import java.util import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter} @@ -50,7 +51,9 @@ class HoodieCatalog extends DelegatingCatalogExtension override def stageCreate(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { if (sparkAdapter.isHoodieTable(properties)) { - HoodieStagedTable(ident, this, schema, partitions, properties, TableCreationMode.STAGE_CREATE) + val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties) + HoodieStagedTable(ident, locUriAndTableType, this, schema, partitions, + properties, TableCreationMode.STAGE_CREATE) } else { BasicStagedTable( ident, @@ -61,7 +64,9 @@ class HoodieCatalog extends DelegatingCatalogExtension override def stageReplace(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { if (sparkAdapter.isHoodieTable(properties)) { - HoodieStagedTable(ident, this, schema, partitions, properties, TableCreationMode.STAGE_REPLACE) + val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties) + HoodieStagedTable(ident, locUriAndTableType, this, schema, partitions, + properties, TableCreationMode.STAGE_REPLACE) } else { super.dropTable(ident) BasicStagedTable( @@ -76,8 +81,9 @@ class HoodieCatalog extends DelegatingCatalogExtension partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { if (sparkAdapter.isHoodieTable(properties)) { - HoodieStagedTable( - ident, this, schema, partitions, properties, TableCreationMode.CREATE_OR_REPLACE) + val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties) + HoodieStagedTable(ident, locUriAndTableType, this, schema, partitions, + properties, TableCreationMode.CREATE_OR_REPLACE) } else { try super.dropTable(ident) catch { case _: NoSuchTableException => // ignore the exception @@ -112,7 +118,9 @@ class HoodieCatalog extends DelegatingCatalogExtension schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): Table = { - createHoodieTable(ident, schema, partitions, properties, Map.empty, Option.empty, TableCreationMode.CREATE) + val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties) + createHoodieTable(ident, schema, locUriAndTableType, partitions, properties, + Map.empty, Option.empty, TableCreationMode.CREATE) } override def tableExists(ident: Identifier): Boolean = super.tableExists(ident) @@ -193,8 +201,30 @@ class HoodieCatalog extends DelegatingCatalogExtension loadTable(ident) } + private def deduceTableLocationURIAndTableType( + ident: Identifier, properties: util.Map[String, String]): (URI, CatalogTableType) = { + val locOpt = if (isPathIdentifier(ident)) { + Option(ident.name()) + } else { + Option(properties.get("location")) + } + val tableType = if (locOpt.nonEmpty) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + val locUriOpt = locOpt.map(CatalogUtils.stringToURI) + val tableIdent = ident.asTableIdentifier + val existingTableOpt = getExistingTableIfExists(tableIdent) + val locURI = locUriOpt + .orElse(existingTableOpt.flatMap(_.storage.locationUri)) + .getOrElse(spark.sessionState.catalog.defaultTablePath(tableIdent)) + (locURI, tableType) + } + def createHoodieTable(ident: Identifier, schema: StructType, + locUriAndTableType: (URI, CatalogTableType), partitions: Array[Transform], allTableProperties: util.Map[String, String], writeOptions: Map[String, String], @@ -206,29 +236,17 @@ class HoodieCatalog extends DelegatingCatalogExtension val newPartitionColumns = partitionColumns val newBucketSpec = maybeBucketSpec - val isByPath = isPathIdentifier(ident) - - val location = if (isByPath) Option(ident.name()) else Option(allTableProperties.get("location")) - val id = ident.asTableIdentifier - - val locUriOpt = location.map(CatalogUtils.stringToURI) - val existingTableOpt = getExistingTableIfExists(id) - val loc = locUriOpt - .orElse(existingTableOpt.flatMap(_.storage.locationUri)) - .getOrElse(spark.sessionState.catalog.defaultTablePath(id)) val storage = DataSource.buildStorageFormatFromOptions(writeOptions.--(needFilterProps)) - .copy(locationUri = Option(loc)) - val tableType = - if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + .copy(locationUri = Option(locUriAndTableType._1)) val commentOpt = Option(allTableProperties.get("comment")) val tablePropertiesNew = new util.HashMap[String, String](allTableProperties) // put path to table properties. - tablePropertiesNew.put("path", loc.getPath) + tablePropertiesNew.put("path", locUriAndTableType._1.getPath) val tableDesc = new CatalogTable( - identifier = id, - tableType = tableType, + identifier = ident.asTableIdentifier, + tableType = locUriAndTableType._2, storage = storage, schema = newSchema, provider = Option("hudi"), diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala index 4034862167aa5..e18f23ebde03f 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala @@ -21,16 +21,18 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceWriteOptions.RECORDKEY_FIELD import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, SupportsWrite, TableCapability} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write, WriteBuilder} -import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.StructType +import java.net.URI import java.util import scala.collection.JavaConverters.{mapAsScalaMapConverter, setAsJavaSetConverter} case class HoodieStagedTable(ident: Identifier, + locUriAndTableType: (URI, CatalogTableType), catalog: HoodieCatalog, override val schema: StructType, partitions: Array[Transform], @@ -59,13 +61,14 @@ case class HoodieStagedTable(ident: Identifier, props.putAll(properties) props.put("hoodie.table.name", ident.name()) props.put(RECORDKEY_FIELD.key, properties.get("primaryKey")) - catalog.createHoodieTable(ident, schema, partitions, props, writeOptions, sourceQuery, mode) + catalog.createHoodieTable( + ident, schema, locUriAndTableType, partitions, props, writeOptions, sourceQuery, mode) } override def name(): String = ident.name() override def abortStagedChanges(): Unit = { - clearTablePath(properties.get("location"), catalog.spark.sparkContext.hadoopConfiguration) + clearTablePath(locUriAndTableType._1.getPath, catalog.spark.sparkContext.hadoopConfiguration) } private def clearTablePath(tablePath: String, conf: Configuration): Unit = { @@ -85,13 +88,9 @@ case class HoodieStagedTable(ident: Identifier, * WriteBuilder for creating a Hoodie table. */ private class HoodieV1WriteBuilder extends WriteBuilder { - override def build(): V1Write = new V1Write { - override def toInsertableRelation(): InsertableRelation = { - new InsertableRelation { - override def insert(data: DataFrame, overwrite: Boolean): Unit = { - sourceQuery = Option(data) - } - } + override def build(): V1Write = () => { + (data: DataFrame, overwrite: Boolean) => { + sourceQuery = Option(data) } } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java index abe044cb114f4..127fc7a438724 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HiveSyncMode.java @@ -37,8 +37,7 @@ public enum HiveSyncMode { /** * The JDBC mode use hive jdbc to sync metadata. */ - JDBC - ; + JDBC; public static HiveSyncMode of(String syncMode) { return HiveSyncMode.valueOf(syncMode.toUpperCase(Locale.ROOT)); 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 3cdbe0d8bb757..8be2ace89f8f1 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 @@ -234,7 +234,7 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil fileSystem.mkdirs(partPath); List writeStats = new ArrayList<>(); String fileId = UUID.randomUUID().toString(); - Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(instantTime, "1-0-1", fileId)); + Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId)); Schema schema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, schemaFileName); generateParquetDataWithSchema(filePath, schema); HoodieWriteStat writeStat = new HoodieWriteStat(); @@ -371,7 +371,7 @@ private static List createTestData(Path partPath, boolean isPar for (int i = 0; i < 5; i++) { // Create 5 files String fileId = UUID.randomUUID().toString(); - Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(instantTime, "1-0-1", fileId)); + Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId)); generateParquetData(filePath, isParquetSchemaSimple); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java index c059c63a6a6f9..c1f891fce8431 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java @@ -211,7 +211,7 @@ private List createTestData(Path partPath, boolean isParquetSch // Create 5 files String fileId = UUID.randomUUID().toString(); Path filePath = new Path(partPath.toString() + "/" + FSUtils - .makeDataFileName(commitTime, "1-0-1", fileId)); + .makeBaseFileName(commitTime, "1-0-1", fileId)); generateParquetData(filePath, isParquetSchemaSimple); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index a1a804b9ed123..a4a7e10abc004 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -840,8 +840,15 @@ private Schema getSchemaForWriteConfig(Schema targetSchema) { && SchemaCompatibility.checkReaderWriterCompatibility(InputBatch.NULL_SCHEMA, targetSchema).getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE) { // target schema is null. fetch schema from commit metadata and use it HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName).build(); - TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); - newWriteSchema = schemaResolver.getTableAvroSchema(false); + int totalCompleted = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants(); + if (totalCompleted > 0) { + try { + TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); + newWriteSchema = schemaResolver.getTableAvroSchema(false); + } catch (IllegalArgumentException e) { + LOG.warn("Could not fetch schema from table. Falling back to using target schema from schema provider"); + } + } } } return newWriteSchema; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java index dd25e7f8bebad..9d4ce71d8f25b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java @@ -99,27 +99,27 @@ public void testSnapshotCopy() throws Exception { HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"}, basePath); // Make commit1 - File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id11")); + File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id11")); file11.createNewFile(); - File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id12")); + File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id12")); file12.createNewFile(); - File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, "id13")); + File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime1, TEST_WRITE_TOKEN, "id13")); file13.createNewFile(); // Make commit2 - File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id21")); + File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id21")); file21.createNewFile(); - File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id22")); + File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id22")); file22.createNewFile(); - File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, "id23")); + File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, "id23")); file23.createNewFile(); // Make commit3 - File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id31")); + File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id31")); file31.createNewFile(); - File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id32")); + File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id32")); file32.createNewFile(); - File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, "id33")); + File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, "id33")); file33.createNewFile(); // Do a snapshot copy diff --git a/pom.xml b/pom.xml index 9e7b4ab14a559..d898d34d35e43 100644 --- a/pom.xml +++ b/pom.xml @@ -444,7 +444,6 @@ **/target/** **/generated-sources/** .github/** - **/*.sql **/*.iml .mvn/** diff --git a/rfc/README.md b/rfc/README.md index 0ccf7b1bbe285..5040860423014 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -89,4 +89,4 @@ The list of all RFCs can be found here. | 51 | [Change Data Capture](./rfc-51/rfc-51.md) | `UNDER REVIEW` | | 52 | [Introduce Secondary Index to Improve HUDI Query Performance](./rfc-52/rfc-52.md) | `UNDER REVIEW` | | 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `UNDER REVIEW` | - +| 54 | [New Table APIs and Streamline Hudi Configs](./rfc-54/rfc-54.md) | `UNDER REVIEW` |