From 0babe57446589cfa8896a4ba921e01cf01876f95 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 17 Apr 2025 15:14:37 -0700 Subject: [PATCH 01/11] [HUDI-9332] Pluggable Table Format Support - Interface Integration Address review comments Avoid local timezone conversion for completion time Handle rollback and savepoint. Add TableFormatCompletionAction functional interface. Fix bug in saveAsComplete. Use Consumer instead Add functional tests for TableFormat Fix check-style Fix renames Address comments from Vinoth Fix tests and improve test coverage Refactor FsUtils to take metaClient and remove overloaded methods --- .../cli/commands/FileSystemViewCommand.java | 3 +- .../hudi/cli/commands/RepairsCommand.java | 2 +- .../client/BaseHoodieTableServiceClient.java | 5 +- .../hudi/client/BaseHoodieWriteClient.java | 5 +- .../versioning/v1/TimelineArchiverV1.java | 5 +- .../versioning/v2/TimelineArchiverV2.java | 9 +- .../index/bloom/HoodieGlobalBloomIndex.java | 2 +- .../index/simple/HoodieGlobalSimpleIndex.java | 2 +- .../org/apache/hudi/table/HoodieTable.java | 14 +- .../BaseTableServicePlanActionExecutor.java | 3 +- .../commit/BaseCommitActionExecutor.java | 3 +- .../BaseHoodieCompactionPlanGenerator.java | 2 +- .../rollback/BaseRollbackActionExecutor.java | 3 +- .../CopyOnWriteRollbackActionExecutor.java | 13 + .../ListingBasedRollbackStrategy.java | 4 +- .../savepoint/SavepointActionExecutor.java | 8 +- .../ttl/strategy/PartitionTTLStrategy.java | 3 +- .../upgrade/EightToSevenDowngradeHandler.java | 10 +- .../table/upgrade/UpgradeDowngradeUtils.java | 2 +- .../timeline/TestCompletionTimeQueryView.java | 4 +- .../TestEightToSevenDowngradeHandler.java | 2 +- .../utils/HoodieWriterClientTestHarness.java | 4 +- .../client/HoodieFlinkTableServiceClient.java | 3 +- .../hudi/client/HoodieFlinkWriteClient.java | 2 +- ...ertOverwriteTableCommitActionExecutor.java | 3 +- .../client/TestJavaHoodieBackedMetadata.java | 2 +- ...ientOnCopyOnWriteStorageForTestFormat.java | 51 ++ .../HoodieJavaClientTestHarness.java | 5 +- .../org.apache.hudi.common.TableFormat | 18 + ...ertOverwriteTableCommitActionExecutor.java | 3 +- .../hudi/client/TestClientRollback.java | 8 +- .../functional/TestExternalPathHandling.java | 3 +- .../HoodieSparkClientTestHarness.java | 5 +- .../apache/hudi/BaseHoodieTableFileIndex.java | 6 +- .../apache/hudi/common/NativeTableFormat.java | 48 ++ .../org/apache/hudi/common/TableFormat.java | 155 ++++ .../org/apache/hudi/common/fs/FSUtils.java | 36 +- .../hudi/common/table/HoodieTableConfig.java | 24 +- .../common/table/HoodieTableMetaClient.java | 36 +- .../table/read/IncrementalQueryAnalyzer.java | 2 +- .../table/timeline/BaseHoodieTimeline.java | 12 + .../table/timeline/HoodieActiveTimeline.java | 29 +- .../timeline/HoodieInstantTimeGenerator.java | 1 + .../timeline/SkewAdjustingTimeGenerator.java | 3 +- .../timeline/TableFormatCompletionAction.java | 31 + .../common/table/timeline/TimeGenerator.java | 4 +- .../common/table/timeline/TimelineUtils.java | 4 +- .../common/table/timeline/dto/InstantDTO.java | 10 +- .../table/timeline/dto/TimelineDTO.java | 13 +- .../versioning/v1/ActiveTimelineV1.java | 63 +- .../versioning/v2/ActiveTimelineV2.java | 86 ++- .../v2/CompletionTimeQueryViewV2.java | 4 +- .../view/AbstractTableFileSystemView.java | 4 +- .../table/view/FileSystemViewManager.java | 4 +- .../hudi/common/util/ClusteringUtils.java | 8 +- .../hudi/metadata/HoodieTableMetadata.java | 41 - .../metadata/HoodieTableMetadataUtil.java | 2 +- .../metadata/NativeTableMetadataFactory.java | 59 ++ .../hudi/metadata/TableMetadataFactory.java | 44 ++ .../read/TestHoodieFileGroupReaderBase.java | 7 +- .../common/testutils/HoodieTestUtils.java | 3 + .../hudi/tableformat/TestActiveTimeline.java | 96 +++ .../TestFormatBackedTableMetadata.java | 34 + .../hudi/tableformat/TestTableFormat.java | 110 +++ .../tableformat/TestTableMetadataFactory.java | 46 ++ .../hudi/tableformat/TestTimelineFactory.java | 91 +++ .../sink/bootstrap/BootstrapOperator.java | 3 +- .../org/apache/hudi/source/FileIndex.java | 14 +- .../hudi/source/stats/FileStatsIndex.java | 7 +- .../compact/ITTestHoodieFlinkCompactor.java | 2 +- .../common/table/TestHoodieTableConfig.java | 2 +- .../reader/DFSHoodieDatasetInputReader.java | 3 +- .../HoodieMetaserverBasedTimeline.java | 4 +- .../client/HoodieMetaserverClient.java | 2 +- .../client/HoodieMetaserverClientImp.java | 3 +- ...ertOverwriteTableCommitActionExecutor.java | 5 +- .../apache/hudi/HoodieSparkSqlWriter.scala | 7 +- .../apache/hudi/SparkBaseIndexSupport.scala | 2 +- .../catalyst/catalog/HoodieCatalogTable.scala | 2 +- .../spark/sql/hudi/HoodieSqlCommonUtils.scala | 9 +- .../command/RepairHoodieTableCommand.scala | 7 +- .../PartitionBucketIndexManager.scala | 6 +- .../RepairMigratePartitionMetaProcedure.scala | 2 +- .../ShowColumnStatsOverlapProcedure.scala | 4 +- .../ShowFileSystemViewProcedure.scala | 4 +- .../ShowInvalidParquetProcedure.scala | 4 +- ...howMetadataTableColumnStatsProcedure.scala | 2 +- ...tMetadataUtilRLIandSIRecordGeneration.java | 2 +- .../apache/hudi/functional/TestBootstrap.java | 12 +- .../functional/TestHoodieBackedMetadata.java | 22 +- .../functional/TestHoodieFileSystemViews.java | 4 +- ...HoodieSparkMergeOnReadTableCompaction.java | 8 +- .../org.apache.hudi.common.TableFormat | 19 + .../org/apache/hudi/TestHoodieFileIndex.scala | 2 +- ...stHoodieSparkSqlWriterWithTestFormat.scala | 717 ++++++++++++++++++ .../hudi/functional/TestCOWDataSource.scala | 6 +- .../TestHoodieMultipleBaseFileFormat.scala | 2 +- .../hudi/common/HoodieSparkSqlTestBase.scala | 2 +- .../hudi/sync/common/HoodieSyncClient.java | 3 +- .../sync/common/util/ManifestFileWriter.java | 2 +- .../HoodieMetadataTableValidator.java | 19 +- .../utilities/HoodieSnapshotExporter.java | 3 +- .../apache/hudi/utilities/TableSizeStats.java | 8 +- .../utilities/perf/TimelineServerPerf.java | 12 +- .../hudi/utilities/streamer/StreamSync.java | 2 + .../TestHoodieMetadataTableValidator.java | 4 +- .../TestHoodieDeltaStreamer.java | 3 +- 107 files changed, 1966 insertions(+), 307 deletions(-) create mode 100644 hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java create mode 100644 hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TableFormatCompletionAction.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/metadata/NativeTableMetadataFactory.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/metadata/TableMetadataFactory.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/tableformat/TestActiveTimeline.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/tableformat/TestFormatBackedTableMetadata.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableMetadataFactory.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/tableformat/TestTimelineFactory.java create mode 100644 hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java index 977575e042cd2..8fb119a2a0567 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/FileSystemViewCommand.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.NumericUtils; + import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; @@ -270,7 +271,7 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m } instantsStream = instantsStream.filter(is -> predicate.test(maxInstant, is.requestedTime())); } - TimelineFactory timelineFactory = metaClient.getTimelineLayout().getTimelineFactory(); + TimelineFactory timelineFactory = metaClient.getTableFormat().getTimelineFactory(); HoodieTimeline filteredTimeline = timelineFactory.createDefaultTimeline(instantsStream, metaClient.getActiveTimeline()); return new HoodieTableFileSystemView(metaClient, filteredTimeline, pathInfoList); } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java index b75e5b9875d8c..868ea40eb7645 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java @@ -231,7 +231,7 @@ public String migratePartitionMeta( HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(HoodieCLI.conf); HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); List partitionPaths = - FSUtils.getAllPartitionPaths(engineContext, client.getStorage(), client.getBasePath(), false); + FSUtils.getAllPartitionPaths(engineContext, client, false); StoragePath basePath = client.getBasePath(); String[][] rows = new String[partitionPaths.size()][]; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index df8c574cdff39..cd06a875d1a65 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -402,7 +402,7 @@ public void commitLogCompaction(String compactionInstantTime, HoodieWriteMetadat HoodieTable table = tableOpt.orElseGet(() -> createTable(config, context.getStorageConf())); completeLogCompaction(writeMetadata.getCommitMetadata().get(), table, compactionInstantTime, tableWriteStats.getMetadataTableWriteStats()); } - + /** * Schedules a new log compaction instant. * @@ -575,7 +575,8 @@ private void completeClustering(HoodieReplaceCommitMetadata replaceCommitMetadat LOG.info("Committing Clustering {} for table {}", clusteringCommitTime, table.getConfig().getBasePath()); - ClusteringUtils.transitionClusteringOrReplaceInflightToComplete(false, clusteringInstant, replaceCommitMetadata, table.getActiveTimeline()); + ClusteringUtils.transitionClusteringOrReplaceInflightToComplete(false, clusteringInstant, replaceCommitMetadata, table.getActiveTimeline(), + completedInstant -> table.getMetaClient().getTableFormat().commit(replaceCommitMetadata, completedInstant, table.getContext(), table.getMetaClient(), table.getViewManager())); LOG.debug("Clustering {} finished with result {}", clusteringCommitTime, replaceCommitMetadata); } catch (Exception e) { throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e); 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 3cad79476da80..6496c89cdf9c6 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 @@ -316,7 +316,10 @@ protected void commit(HoodieTable table, } // update Metadata table writeToMetadataTable(skipStreamingWritesToMetadataTable, table, instantTime, tableWriteStats.getMetadataTableWriteStats(), metadata); - activeTimeline.saveAsComplete(false, table.getMetaClient().createNewInstant(HoodieInstant.State.INFLIGHT, commitActionType, instantTime), Option.of(metadata)); + activeTimeline.saveAsComplete(false, + table.getMetaClient().createNewInstant(HoodieInstant.State.INFLIGHT, commitActionType, instantTime), Option.of(metadata), + completedInstant -> table.getMetaClient().getTableFormat().commit(metadata, completedInstant, getEngineContext(), table.getMetaClient(), table.getViewManager()) + ); // update cols to Index as applicable HoodieColumnStatsIndexUtils.updateColsToIndex(table, config, metadata, commitActionType, (Functions.Function2, Void>) (metaClient, columnsToIndex) -> { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java index 8da69585b0445..db2ed4f5799fc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java @@ -297,7 +297,7 @@ private List getInstantsToArchive() throws IOException { // If metadata table is enabled, do not archive instants which are more recent than the last compaction on the // metadata table. if (config.isMetadataTableEnabled() && table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), table.getStorage(), config.getMetadataConfig(), config.getBasePath())) { + try (HoodieTableMetadata tableMetadata = table.refreshAndGetTableMetadata()) { Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); if (!latestCompactionTime.isPresent()) { LOG.info("Not archiving as there is no compaction yet on the metadata table"); @@ -400,7 +400,8 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo completedInstants.stream() .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant)); } - + // Call Table Format archive to allow archiving in table format. + table.getMetaClient().getTableFormat().archive(archivedInstants, table.getContext(), table.getMetaClient(), table.getViewManager()); return true; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java index e18617232e519..8803583bcd4ac 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java @@ -51,6 +51,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -117,6 +118,11 @@ public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) t deleteArchivedActions(instantsToArchive, context); // triggers compaction and cleaning only after archiving action this.timelineWriter.compactAndClean(context); + List archivedInstants = instantsToArchive.stream() + .map(action -> Stream.concat(action.getCompletedInstants().stream(), action.getPendingInstants().stream()).collect(Collectors.toList())) + .flatMap(Collection::stream).collect(Collectors.toList()); + // Call Table Format archive to allow archiving in table format. + table.getMetaClient().getTableFormat().archive(archivedInstants, table.getContext(), table.getMetaClient(), table.getViewManager()); } else { LOG.info("No Instants to archive"); } @@ -209,8 +215,7 @@ private List getCommitInstantsToArchive() throws IOException { // 4. If metadata table is enabled, do not archive instants which are more recent than the last compaction on the // metadata table. if (config.isMetadataTableEnabled() && table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create( - table.getContext(), table.getStorage(), config.getMetadataConfig(), config.getBasePath())) { + try (HoodieTableMetadata tableMetadata = table.refreshAndGetTableMetadata()) { Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); if (!latestCompactionTime.isPresent()) { LOG.info("Not archiving as there is no compaction yet on the metadata table"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java index 25ca3984bdeaa..f94788197ad71 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java @@ -57,7 +57,7 @@ public HoodieGlobalBloomIndex(HoodieWriteConfig config, BaseHoodieBloomIndexHelp List> loadColumnRangesFromFiles(List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - List allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getStorage(), config.getMetadataConfig(), metaClient.getBasePath()); + List allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient, config.getMetadataConfig()); return super.loadColumnRangesFromFiles(allPartitionPaths, context, hoodieTable); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java index 4abc3cb1d87ba..336f6376bcd03 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java @@ -94,7 +94,7 @@ private HoodiePairData fetchRecordGlobalLoca private List> getAllBaseFilesInTable( final HoodieEngineContext context, final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - List allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient.getStorage(), config.getMetadataConfig(), metaClient.getBasePath()); + List allPartitionPaths = FSUtils.getAllPartitionPaths(context, metaClient, config.getMetadataConfig()); // Obtain the latest data files from all the partitions. return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable); } 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 70a3e8e343427..667e2a183a77d 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 @@ -183,7 +183,7 @@ public HoodieTableVersion version() { protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); - private synchronized FileSystemViewManager getViewManager() { + public synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { viewManager = FileSystemViewManager.createViewManager(getContext(), config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), unused -> getMetadataTable()); } @@ -1166,14 +1166,18 @@ private void clearMetadataTablePartitionsConfig(Option pa public HoodieTableMetadata getMetadataTable() { if (metadata == null) { - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() - .fromProperties(config.getMetadataConfig().getProps()) - .build(); - metadata = HoodieTableMetadata.create(context, metaClient.getStorage(), metadataConfig, config.getBasePath()); + metadata = refreshAndGetTableMetadata(); } return metadata; } + public HoodieTableMetadata refreshAndGetTableMetadata() { + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .fromProperties(config.getMetadataConfig().getProps()) + .build(); + return metaClient.getTableFormat().getMetadataFactory().create(context, metaClient.getStorage(), metadataConfig, config.getBasePath()); + } + /** * When {@link HoodieTableConfig#POPULATE_META_FIELDS} is enabled, * we need to track written records within WriteStatus in two cases: diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseTableServicePlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseTableServicePlanActionExecutor.java index ad4edb2433def..f4ab580d6080a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseTableServicePlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseTableServicePlanActionExecutor.java @@ -102,8 +102,7 @@ public List getPartitions(Object strategy, TableServiceType type) { // get all partitions LOG.info("Start to fetch all partitions for " + type + ". Instant " + instantTime); - return FSUtils.getAllPartitionPaths(context, table.getMetaClient().getStorage(), - config.getMetadataConfig(), table.getMetaClient().getBasePath()); + return FSUtils.getAllPartitionPaths(context, table.getMetaClient(), config.getMetadataConfig()); } public Pair, Set> getIncrementalPartitions(TableServiceType type) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 549afc7d90d66..d4e6eca9077da 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -234,7 +234,8 @@ protected void commit(HoodieWriteMetadata result, List write // cannot serialize maps with null values metadata.getExtraMetadata().entrySet().removeIf(entry -> entry.getValue() == null); activeTimeline.saveAsComplete(false, - table.getMetaClient().createNewInstant(State.INFLIGHT, actionType, instantTime), Option.of(metadata)); + table.getMetaClient().createNewInstant(State.INFLIGHT, actionType, instantTime), Option.of(metadata), + completedInstant -> table.getMetaClient().getTableFormat().commit(metadata, completedInstant, table.getContext(), table.getMetaClient(), table.getViewManager())); LOG.info("Committed " + instantTime); result.setCommitMetadata(Option.of(metadata)); // update cols to Index as applicable diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java index 09638bebca36d..53c55ba0bc8b8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java @@ -84,7 +84,7 @@ public HoodieCompactionPlan generateCompactionPlan(String compactionInstant) thr // TODO : check if maxMemory is not greater than JVM or executor memory // TODO - rollback any compactions in flight HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - CompletionTimeQueryView completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); + CompletionTimeQueryView completionTimeQueryView = metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(metaClient); List partitionPaths = getPartitions(); int allPartitionSize = partitionPaths.size(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 2c17de655bc13..246a7afd346b9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -288,7 +288,8 @@ protected void finishRollback(HoodieInstant inflightInstant, HoodieRollbackMetad // NOTE: no need to lock here, since !skipTimelinePublish is always true, // when skipLocking is false, txnManager above-mentioned should lock it. // when skipLocking is true, the caller should have already held the lock. - table.getActiveTimeline().transitionRollbackInflightToComplete(false, inflightInstant, rollbackMetadata); + table.getActiveTimeline().transitionRollbackInflightToComplete(false, inflightInstant, rollbackMetadata, + completedInstant -> table.getMetaClient().getTableFormat().completedRollback(completedInstant, table.getContext(), table.getMetaClient(), table.getViewManager())); LOG.info("Rollback of Commits " + rollbackMetadata.getCommitsRollback() + " is complete"); } } finally { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java index 7abb3e7a931a8..80e4e5b80732c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -67,11 +68,23 @@ protected List executeRollback(HoodieRollbackPlan hoodieRoll if (instantToRollback.isCompleted()) { LOG.info("Unpublishing instant " + instantToRollback); + table.getMetaClient().getTableFormat().rollback(instantToRollback, table.getContext(), table.getMetaClient(), table.getViewManager()); + // Revert the completed instant to inflight in native format. resolvedInstant = activeTimeline.revertToInflight(instantToRollback); // reload meta-client to reflect latest timeline status table.getMetaClient().reloadActiveTimeline(); } + // If instant is inflight but marked as completed in native format, delete the completed instant from storage. + if (instantToRollback.isInflight()) { + HoodieActiveTimeline activeTimelineForNativeFormat = table.getMetaClient().getActiveTimelineForNativeFormat(); + Option instantToRollbackInNativeFormat = activeTimelineForNativeFormat.filter(instant -> instant.requestedTime().equals(instantToRollback.requestedTime())).lastInstant(); + if (instantToRollbackInNativeFormat.isPresent() && instantToRollbackInNativeFormat.get().isCompleted()) { + resolvedInstant = activeTimelineForNativeFormat.revertToInflight(instantToRollbackInNativeFormat.get()); + table.getMetaClient().reloadActiveTimeline(); + } + } + // For Requested State (like failure during index lookup), there is nothing to do rollback other than // deleting the timeline file if (!resolvedInstant.isRequested()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java index d3c1f35b7d8cf..28905f75f75d5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java @@ -101,7 +101,7 @@ public List getRollbackRequests(HoodieInstant instantToRo HoodieTableMetaClient metaClient = table.getMetaClient(); boolean isTableVersionLessThanEight = metaClient.getTableConfig().getTableVersion().lesserThan(HoodieTableVersion.EIGHT); List partitionPaths = - FSUtils.getAllPartitionPaths(context, table.getStorage(), table.getMetaClient().getBasePath(), false); + FSUtils.getAllPartitionPaths(context, table.getMetaClient(), false); int numPartitions = Math.max(Math.min(partitionPaths.size(), config.getRollbackParallelism()), 1); context.setJobStatus(this.getClass().getSimpleName(), "Creating Listing Rollback Plan: " + config.getTableName()); @@ -285,7 +285,7 @@ private List listAllFilesSinceCommit(String commit, String partitionPath, HoodieTableMetaClient metaClient) throws IOException { LOG.info("Collecting files to be cleaned/rolledback up for path " + partitionPath + " and commit " + commit); - CompletionTimeQueryView completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); + CompletionTimeQueryView completionTimeQueryView = metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(metaClient); StoragePathFilter filter = (path) -> { if (path.toString().contains(baseFileExtension)) { String fileCommitTime = FSUtils.getCommitTime(path.getName()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index ca4def781bb67..56ce313ef8dfd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -122,8 +122,7 @@ public HoodieSavepointMetadata execute() { return latestFiles; })); } else { - List partitions = FSUtils.getAllPartitionPaths( - context, table.getStorage(), config.getMetadataConfig(), table.getMetaClient().getBasePath()); + List partitions = FSUtils.getAllPartitionPaths(context, table.getMetaClient(), config.getMetadataConfig()); latestFilesMap = context.mapToPair(partitions, partitionPath -> { // Scan all partitions files with this commit time LOG.info("Collecting latest files in partition path " + partitionPath); @@ -143,8 +142,9 @@ public HoodieSavepointMetadata execute() { table.getActiveTimeline().createNewInstant( instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, instantTime)); table.getActiveTimeline() - .saveAsComplete(instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, instantTime), - Option.of(metadata)); + .saveAsComplete( + true, instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, instantTime), Option.of(metadata), + savepointCompletedInstant -> table.getMetaClient().getTableFormat().savepoint(savepointCompletedInstant, table.getContext(), table.getMetaClient(), table.getViewManager())); LOG.info("Savepoint " + instantTime + " created"); return metadata; } catch (HoodieIOException e) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/PartitionTTLStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/PartitionTTLStrategy.java index dadbf06b7c9fe..9d6275d5f257d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/PartitionTTLStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/PartitionTTLStrategy.java @@ -66,8 +66,7 @@ protected List getPartitionPathsForTTL() { List partitionsForTTL; if (StringUtils.isNullOrEmpty(partitionSelected)) { // Return all partition paths. - partitionsForTTL = FSUtils.getAllPartitionPaths( - hoodieTable.getContext(), hoodieTable.getStorage(), writeConfig.getMetadataConfig(), writeConfig.getBasePath()); + partitionsForTTL = FSUtils.getAllPartitionPaths(hoodieTable.getContext(), hoodieTable.getMetaClient(), writeConfig.getMetadataConfig()); } else { partitionsForTTL = Arrays.asList(partitionSelected.split(",")); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java index 04cd07162be34..4bbfef242e02a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java @@ -342,12 +342,14 @@ static void downgradeMetadataPartitions(HoodieEngineContext context, // Get base path for metadata table. StoragePath metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()); + HoodieTableMetaClient metadataMetaClient = + HoodieTableMetaClient.builder() + .setBasePath(metadataTableBasePath.toUri().toString()) + .setConf(hoodieStorage.getConf()) + .build(); // Fetch metadata partition paths. - List metadataPartitions = FSUtils.getAllPartitionPaths(context, - hoodieStorage, - metadataTableBasePath, - false); + List metadataPartitions = FSUtils.getAllPartitionPaths(context, metadataMetaClient, false); // Delete partitions. List validPartitionPaths = deleteMetadataPartition(context, metaClient, metadataPartitions); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java index 7d8ee567d0e3a..3d39a47809b2c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java @@ -107,7 +107,7 @@ public static void runCompaction(HoodieTable table, HoodieEngineContext context, */ public static void syncCompactionRequestedFileToAuxiliaryFolder(HoodieTable table) { HoodieTableMetaClient metaClient = table.getMetaClient(); - TimelineFactory timelineFactory = metaClient.getTimelineLayout().getTimelineFactory(); + TimelineFactory timelineFactory = metaClient.getTableFormat().getTimelineFactory(); InstantFileNameGenerator instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); HoodieTimeline compactionTimeline = timelineFactory.createActiveTimeline(metaClient, false).filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java index 9f290cf449797..944ae5678df82 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java @@ -77,7 +77,7 @@ void testReadCompletionTime() throws Exception { HoodieTestUtils.getDefaultStorageConf(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareTimeline(tablePath, metaClient); try (CompletionTimeQueryView view = - metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient, String.format("%08d", 3))) { + metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(metaClient, String.format("%08d", 3))) { // query completion time from LSM timeline for (int i = 3; i < 7; i++) { assertThat(view.getCompletionTime(String.format("%08d", i)).orElse(""), is(String.format("%08d", i + 1000))); @@ -108,7 +108,7 @@ void testReadStartTime() throws Exception { HoodieTestUtils.getDefaultStorageConf(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareTimeline(tablePath, metaClient); try (CompletionTimeQueryView view = - metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient, String.format("%08d", 3))) { + metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(metaClient, String.format("%08d", 3))) { // query start time from LSM timeline assertThat(getInstantTimeSetFormattedString(view, 3 + 1000, 6 + 1000), is("00000003,00000004,00000005,00000006")); // query start time from active timeline diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java index 1752e0e06b8c4..a4f7c45b3ce02 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java @@ -141,7 +141,7 @@ void testDowngradeMetadataPartitions() { MockedStatic mockedMetadataUtils = mockStatic(HoodieTableMetadataUtil.class)) { StoragePath mdtBasePath = HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()); mockedFSUtils - .when(() -> FSUtils.getAllPartitionPaths(context, hoodieStorage, mdtBasePath, false)) + .when(() -> FSUtils.getAllPartitionPaths(context, metaClient, false)) .thenReturn(SAMPLE_METADATA_PATHS); EightToSevenDowngradeHandler.downgradeMetadataPartitions(context, hoodieStorage, metaClient, tablePropsToAdd); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java index 0f2a23eb5971c..3fdad193655d9 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java @@ -1230,7 +1230,7 @@ protected void testUpsertsInternal(Function3 table.getMetaClient().getTableFormat().commit(metadata, completedInstant, table.getContext(), table.getMetaClient(), table.getViewManager())); } catch (HoodieIOException e) { throw new HoodieClusteringException( "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + clusteringCommitTime, e); 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 39e481dff1099..e33d9c5acb597 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 @@ -487,7 +487,7 @@ public Map> getPartitionToReplacedFileIds( case INSERT_OVERWRITE_TABLE: Map> partitionToExistingFileIds = new HashMap<>(); List partitionPaths = - FSUtils.getAllPartitionPaths(context, table.getStorage(), config.getMetadataConfig(), table.getMetaClient().getBasePath()); + FSUtils.getAllPartitionPaths(context, table.getMetaClient(), config.getMetadataConfig()); if (partitionPaths != null && partitionPaths.size() > 0) { context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions: " + config.getTableName()); partitionToExistingFileIds = partitionPaths.stream().parallel() diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java index f65e8a3d8b3bd..b10f6ebfdc7ab 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java @@ -50,8 +50,7 @@ protected List getAllExistingFileIds(String partitionPath) { @Override protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeResult) { Map> partitionToExistingFileIds = new HashMap<>(); - List partitionPaths = FSUtils.getAllPartitionPaths(context, - table.getStorage(), table.getMetaClient().getBasePath(), config.isMetadataTableEnabled()); + List partitionPaths = FSUtils.getAllPartitionPaths(context, table.getMetaClient(), config.isMetadataTableEnabled()); if (partitionPaths != null && partitionPaths.size() > 0) { partitionToExistingFileIds = context.mapToPair(partitionPaths, diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java index 5165ca5f3d71b..00195ea86cbeb 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java @@ -2837,7 +2837,7 @@ private void validateMetadata(HoodieJavaWriteClient testClient, Option i // Metadata table has a fixed number of partitions // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. - List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, storage, getMetadataTableBasePath(basePath), false); + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient, false); // check if the last instant is restore, then the metadata table should have only the partitions that are not deleted metaClient.reloadActiveTimeline().getReverseOrderedInstants().findFirst().ifPresent(instant -> { if (instant.getAction().equals(HoodieActiveTimeline.RESTORE_ACTION)) { diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java new file mode 100644 index 0000000000000..7ee8c427050a9 --- /dev/null +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java @@ -0,0 +1,51 @@ +/* + * 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.functional; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.tableformat.TestTableFormat; + +import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.AfterAll; + +import java.io.IOException; + +public class TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat extends TestHoodieJavaClientOnCopyOnWriteStorage { + @Override + protected void initMetaClient() throws IOException { + if (basePath == null) { + initPath(); + } + TypedProperties properties = new TypedProperties(); + properties.put(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "false"); + properties.put(HoodieTableConfig.TABLE_FORMAT.key(), "test-format"); + properties.put(HoodieMetadataConfig.ENABLE.key(), "false"); + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, properties); + } + + @AfterAll + public static void tearDownAll() throws IOException { + TestTableFormat.tearDown(); + FileSystem.closeAll(); + } +} diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java index 5595a1aa77bca..e4787f97a43cd 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java @@ -280,7 +280,7 @@ public void syncTableMetadata(HoodieWriteConfig writeConfig) { } protected HoodieTableMetadata metadata(HoodieWriteConfig clientConfig, HoodieEngineContext engineContext) { - return HoodieTableMetadata.create(engineContext, metaClient.getStorage(), clientConfig.getMetadataConfig(), clientConfig.getBasePath()); + return metaClient.getTableFormat().getMetadataFactory().create(engineContext, metaClient.getStorage(), clientConfig.getMetadataConfig(), clientConfig.getBasePath()); } @Override @@ -439,8 +439,7 @@ private void runFullValidation(HoodieWriteConfig writeConfig, // Metadata table has a fixed number of partitions // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. - List metadataTablePartitions = FSUtils.getAllPartitionPaths( - engineContext, storage, HoodieTableMetadata.getMetadataTableBasePath(basePath), false); + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient, false); // Metadata table should automatically compact and clean // versions are +1 as autoClean / compaction happens end of commits diff --git a/hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat b/hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat new file mode 100644 index 0000000000000..361558b871c18 --- /dev/null +++ b/hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat @@ -0,0 +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. +########################################################################## +org.apache.hudi.tableformat.TestTableFormat diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java index 776ec1dbf1ec2..d300ea683a900 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -45,8 +45,7 @@ public SparkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context @Override protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - List partitionPaths = FSUtils.getAllPartitionPaths( - context, table.getStorage(), config.getMetadataConfig(), table.getMetaClient().getBasePath()); + List partitionPaths = FSUtils.getAllPartitionPaths(context, table.getMetaClient(), config.getMetadataConfig()); if (partitionPaths == null || partitionPaths.isEmpty()) { return Collections.emptyMap(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index 0ac5d652eb2ec..375f3eb592fff 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -139,9 +139,9 @@ public void testSavepointAndRollback(Boolean testFailedRestore, Boolean failedRe client.commit(newCommitTime, jsc.parallelize(statusList), Option.empty(), COMMIT_ACTION, Collections.emptyMap(), Option.empty()); HoodieWriteConfig config = getConfig(); - List partitionPaths = - FSUtils.getAllPartitionPaths(context, storage, config.getMetadataConfig(), cfg.getBasePath()); metaClient = HoodieTableMetaClient.reload(metaClient); + List partitionPaths = + FSUtils.getAllPartitionPaths(context, metaClient, config.getMetadataConfig()); HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view1 = table.getBaseFileOnlyView(); @@ -322,9 +322,9 @@ public void testSavepointAndRollbackWithKeepLatestFileVersionPolicy() throws Exc assertNoWriteErrors(statusList); HoodieWriteConfig config = getConfig(); - List partitionPaths = - FSUtils.getAllPartitionPaths(context, storage, config.getMetadataConfig(), cfg.getBasePath()); metaClient = HoodieTableMetaClient.reload(metaClient); + List partitionPaths = + FSUtils.getAllPartitionPaths(context, metaClient, config.getMetadataConfig()); HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view1 = table.getBaseFileOnlyView(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java index 651ca99d4c1af..9ed49a52960ee 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java @@ -216,7 +216,8 @@ private interface FileIdAndNameGenerator { } private void assertFileGroupCorrectness(String instantTime, String partitionPath, String filePath, String fileId, int expectedSize) { - HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(context, metaClient.getStorage(), writeConfig.getMetadataConfig(), metaClient.getBasePath().toString()); + HoodieTableMetadata tableMetadata = metaClient.getTableFormat().getMetadataFactory().create( + context, metaClient.getStorage(), writeConfig.getMetadataConfig(), metaClient.getBasePath().toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(tableMetadata, metaClient, metaClient.reloadActiveTimeline()); List fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList()); Assertions.assertEquals(expectedSize, fileGroups.size()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java index 5278c0f7f4188..49af02e058083 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java @@ -574,7 +574,7 @@ public HoodieBackedTableMetadataWriter metadataWriter(HoodieWriteConfig clientCo public HoodieTableMetadata metadata(HoodieWriteConfig clientConfig, HoodieEngineContext hoodieEngineContext) { - return HoodieTableMetadata.create( + return metaClient.getTableFormat().getMetadataFactory().create( hoodieEngineContext, storage, clientConfig.getMetadataConfig(), clientConfig.getBasePath()); } @@ -651,8 +651,7 @@ private void runFullValidation(HoodieMetadataConfig metadataConfig, // Metadata table has a fixed number of partitions // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. - List metadataTablePartitions = FSUtils.getAllPartitionPaths( - engineContext, storage, HoodieTableMetadata.getMetadataTableBasePath(basePath), false); + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient, false); // Metadata table should automatically compact and clean // versions are +1 as autoClean / compaction happens end of commits diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index 9d6aa0316c558..9919e0bde3cb0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -48,6 +48,7 @@ import org.apache.hudi.internal.schema.Types; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.TableMetadataFactory; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; @@ -436,7 +437,7 @@ private List listPartitionPathFiles(List partiti private void doRefresh() { HoodieTimer timer = HoodieTimer.start(); - resetTableMetadata(createMetadataTable(engineContext, metaClient.getStorage(), metadataConfig, basePath)); + resetTableMetadata(createMetadataTable(engineContext, metaClient.getStorage(), metaClient.getTableFormat().getMetadataFactory(), metadataConfig, basePath)); // Make sure we reload active timeline metaClient.reloadActiveTimeline(); @@ -535,10 +536,11 @@ private void resetTableMetadata(HoodieTableMetadata newTableMetadata) { private static HoodieTableMetadata createMetadataTable( HoodieEngineContext engineContext, HoodieStorage storage, + TableMetadataFactory metadataFactory, HoodieMetadataConfig metadataConfig, StoragePath basePath ) { - HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create( + HoodieTableMetadata newTableMetadata = metadataFactory.create( engineContext, storage, metadataConfig, basePath.toString(), true); return newTableMetadata; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java new file mode 100644 index 0000000000000..ecc13612a71eb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common; + +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.metadata.NativeTableMetadataFactory; +import org.apache.hudi.metadata.TableMetadataFactory; + +public class NativeTableFormat implements TableFormat { + public static final String TABLE_FORMAT = "native"; + private final TimelineLayoutVersion timelineLayoutVersion; + + public NativeTableFormat(TimelineLayoutVersion timelineLayoutVersion) { + this.timelineLayoutVersion = timelineLayoutVersion; + } + + @Override + public String getName() { + return NativeTableFormat.TABLE_FORMAT; + } + + public TimelineFactory getTimelineFactory() { + return TimelineLayout.fromVersion(timelineLayoutVersion).getTimelineFactory(); + } + + @Override + public TableMetadataFactory getMetadataFactory() { + return NativeTableMetadataFactory.getInstance(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java new file mode 100644 index 0000000000000..17fcf8233d310 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.metadata.TableMetadataFactory; + +import java.io.Serializable; +import java.util.List; +import java.util.Properties; + +/** + * External Table Format needs to implement this interface + */ +public interface TableFormat extends Serializable { + + /** + * Returns the name of the table format. + */ + String getName(); + + /** + * Initializes the table format implementation with the properties supplied from {@link org.apache.hudi.common.table.HoodieTableConfig} + */ + default void init(Properties properties) { + } + + /** + * Called just after marking the write action as complete in hoodie timeline. Implementation expected to save additional state needed in + * extraMetadata. + * + * @param commitMetadata HoodieCommitMetadata for commit or clustering action. + * @param completedInstant completed instant in hoodie timeline + * @param engineContext engine context used for execution - local,spark or flink etc. + * @param metaClient metaClient from HoodieTable. + * @param viewManager viewManager from HoodieTable. + */ + + default void commit( + HoodieCommitMetadata commitMetadata, + HoodieInstant completedInstant, + HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + } + + /** + * Called after marking the clean action as complete in hoodie timeline. + * + * @param cleanMetadata HoodieCleanMetadata for clean action. + * @param completedInstant completed instant in hoodie timeline + * @param engineContext engine context used for execution - local,spark or flink etc. + * @param metaClient metaClient from HoodieTable. + * @param viewManager viewManager from HoodieTable. + */ + default void clean( + HoodieCleanMetadata cleanMetadata, + HoodieInstant completedInstant, + HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + } + + /** + * Called after archiving the instants in hoodie timeline. + * + * @param archivedInstants List of instants archived in hoodie timeline + * @param engineContext engine context used for execution - local,spark or flink etc. + * @param metaClient metaClient from HoodieTable. + * @param viewManager viewManager from HoodieTable. + */ + default void archive( + List archivedInstants, + HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + } + + /** + * Called before rolling back the instant in hoodie timeline. + * + * @param completedInstant completed rollback instant in hoodie timeline + * @param engineContext engine context used for execution - local,spark or flink etc. + * @param metaClient metaClient from HoodieTable. + * @param viewManager viewManager from HoodieTable. + */ + default void rollback( + HoodieInstant completedInstant, + HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + } + + /** + * Called after marking a rollback action as complete in hoodie timeline. + * + * @param rollbackInstant The completed rollback instant in hoodie timeline. + * @param engineContext engine context used for execution - local,spark or flink etc. + * @param metaClient metaClient from HoodieTable. + * @param viewManager viewManager from HoodieTable. + */ + default void completedRollback( + HoodieInstant rollbackInstant, + HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + } + + /** + * Called after marking a complete write action as "savepoint" in the hoodie timeline. + * + * @param savepointInstant The completed instant to be marked as savepoint. + * @param engineContext engine context used for execution - local,spark or flink etc. + * @param metaClient metaClient from HoodieTable. + * @param viewManager viewManager from HoodieTable. + */ + default void savepoint( + HoodieInstant savepointInstant, + HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + } + + /** + * Return the timeline factory for table format. + */ + TimelineFactory getTimelineFactory(); + + /** + * Return the table metadata factory for table format. + */ + TableMetadataFactory getMetadataFactory(); +} 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 451da6c179ba1..a84d518bfcccc 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 @@ -238,13 +238,13 @@ public static void processFiles(HoodieStorage storage, String basePathStr, Funct } public static List getAllPartitionPaths(HoodieEngineContext engineContext, - HoodieStorage storage, - String basePathStr, + HoodieTableMetaClient metaClient, boolean useFileListingFromMetadata) { HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() .enable(useFileListingFromMetadata) .build(); - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, storage, metadataConfig, basePathStr)) { + try (HoodieTableMetadata tableMetadata = metaClient.getTableFormat().getMetadataFactory() + .create(engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString())) { return tableMetadata.getAllPartitionPaths(); } catch (Exception e) { throw new HoodieException("Error fetching partition paths from metadata table", e); @@ -252,38 +252,22 @@ public static List getAllPartitionPaths(HoodieEngineContext engineContex } public static List getAllPartitionPaths(HoodieEngineContext engineContext, - HoodieStorage storage, - HoodieMetadataConfig metadataConfig, - String basePathStr) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, storage, metadataConfig, - basePathStr)) { + HoodieTableMetaClient metaClient, + HoodieMetadataConfig metadataConfig) { + try (HoodieTableMetadata tableMetadata = metaClient.getTableFormat().getMetadataFactory() + .create(engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString())) { return tableMetadata.getAllPartitionPaths(); } catch (Exception e) { throw new HoodieException("Error fetching partition paths from metadata table", e); } } - public static List getAllPartitionPaths(HoodieEngineContext engineContext, - HoodieStorage storage, - StoragePath basePath, - boolean useFileListingFromMetadata) { - return getAllPartitionPaths(engineContext, storage, basePath.toString(), useFileListingFromMetadata); - } - - public static List getAllPartitionPaths(HoodieEngineContext engineContext, - HoodieStorage storage, - HoodieMetadataConfig metadataConfig, - StoragePath basePath) { - return getAllPartitionPaths(engineContext, storage, metadataConfig, basePath.toString()); - } - public static Map> getFilesInPartitions(HoodieEngineContext engineContext, - HoodieStorage storage, + HoodieTableMetaClient metaClient, HoodieMetadataConfig metadataConfig, - String basePathStr, String[] partitionPaths) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, storage, metadataConfig, - basePathStr)) { + try (HoodieTableMetadata tableMetadata = metaClient.getTableFormat().getMetadataFactory() + .create(engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString())) { return tableMetadata.getAllFilesInPartitions(Arrays.asList(partitionPaths)); } catch (Exception ex) { throw new HoodieException("Error get files in partitions: " + String.join(",", partitionPaths), ex); 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 03db7a56739f8..4da5603298062 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 @@ -18,6 +18,8 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.TableFormat; +import org.apache.hudi.common.NativeTableFormat; import org.apache.hudi.common.bootstrap.index.hfile.HFileBootstrapIndex; import org.apache.hudi.common.config.ConfigClassProperty; import org.apache.hudi.common.config.ConfigGroups; @@ -72,6 +74,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.ServiceLoader; import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Function; @@ -198,7 +201,12 @@ public class HoodieTableConfig extends HoodieConfig { .key("hoodie.timeline.layout.version") .noDefaultValue() .withDocumentation("Version of timeline used, by the table."); - + + public static final ConfigProperty TABLE_FORMAT = ConfigProperty + .key("hoodie.table.format") + .defaultValue(NativeTableFormat.TABLE_FORMAT) + .withDocumentation("Table format name used when writing to the table."); + public static final ConfigProperty RECORD_MERGE_MODE = ConfigProperty .key("hoodie.record.merge.mode") .defaultValue((RecordMergeMode) null, @@ -720,6 +728,20 @@ public Option getTimelineLayoutVersion() { : Option.empty(); } + public TableFormat getTableFormat(TimelineLayoutVersion layoutVersion) { + String tableFormat = getStringOrDefault(TABLE_FORMAT); + if (!tableFormat.equals(NativeTableFormat.TABLE_FORMAT)) { + ServiceLoader loader = ServiceLoader.load(TableFormat.class); + for (TableFormat tableFormatImpl : loader) { + if (getString(TABLE_FORMAT).equals(tableFormatImpl.getName())) { + tableFormatImpl.init(props); + return tableFormatImpl; + } + } + } + return new NativeTableFormat(layoutVersion); + } + /** * @return the hoodie.table.version from hoodie.properties file. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index a6c868768f5e3..8a7f918272ca4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.NativeTableFormat; +import org.apache.hudi.common.TableFormat; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetaserverConfig; @@ -85,6 +87,7 @@ import static org.apache.hudi.common.table.HoodieTableConfig.PAYLOAD_CLASS_NAME; import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGE_MODE; import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGE_STRATEGY_ID; +import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_FORMAT; import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_PATH; import static org.apache.hudi.common.table.HoodieTableConfig.VERSION; import static org.apache.hudi.common.util.ConfigUtils.containsConfigProperty; @@ -165,6 +168,7 @@ public class HoodieTableMetaClient implements Serializable { protected HoodieMetaserverConfig metaserverConfig; private HoodieTimeGeneratorConfig timeGeneratorConfig; private Option indexMetadataOpt = Option.empty(); + private TableFormat tableFormat; /** * Instantiate HoodieTableMetaClient. @@ -195,6 +199,7 @@ protected HoodieTableMetaClient(HoodieStorage storage, String basePath, boolean throw new TableNotFoundException("Table does not exist"); } this.timelineLayoutVersion = layoutVersion.orElseGet(tableConfigVersion::get); + this.tableFormat = tableConfig.getTableFormat(timelineLayoutVersion); this.timelineLayout = TimelineLayout.fromVersion(timelineLayoutVersion); this.timelinePath = timelineLayout.getTimelinePathProvider().getTimelinePath(tableConfig, this.basePath); this.timelineHistoryPath = timelineLayout.getTimelinePathProvider().getTimelineHistoryPath(tableConfig, this.basePath); @@ -357,6 +362,10 @@ public StoragePath getTimelinePath() { return timelinePath; } + public TableFormat getTableFormat() { + return tableFormat; + } + /** * @return schema folder path */ @@ -501,7 +510,7 @@ public StorageConfiguration getStorageConf() { */ public synchronized HoodieActiveTimeline getActiveTimeline() { if (activeTimeline == null) { - activeTimeline = timelineLayout.getTimelineFactory().createActiveTimeline(this); + activeTimeline = tableFormat.getTimelineFactory().createActiveTimeline(this); } return activeTimeline; } @@ -514,13 +523,22 @@ public synchronized void reload() { reloadTableConfig(); } + /** + * Get the active instants as a timeline in native format. + * + * @return Active instants timeline + */ + public synchronized HoodieActiveTimeline getActiveTimelineForNativeFormat() { + return new NativeTableFormat(activeTimeline.getTimelineLayoutVersion()).getTimelineFactory().createActiveTimeline(this); + } + /** * Reload ActiveTimeline and cache. * * @return Active instants timeline */ public synchronized HoodieActiveTimeline reloadActiveTimeline() { - activeTimeline = timelineLayout.getTimelineFactory().createActiveTimeline(this); + activeTimeline = tableFormat.getTimelineFactory().createActiveTimeline(this); return activeTimeline; } @@ -538,6 +556,7 @@ public synchronized void reloadTableConfig() { private void reloadTimelineLayoutAndPath() { this.timelineLayoutVersion = tableConfig.getTimelineLayoutVersion().get(); this.timelineLayout = TimelineLayout.fromVersion(timelineLayoutVersion); + this.tableFormat = tableConfig.getTableFormat(timelineLayoutVersion); this.timelinePath = timelineLayout.getTimelinePathProvider().getTimelinePath(tableConfig, basePath); this.timelineHistoryPath = timelineLayout.getTimelinePathProvider().getTimelineHistoryPath(tableConfig, basePath); } @@ -619,8 +638,8 @@ public HoodieArchivedTimeline getArchivedTimeline(String startTs, boolean useCac private HoodieArchivedTimeline instantiateArchivedTimeline(String startTs) { return StringUtils.isNullOrEmpty(startTs) - ? timelineLayout.getTimelineFactory().createArchivedTimeline(this) - : timelineLayout.getTimelineFactory().createArchivedTimeline(this, startTs); + ? tableFormat.getTimelineFactory().createArchivedTimeline(this) + : tableFormat.getTimelineFactory().createArchivedTimeline(this, startTs); } public static void createTableLayoutOnStorage(StorageConfiguration storageConf, @@ -1035,6 +1054,7 @@ public static class TableBuilder { private Boolean multipleBaseFileFormatsEnabled; private String indexDefinitionPath; + private String tableFormat; /** * Persist the configs that is written at the first time, and should not be changed. @@ -1234,6 +1254,11 @@ public TableBuilder setIndexDefinitionPath(String indexDefinitionPath) { return this; } + public TableBuilder setTableFormat(String tableFormat) { + this.tableFormat = tableFormat; + return this; + } + public TableBuilder set(Map props) { for (ConfigProperty configProperty : HoodieTableConfig.PERSISTED_CONFIG_LIST) { if (containsConfigProperty(props, configProperty)) { @@ -1515,6 +1540,9 @@ public Properties build() { if (null != indexDefinitionPath) { tableConfig.setValue(HoodieTableConfig.RELATIVE_INDEX_DEFINITION_PATH, indexDefinitionPath); } + if (null != tableFormat) { + tableConfig.setValue(TABLE_FORMAT, tableFormat); + } return tableConfig.getProps(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java index 9f22586a560d3..b144eed94e74e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java @@ -160,7 +160,7 @@ public static Builder builder() { * @return An incremental query context including the instant time range info. */ public QueryContext analyze() { - try (CompletionTimeQueryView completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(this.metaClient)) { + try (CompletionTimeQueryView completionTimeQueryView = metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(this.metaClient)) { if (completionTimeQueryView.isEmptyTable()) { // no dataset committed in the table return QueryContext.EMPTY; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java index d09a4f9bed281..0eaedda3492d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java @@ -18,15 +18,19 @@ package org.apache.hudi.common.table.timeline; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.io.InputStream; import java.io.Serializable; import java.security.MessageDigest; @@ -115,6 +119,14 @@ protected void appendInstants(List newInstants) { clearState(); } + protected List getInstantsFromFileSystem(HoodieTableMetaClient metaClient, Set includedExtensions, boolean applyLayoutFilters) { + try { + return metaClient.scanHoodieInstantsFromFileSystem(metaClient.getTimelinePath(), includedExtensions, applyLayoutFilters); + } catch (IOException e) { + throw new HoodieIOException("Failed to scan metadata", e); + } + } + @Override public HoodieTimeline filterInflights() { return factory.createDefaultTimeline(getInstantsAsStream().filter(HoodieInstant::isInflight), getInstantReader()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 619cc402dbb38..7c48720ba978f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -69,7 +69,7 @@ public interface HoodieActiveTimeline extends HoodieTimeline { * @param instant Instant to be saved. * @param metadata metadata to write into the instant file */ - void saveAsComplete(HoodieInstant instant, Option metadata); + HoodieInstant saveAsComplete(HoodieInstant instant, Option metadata); /** * Save Completed instant in active timeline. @@ -77,7 +77,20 @@ public interface HoodieActiveTimeline extends HoodieTimeline { * @param instant Instant to be saved. * @param metadata metadata to write into the instant file */ - void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata); + HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata); + + /** + * Save Completed instant in active timeline with table format completion actions. + * + * @param shouldLock Lock before writing to timeline. + * @param instant Instant to be saved. + * @param metadata metadata to write into the instant file + * @param tableFormatCompletionAction functional interface to perform table format specific completion actions. + * @return The completed hoodie instant + * @param + */ + HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, TableFormatCompletionAction tableFormatCompletionAction); + /** * Save Completed instant in active timeline with an optional completion time. For version 8 tables, completion times are generated just before wrapping up the commit and serialized as part of @@ -88,7 +101,7 @@ public interface HoodieActiveTimeline extends HoodieTimeline { * @param completionTimeOpt an optinal instance of completion time. * @param */ - void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, Option completionTimeOpt); + HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, Option completionTimeOpt); /** * Delete Compaction requested instant file from timeline. @@ -227,6 +240,8 @@ public interface HoodieActiveTimeline extends HoodieTimeline { */ HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, Option metadata); + HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, Option metadata, TableFormatCompletionAction tableFormatCompletionAction); + /** * Transition Clean State from requested to inflight. * @@ -245,6 +260,8 @@ public interface HoodieActiveTimeline extends HoodieTimeline { */ HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieRollbackMetadata metadata); + HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieRollbackMetadata metadata, TableFormatCompletionAction tableFormatCompletionAction); + /** * Transition Rollback State from requested to inflight. * @@ -289,6 +306,8 @@ public interface HoodieActiveTimeline extends HoodieTimeline { */ HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata); + HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata, TableFormatCompletionAction tableFormatCompletionAction); + /** * Transition cluster inflight to replace committed. * @@ -299,6 +318,8 @@ public interface HoodieActiveTimeline extends HoodieTimeline { */ HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata); + HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata, TableFormatCompletionAction tableFormatCompletionAction); + /** * Save Restore requested instant with metadata. * @param commitType Instant type. @@ -416,4 +437,4 @@ public interface HoodieActiveTimeline extends HoodieTimeline { * @return */ Set getValidExtensions(); -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index 6224b5a921354..53a1f43c45407 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -32,6 +32,7 @@ import java.time.temporal.ChronoField; import java.time.temporal.TemporalAccessor; import java.util.Date; +import java.util.TimeZone; import java.util.concurrent.atomic.AtomicReference; import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java index 0461ec2c43e96..ce39c7b6ad751 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java @@ -54,13 +54,14 @@ public long generateTime(boolean skipLocking) { } @Override - public void consumeTime(boolean skipLocking, Consumer func) { + public long consumeTime(boolean skipLocking, Consumer func) { try { if (!skipLocking) { lock(); } long currentTimeMillis = generateTime(true); func.accept(currentTimeMillis); + return currentTimeMillis; } finally { if (!skipLocking) { unlock(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TableFormatCompletionAction.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TableFormatCompletionAction.java new file mode 100644 index 0000000000000..b2ad0eb5a2354 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TableFormatCompletionAction.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + + +/** + * Functional Interface for executing table format actions. + */ +@FunctionalInterface +public interface TableFormatCompletionAction { + /** + * Execute the table format action with the given completed instant. + */ + void execute(HoodieInstant completedInstant); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java index f797eb33c501b..c64ce043c2548 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java @@ -47,6 +47,8 @@ public interface TimeGenerator { * * @param skipLocking If this is triggered by another parent transaction, locking can be skipped. * @param func A consumer that takes a monotonically increasing timestamp. + * + * @return Return TrueTime as milliseconds in consumer function. */ - void consumeTime(boolean skipLocking, Consumer func); + long consumeTime(boolean skipLocking, Consumer func); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java index 5fc87292c75ea..8ad6239d4ef9d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java @@ -582,7 +582,7 @@ public static HoodieInstant getInflightInstant(final HoodieInstant instant, fina } else if (instant.getAction().equals(DELTA_COMMIT_ACTION)) { // Deltacommit is used by both ingestion and logcompaction. // So, distinguish both of them check for the inflight file being present. - HoodieActiveTimeline rawActiveTimeline = metaClient.getTimelineLayout().getTimelineFactory().createActiveTimeline(metaClient, false); + HoodieActiveTimeline rawActiveTimeline = metaClient.getTableFormat().getTimelineFactory().createActiveTimeline(metaClient, false); Option logCompactionInstant = Option.fromJavaOptional(rawActiveTimeline.getInstantsAsStream() .filter(hoodieInstant -> hoodieInstant.requestedTime().equals(instant.requestedTime()) && LOG_COMPACTION_ACTION.equals(hoodieInstant.getAction())).findFirst()); @@ -603,7 +603,7 @@ public enum HollowCommitHandling { */ public static HoodieTimeline concatTimeline(HoodieTimeline timeline1, HoodieTimeline timeline2, HoodieTableMetaClient metaClient) { - return metaClient.getTimelineLayout().getTimelineFactory().createDefaultTimeline( + return metaClient.getTableFormat().getTimelineFactory().createDefaultTimeline( Stream.concat(timeline1.getInstantsAsStream(), timeline2.getInstantsAsStream()).sorted(), metaClient.getActiveTimeline()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java index 006daf88a538c..662c27ee9c1f2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java @@ -37,6 +37,12 @@ public class InstantDTO { @JsonProperty("state") String state; + @JsonProperty("requestedTime") + String requestedTime; + + @JsonProperty("completionTime") + String completionTime; + public static InstantDTO fromInstant(HoodieInstant instant) { if (null == instant) { return null; @@ -45,6 +51,8 @@ public static InstantDTO fromInstant(HoodieInstant instant) { InstantDTO dto = new InstantDTO(); dto.action = instant.getAction(); dto.timestamp = instant.requestedTime(); + dto.requestedTime = instant.requestedTime(); + dto.completionTime = instant.getCompletionTime(); dto.state = instant.getState().toString(); return dto; } @@ -54,6 +62,6 @@ public static HoodieInstant toInstant(InstantDTO dto, InstantGenerator factory) return null; } - return factory.createNewInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.timestamp); + return factory.createNewInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.requestedTime, dto.completionTime); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java index f4a7deb4c0343..5822832af616f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/TimelineDTO.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.timeline.dto; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineFactory; @@ -44,11 +45,21 @@ public static TimelineDTO fromTimeline(HoodieTimeline timeline) { return dto; } + public static TimelineDTO fromInstants(List instants) { + TimelineDTO dto = new TimelineDTO(); + dto.instants = instants.stream().map(InstantDTO::fromInstant).collect(Collectors.toList()); + return dto; + } + public static HoodieTimeline toTimeline(TimelineDTO dto, HoodieTableMetaClient metaClient) { InstantGenerator instantGenerator = metaClient.getInstantGenerator(); - TimelineFactory factory = metaClient.getTimelineLayout().getTimelineFactory(); + TimelineFactory factory = metaClient.getTableFormat().getTimelineFactory(); // TODO: For Now, we will assume, only active-timeline will be transferred. return factory.createDefaultTimeline(dto.instants.stream().map(d -> InstantDTO.toInstant(d, instantGenerator)), metaClient.getActiveTimeline()); } + + public static HoodieTimeline toTimeline(TimelineDTO dto, TimelineFactory factory, HoodieTimeline timeline, InstantGenerator instantGenerator) { + return factory.createDefaultTimeline(dto.instants.stream().map(d -> InstantDTO.toInstant(d, instantGenerator)), timeline.getInstantReader()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java index a3537f2a94637..cd5d8d7664be4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java @@ -35,6 +35,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstantReader; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.TableFormatCompletionAction; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; @@ -84,12 +85,7 @@ protected ActiveTimelineV1(HoodieTableMetaClient metaClient, Set include boolean applyLayoutFilters) { // Filter all the filter in the metapath and include only the extensions passed and // convert them into HoodieInstant - try { - this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(metaClient.getTimelinePath(), - includedExtensions, applyLayoutFilters)); - } catch (IOException e) { - throw new HoodieIOException("Failed to scan metadata", e); - } + this.setInstants(getInstantsFromFileSystem(metaClient, includedExtensions, applyLayoutFilters)); this.metaClient = metaClient; // multiple casts will make this lambda serializable - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 @@ -151,22 +147,31 @@ public HoodieInstant createRequestedCommitWithReplaceMetadata(String instantTime } @Override - public void saveAsComplete(HoodieInstant instant, Option metadata) { - LOG.info("Marking instant complete {}", instant); + public HoodieInstant saveAsComplete(HoodieInstant instant, Option metadata) { + LOG.info("Marking instant complete " + instant); ValidationUtils.checkArgument(instant.isInflight(), "Could not mark an already completed instant as complete again " + instant); - transitionState(instant, instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, instant.getAction(), instant.requestedTime()), metadata); + HoodieInstant completedInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, instant.getAction(), instant.requestedTime()); + transitionState(instant, completedInstant, metadata); LOG.info("Completed {}", instant); + return completedInstant; } @Override - public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata) { - saveAsComplete(instant, metadata); + public HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata) { + return saveAsComplete(instant, metadata); } @Override - public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, Option completionTimeOpt) { - saveAsComplete(instant, metadata); + public HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, Option completionTimeOpt) { + return saveAsComplete(instant, metadata); + } + + @Override + public HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = saveAsComplete(shouldLock, instant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; } @Override @@ -401,6 +406,14 @@ public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, Hoodi return commitInstant; } + @Override + public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, Option metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionCleanInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; + } + @Override public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant) { ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); @@ -420,6 +433,14 @@ public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, Ho return commitInstant; } + @Override + public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieRollbackMetadata metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionRollbackInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; + } + @Override public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant) { ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); @@ -466,12 +487,28 @@ public HoodieInstant transitionReplaceInflightToComplete( return commitInstant; } + @Override + public HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionReplaceInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; + } + @Override public HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata) { // In 0.x, no separate clustering action, reuse replace action. return transitionReplaceInflightToComplete(shouldLock, inflightInstant, metadata); } + @Override + public HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionClusterInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; + } + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata) { transitionState(fromInstant, toInstant, metadata, false); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java index 9da142c2b3d13..02fa180722d15 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.TableFormatCompletionAction; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; import org.apache.hudi.common.table.timeline.TimelineUtils; @@ -89,12 +90,7 @@ private ActiveTimelineV2(HoodieTableMetaClient metaClient, Set includedE boolean applyLayoutFilters) { // Filter all the filter in the metapath and include only the extensions passed and // convert them into HoodieInstant - try { - this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(metaClient.getTimelinePath(), - includedExtensions, applyLayoutFilters)); - } catch (IOException e) { - throw new HoodieIOException("Failed to scan metadata", e); - } + this.setInstants(getInstantsFromFileSystem(metaClient, includedExtensions, applyLayoutFilters)); this.metaClient = metaClient; // multiple casts will make this lambda serializable - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 @@ -156,23 +152,31 @@ public HoodieInstant createRequestedCommitWithReplaceMetadata(String instantTime } @Override - public void saveAsComplete(HoodieInstant instant, Option metadata) { - saveAsComplete(true, instant, metadata); + public HoodieInstant saveAsComplete(HoodieInstant instant, Option metadata) { + return saveAsComplete(true, instant, metadata); } @Override - public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata) { - saveAsComplete(shouldLock, instant, metadata, Option.empty()); + public HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata) { + return saveAsComplete(shouldLock, instant, metadata, Option.empty()); } @Override - public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, Option completionTimeOpt) { + public HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, Option completionTimeOpt) { LOG.info("Marking instant complete {}", instant); ValidationUtils.checkArgument(instant.isInflight(), "Could not mark an already completed instant as complete again " + instant); HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, instant.getAction(), instant.requestedTime()); - transitionStateToComplete(shouldLock, instant, commitInstant, metadata, completionTimeOpt); + HoodieInstant completedInstant = transitionStateToComplete(shouldLock, instant, commitInstant, metadata, completionTimeOpt); LOG.info("Completed " + instant); + return completedInstant; + } + + @Override + public HoodieInstant saveAsComplete(boolean shouldLock, HoodieInstant instant, Option metadata, TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = saveAsComplete(shouldLock, instant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; } @Override @@ -413,8 +417,15 @@ public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, Hoodi ValidationUtils.checkArgument(inflightInstant.isInflight()); HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, CLEAN_ACTION, inflightInstant.requestedTime()); // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, metadata); - return commitInstant; + return transitionStateToComplete(shouldLock, inflightInstant, commitInstant, metadata); + } + + @Override + public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, Option metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionCleanInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; } @Override @@ -432,8 +443,15 @@ public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, Ho ValidationUtils.checkArgument(inflightInstant.isInflight()); HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, ROLLBACK_ACTION, inflightInstant.requestedTime()); // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, Option.of(metadata)); - return commitInstant; + return transitionStateToComplete(shouldLock, inflightInstant, commitInstant, Option.of(metadata)); + } + + @Override + public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieRollbackMetadata metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionRollbackInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; } @Override @@ -482,8 +500,15 @@ public HoodieInstant transitionReplaceInflightToComplete( ValidationUtils.checkArgument(inflightInstant.isInflight()); HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.requestedTime()); // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, Option.of(metadata)); - return commitInstant; + return transitionStateToComplete(shouldLock, inflightInstant, commitInstant, Option.of(metadata)); + } + + @Override + public HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionReplaceInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; } @Override @@ -493,20 +518,27 @@ public HoodieInstant transitionClusterInflightToComplete( ValidationUtils.checkArgument(inflightInstant.isInflight()); HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.requestedTime()); // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, Option.of(metadata), Option.empty()); - return commitInstant; + return transitionStateToComplete(shouldLock, inflightInstant, commitInstant, Option.of(metadata), Option.empty()); + } + + @Override + public HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, HoodieReplaceCommitMetadata metadata, + TableFormatCompletionAction tableFormatCompletionAction) { + HoodieInstant completedInstant = transitionClusterInflightToComplete(shouldLock, inflightInstant, metadata); + tableFormatCompletionAction.execute(completedInstant); + return completedInstant; } private void transitionPendingState(HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata) { transitionPendingState(fromInstant, toInstant, metadata, false); } - protected void transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata) { - transitionStateToComplete(shouldLock, fromInstant, toInstant, metadata, Option.empty()); + protected HoodieInstant transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata) { + return transitionStateToComplete(shouldLock, fromInstant, toInstant, metadata, Option.empty()); } - protected void transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata, - Option completionTimeOpt) { + protected HoodieInstant transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata, + Option completionTimeOpt) { ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime()), String.format("%s and %s are not consistent when transition state.", fromInstant, toInstant)); String fromInstantFileName = instantFileNameGenerator.getFileName(fromInstant); try { @@ -524,12 +556,14 @@ protected void transitionStateToComplete(boolean shouldLock, HoodieInstant f throw new HoodieIOException( "Could not rename " + fromInstantPath + " to " + toInstantPath); } + return instantWithCompletionTime; } else { // Ensures old state exists in timeline ValidationUtils.checkArgument( metaClient.getStorage().exists(getInstantFileNamePath(fromInstantFileName)), "File " + getInstantFileNamePath(fromInstantFileName) + " does not exist!"); - createCompleteFileInMetaPath(shouldLock, toInstant, metadata); + String completionTime = HoodieInstantTimeGenerator.formatDateUTC(new Date(createCompleteFileInMetaPath(shouldLock, toInstant, metadata))); + return new HoodieInstant(toInstant.getState(), toInstant.getAction(), toInstant.requestedTime(), completionTime, instantComparator.requestedTimeOrderedComparator()); } } catch (IOException e) { throw new HoodieIOException("Could not complete " + fromInstant, e); @@ -708,7 +742,7 @@ public void createFileInMetaPath(String filename, Option metadata, boolea } } - protected void createCompleteFileInMetaPath(boolean shouldLock, HoodieInstant instant, Option metadata) { + protected long createCompleteFileInMetaPath(boolean shouldLock, HoodieInstant instant, Option metadata) { Option writerOption = getHoodieInstantWriterOption(this, metadata); TimeGenerator timeGenerator = TimeGenerators .getTimeGenerator(metaClient.getTimeGeneratorConfig(), metaClient.getStorageConf()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java index 8005614c6fdc4..1f67469eae977 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java @@ -288,7 +288,7 @@ private void loadCompletionTimeIncrementally(String startTime) { // This operation is resource costly. synchronized (this) { if (InstantComparison.compareTimestamps(startTime, LESSER_THAN, this.cursorInstant)) { - metaClient.getTimelineLayout().getTimelineFactory().createArchivedTimelineLoader().loadInstants(metaClient, + metaClient.getTableFormat().getTimelineFactory().createArchivedTimelineLoader().loadInstants(metaClient, new HoodieArchivedTimeline.ClosedOpenTimeRangeFilter(startTime, this.cursorInstant), HoodieArchivedTimeline.LoadMode.TIME, r -> true, @@ -310,7 +310,7 @@ private void load() { .filterCompletedInstants().getInstantsAsStream() .forEach(instant -> setCompletionTime(instant.requestedTime(), instant.getCompletionTime())); // then load the archived instants. - metaClient.getTimelineLayout().getTimelineFactory().createArchivedTimelineLoader().loadInstants(metaClient, + metaClient.getTableFormat().getTimelineFactory().createArchivedTimelineLoader().loadInstants(metaClient, new HoodieArchivedTimeline.StartTsFilter(this.cursorInstant), HoodieArchivedTimeline.LoadMode.TIME, r -> true, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index b0b39b8154536..5c0128930d628 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -124,7 +124,7 @@ protected AbstractTableFileSystemView(HoodieTableMetadata tableMetadata) { */ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; - this.completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); + this.completionTimeQueryView = metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(metaClient); refreshTimeline(visibleActiveTimeline); resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline); this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); @@ -152,7 +152,7 @@ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { * Refresh the completion time query view. */ protected void refreshCompletionTimeQueryView() { - this.completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); + this.completionTimeQueryView = metaClient.getTableFormat().getTimelineFactory().createCompletionTimeQueryView(metaClient); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java index 0c7495fa294eb..d1baad8495165 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java @@ -206,7 +206,7 @@ public static HoodieTableFileSystemView createInMemoryFileSystemViewWithTimeline HoodieTimeline timeline) { LOG.info("Creating InMemory based view for basePath {}.", metaClient.getBasePath()); HoodieTableMetadata tableMetadata = getTableMetadata(engineContext, metaClient, metadataConfig.isEnabled(), - unused -> HoodieTableMetadata.create(engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString())); + unused -> metaClient.getTableFormat().getMetadataFactory().create(engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString())); if (metaClient.getMetaserverConfig().isMetaserverEnabled()) { return (HoodieTableFileSystemView) ReflectionUtils.loadClass(HOODIE_METASERVER_FILE_SYSTEM_VIEW_CLASS, @@ -236,7 +236,7 @@ public static FileSystemViewManager createViewManagerWithTableMetadata( final FileSystemViewStorageConfig config, final HoodieCommonConfig commonConfig) { return createViewManager(context, metadataConfig, config, commonConfig, - metaClient -> HoodieTableMetadata.create(context, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString(), true)); + metaClient -> metaClient.getTableFormat().getMetadataFactory().create(context, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString(), true)); } public static FileSystemViewManager createViewManager(final HoodieEngineContext context, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index c6212f5d1f39c..a964301bfdae6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -40,6 +40,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TableFormatCompletionAction; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.timeline.versioning.v2.InstantGeneratorV2; import org.apache.hudi.common.util.collection.Pair; @@ -124,11 +125,12 @@ public static Option getRequestedClusteringInstant(String timesta * action type. After HUDI-7905, the new clustering commits are written with clustering action. */ public static void transitionClusteringOrReplaceInflightToComplete(boolean shouldLock, HoodieInstant clusteringInstant, - HoodieReplaceCommitMetadata metadata, HoodieActiveTimeline activeTimeline) { + HoodieReplaceCommitMetadata metadata, HoodieActiveTimeline activeTimeline, + TableFormatCompletionAction tableFormatCompletionAction) { if (clusteringInstant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)) { - activeTimeline.transitionClusterInflightToComplete(shouldLock, clusteringInstant, metadata); + activeTimeline.transitionClusterInflightToComplete(shouldLock, clusteringInstant, metadata, tableFormatCompletionAction); } else { - activeTimeline.transitionReplaceInflightToComplete(shouldLock, clusteringInstant, metadata); + activeTimeline.transitionReplaceInflightToComplete(shouldLock, clusteringInstant, metadata, tableFormatCompletionAction); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index ec3ce260102f8..3c2ac8884be51 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -20,9 +20,7 @@ import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.data.HoodieData; -import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -31,7 +29,6 @@ import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.expression.Expression; import org.apache.hudi.internal.schema.Types; -import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; @@ -121,44 +118,6 @@ static boolean isMetadataTable(StoragePath basePath) { return isMetadataTable(basePath.toString()); } - static HoodieTableMetadata create(HoodieEngineContext engineContext, - HoodieStorage storage, - HoodieMetadataConfig metadataConfig, - String datasetBasePath) { - return create(engineContext, storage, metadataConfig, datasetBasePath, false); - } - - static HoodieTableMetadata create(HoodieEngineContext engineContext, - HoodieStorage storage, - HoodieMetadataConfig metadataConfig, - String datasetBasePath, - boolean reuse) { - if (metadataConfig.isEnabled()) { - HoodieBackedTableMetadata metadata = createHoodieBackedTableMetadata(engineContext, storage, metadataConfig, datasetBasePath, reuse); - // If the MDT is not initialized then we fallback to FSBackedTableMetadata - if (metadata.isMetadataTableInitialized()) { - return metadata; - } - LOG.warn("Falling back to FileSystemBackedTableMetadata as metadata table is not initialized"); - } - return createFSBackedTableMetadata(engineContext, storage, datasetBasePath); - } - - static FileSystemBackedTableMetadata createFSBackedTableMetadata(HoodieEngineContext engineContext, - HoodieStorage storage, - String datasetBasePath) { - return new FileSystemBackedTableMetadata( - engineContext, storage, datasetBasePath); - } - - static HoodieBackedTableMetadata createHoodieBackedTableMetadata(HoodieEngineContext engineContext, - HoodieStorage storage, - HoodieMetadataConfig metadataConfig, - String datasetBasePath, - boolean reuse) { - return new HoodieBackedTableMetadata(engineContext, storage, metadataConfig, datasetBasePath, reuse); - } - /** * Fetch all the files at the given partition path, per the latest snapshot of the metadata. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 13dd2caf2a3bb..09fe6ba315580 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -1419,7 +1419,7 @@ public static HoodieTableFileSystemView getFileSystemViewForMetadataTable(Hoodie // default FileSystemView will not return any file slices even // though we may have initialized them. HoodieTimeline timeline = metaClient.getActiveTimeline(); - TimelineFactory factory = metaClient.getTimelineLayout().getTimelineFactory(); + TimelineFactory factory = metaClient.getTableFormat().getTimelineFactory(); if (timeline.empty()) { final HoodieInstant instant = metaClient.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieInstantTimeGenerator.getCurrentInstantTimeStr()); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/NativeTableMetadataFactory.java b/hudi-common/src/main/java/org/apache/hudi/metadata/NativeTableMetadataFactory.java new file mode 100644 index 0000000000000..106b6b5c758b1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/NativeTableMetadataFactory.java @@ -0,0 +1,59 @@ +/* + * 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.metadata; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.storage.HoodieStorage; + +public class NativeTableMetadataFactory extends TableMetadataFactory { + private static final NativeTableMetadataFactory INSTANCE = new NativeTableMetadataFactory(); + + public static NativeTableMetadataFactory getInstance() { + return INSTANCE; + } + + @Override + public HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieStorage storage, + HoodieMetadataConfig metadataConfig, String datasetBasePath, boolean reuse) { + if (metadataConfig.isEnabled()) { + HoodieBackedTableMetadata metadata = createHoodieBackedTableMetadata(engineContext, storage, metadataConfig, datasetBasePath, reuse); + // If the MDT is not initialized then we fallback to FSBackedTableMetadata + if (metadata.isMetadataTableInitialized()) { + return metadata; + } + LOG.warn("Falling back to FileSystemBackedTableMetadata as metadata table is not initialized"); + } + return createFSBackedTableMetadata(engineContext, storage, datasetBasePath); + } + + private FileSystemBackedTableMetadata createFSBackedTableMetadata(HoodieEngineContext engineContext, + HoodieStorage storage, + String datasetBasePath) { + return new FileSystemBackedTableMetadata(engineContext, storage, datasetBasePath); + } + + private HoodieBackedTableMetadata createHoodieBackedTableMetadata(HoodieEngineContext engineContext, + HoodieStorage storage, + HoodieMetadataConfig metadataConfig, + String datasetBasePath, + boolean reuse) { + return new HoodieBackedTableMetadata(engineContext, storage, metadataConfig, datasetBasePath, reuse); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/TableMetadataFactory.java b/hudi-common/src/main/java/org/apache/hudi/metadata/TableMetadataFactory.java new file mode 100644 index 0000000000000..4d113fdb2cbaf --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/TableMetadataFactory.java @@ -0,0 +1,44 @@ +/* + * 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.metadata; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.storage.HoodieStorage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class TableMetadataFactory { + + protected static Logger LOG = LoggerFactory.getLogger(TableMetadataFactory.class); + + public HoodieTableMetadata create(HoodieEngineContext engineContext, + HoodieStorage storage, + HoodieMetadataConfig metadataConfig, + String datasetBasePath) { + return create(engineContext, storage, metadataConfig, datasetBasePath, false); + } + + public abstract HoodieTableMetadata create(HoodieEngineContext engineContext, + HoodieStorage storage, + HoodieMetadataConfig metadataConfig, + String datasetBasePath, + boolean reuse); + +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index b35d6ce9c84d5..af0889c289b79 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -52,7 +52,6 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.storage.StorageConfiguration; import com.fasterxml.jackson.databind.ObjectMapper; @@ -347,13 +346,11 @@ private List getFileSlicesToRead(StorageConfiguration storageConf, metadataConfig, FileSystemViewStorageConfig.newBuilder().build(), HoodieCommonConfig.newBuilder().build(), - mc -> HoodieTableMetadata.create( + mc -> metaClient.getTableFormat().getMetadataFactory().create( engineContext, mc.getStorage(), metadataConfig, tablePath)); HoodieTableFileSystemView fsView = (HoodieTableFileSystemView) viewManager.getFileSystemView(metaClient); - List relativePartitionPathList = FSUtils.getAllPartitionPaths( - engineContext, metaClient.getStorage(), - metadataConfig, metaClient.getBasePath().toString()); + List relativePartitionPathList = FSUtils.getAllPartitionPaths(engineContext, metaClient, metadataConfig); List fileSlices = relativePartitionPathList.stream().flatMap(fsView::getAllFileSlices) .collect(Collectors.toList()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index 4a35c5c7a192b..79b32c2b15449 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -236,6 +236,9 @@ public static HoodieTableMetaClient.TableBuilder getMetaClientBuilder(HoodieTabl if (properties.containsKey("hoodie.write.table.version")) { builder.setTableVersion(Integer.parseInt(properties.getProperty("hoodie.write.table.version"))); } + if (properties.containsKey(HoodieTableConfig.TABLE_FORMAT.key())) { + builder.setTableFormat(properties.getProperty(HoodieTableConfig.TABLE_FORMAT.key())); + } String keyGen = properties.getProperty("hoodie.datasource.write.keygenerator.class"); if (!Objects.equals(keyGen, "org.apache.hudi.keygen.NonpartitionedKeyGenerator") diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestActiveTimeline.java new file mode 100644 index 0000000000000..bd3106a68e114 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestActiveTimeline.java @@ -0,0 +1,96 @@ +/* + * 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.tableformat; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantComparatorV2; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * An active timeline for test table format which merges timeline assuming test-format as the source of truth. + */ +public class TestActiveTimeline extends ActiveTimelineV2 { + + public TestActiveTimeline( + HoodieTableMetaClient metaClient, + Set includedExtensions, + boolean applyLayoutFilters) { + this.setInstants(getInstantsFromFileSystem(metaClient, includedExtensions, applyLayoutFilters)); + this.metaClient = metaClient; + } + + public TestActiveTimeline(HoodieTableMetaClient metaClient) { + this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), true); + } + + public TestActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilters) { + this( + metaClient, + Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), + applyLayoutFilters); + } + + public TestActiveTimeline() { + + } + + @Override + protected List getInstantsFromFileSystem( + HoodieTableMetaClient metaClient, + Set includedExtensions, + boolean applyLayoutFilters) { + Map instantsInTestTableFormat = TestTableFormat.getRecordedInstants(metaClient.getBasePath().toString()) + .stream() + .collect(Collectors.toMap(HoodieInstant::requestedTime, instant -> instant)); + List instantsFromHoodieTimeline = + super.getInstantsFromFileSystem(metaClient, includedExtensions, applyLayoutFilters); + List inflightInstantsInTestTableFormat = + instantsFromHoodieTimeline.stream() + .filter( + hoodieInstant -> !instantsInTestTableFormat.containsKey(hoodieInstant.requestedTime())) + .map( + instant -> { + if (instant.isCompleted()) { + return new HoodieInstant( + HoodieInstant.State.INFLIGHT, + instant.getAction(), + instant.requestedTime(), + instant.getCompletionTime(), + InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + return instant; + }) + .collect(Collectors.toList()); + List completedInstantsInTestTableFormat = + instantsInTestTableFormat.values().stream() + .filter(instantsFromHoodieTimeline::contains) + .collect(Collectors.toList()); + return Stream.concat(completedInstantsInTestTableFormat.stream(), inflightInstantsInTestTableFormat.stream()) + .sorted(InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR) + .collect(Collectors.toList()); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestFormatBackedTableMetadata.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestFormatBackedTableMetadata.java new file mode 100644 index 0000000000000..dfdf494f94141 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestFormatBackedTableMetadata.java @@ -0,0 +1,34 @@ +/* + * 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.tableformat; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.FileSystemBackedTableMetadata; +import org.apache.hudi.storage.HoodieStorage; + +/** + * An implementation of {@link FileSystemBackedTableMetadata} for testing used by test-format. + */ +public class TestFormatBackedTableMetadata extends FileSystemBackedTableMetadata { + + public TestFormatBackedTableMetadata( + HoodieEngineContext engineContext, HoodieStorage storage, String datasetBasePath) { + super(engineContext, storage, datasetBasePath); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java new file mode 100644 index 0000000000000..7f7f489483409 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java @@ -0,0 +1,110 @@ +/* + * 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.tableformat; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.common.TableFormat; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.metadata.TableMetadataFactory; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; + +/** + * Test implementation of TableFormat that records all Hoodie instants in memory. + * Used for functional testing of TableFormat. + */ +public class TestTableFormat implements TableFormat { + + private static final Map> RECORDED_INSTANTS = new ConcurrentHashMap<>(); + + public TestTableFormat() { + } + + public static List getRecordedInstants(String basePath) { + return RECORDED_INSTANTS.getOrDefault(basePath, Collections.emptyList()); + } + + public static void tearDown() { + RECORDED_INSTANTS.clear(); + } + + @Override + public String getName() { + return "test-format"; + } + + @Override + public void commit(HoodieCommitMetadata commitMetadata, HoodieInstant completedInstant, + HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + RECORDED_INSTANTS.putIfAbsent(metaClient.getBasePath().toString(), new CopyOnWriteArrayList<>()); + RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).add(completedInstant); + } + + @Override + public void clean(HoodieCleanMetadata cleanMetadata, HoodieInstant completedInstant, + HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, + FileSystemViewManager viewManager) { + RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).add(completedInstant); + } + + @Override + public void archive(List archivedInstants, HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { + RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).removeAll(archivedInstants); + } + + @Override + public void rollback(HoodieInstant completedInstant, HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { + // No-op. + } + + @Override + public void completedRollback(HoodieInstant rollbackInstant, HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { + RECORDED_INSTANTS.putIfAbsent(metaClient.getBasePath().toString(), new CopyOnWriteArrayList<>()); + RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).add(rollbackInstant); + } + + @Override + public void savepoint(HoodieInstant savepointInstant, HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { + RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).add(savepointInstant); + } + + @Override + public TimelineFactory getTimelineFactory() { + return new TestTimelineFactory(null); + } + + @Override + public TableMetadataFactory getMetadataFactory() { + return TestTableMetadataFactory.getInstance(); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableMetadataFactory.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableMetadataFactory.java new file mode 100644 index 0000000000000..033c0f6cd8d40 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableMetadataFactory.java @@ -0,0 +1,46 @@ +/* + * 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.tableformat; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.TableMetadataFactory; +import org.apache.hudi.storage.HoodieStorage; + +/** + * The test implementation of {@link TableMetadataFactory} used by test-format. + */ +public class TestTableMetadataFactory extends TableMetadataFactory { + private static final TestTableMetadataFactory INSTANCE = new TestTableMetadataFactory(); + + public static TestTableMetadataFactory getInstance() { + return INSTANCE; + } + + @Override + public HoodieTableMetadata create( + HoodieEngineContext engineContext, + HoodieStorage storage, + HoodieMetadataConfig metadataConfig, + String datasetBasePath, + boolean reuse) { + return new TestFormatBackedTableMetadata(engineContext, storage, datasetBasePath); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTimelineFactory.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTimelineFactory.java new file mode 100644 index 0000000000000..87a1b967b46d9 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTimelineFactory.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.tableformat; + +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstantReader; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.timeline.versioning.v2.ArchivedTimelineLoaderV2; +import org.apache.hudi.common.table.timeline.versioning.v2.ArchivedTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.BaseTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.CompletionTimeQueryViewV2; + +import java.util.stream.Stream; + +/** + * The test implementation of TimelineFactory used for functional testing. + */ +public class TestTimelineFactory extends TimelineFactory { + + public TestTimelineFactory(HoodieConfig config) { + // To match reflection. + } + + @Override + public HoodieTimeline createDefaultTimeline(Stream instants, HoodieInstantReader instantReader) { + return new BaseTimelineV2(instants, instantReader); + } + + @Override + public HoodieActiveTimeline createActiveTimeline() { + return new TestActiveTimeline(); + } + + @Override + public HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient) { + return new ArchivedTimelineV2(metaClient); + } + + @Override + public HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient, String startTs) { + return new ArchivedTimelineV2(metaClient, startTs); + } + + @Override + public ArchivedTimelineLoader createArchivedTimelineLoader() { + return new ArchivedTimelineLoaderV2(); + } + + @Override + public HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient) { + return new TestActiveTimeline(metaClient); + } + + @Override + public HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) { + return new TestActiveTimeline(metaClient, applyLayoutFilter); + } + + @Override + public CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient) { + return new CompletionTimeQueryViewV2(metaClient); + } + + @Override + public CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient, String eagerInstant) { + return new CompletionTimeQueryViewV2(metaClient, eagerInstant); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java index 8ab6547a6236d..a44ce02272d72 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java @@ -145,8 +145,7 @@ protected void preLoadIndexRecords() throws Exception { StoragePath basePath = hoodieTable.getMetaClient().getBasePath(); int taskID = getRuntimeContext().getIndexOfThisSubtask(); LOG.info("Start loading records in table {} into the index state, taskId = {}", basePath, taskID); - for (String partitionPath : FSUtils.getAllPartitionPaths( - new HoodieFlinkEngineContext(hadoopConf), hoodieTable.getStorage(), metadataConfig(conf), basePath)) { + for (String partitionPath : FSUtils.getAllPartitionPaths(new HoodieFlinkEngineContext(hadoopConf), hoodieTable.getMetaClient(), metadataConfig(conf))) { if (pattern.matcher(partitionPath).matches()) { loadRecords(partitionPath); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java index 351137c3d51b7..82a2830bab202 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java @@ -21,6 +21,8 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -30,7 +32,6 @@ import org.apache.hudi.source.stats.FileStatsIndex; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; -import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hudi.util.StreamerUtil; import org.apache.flink.annotation.VisibleForTesting; @@ -70,7 +71,8 @@ public class FileIndex implements Serializable { private final ColumnStatsProbe colStatsProbe; // for probing column stats private final Function partitionBucketIdFunc; // for bucket pruning private List partitionPaths; // cache of partition paths - private final FileStatsIndex fileStatsIndex; // for data skipping + private final FileStatsIndex fileStatsIndex;// for data skipping + private final Option metaClient; private FileIndex( StoragePath path, @@ -87,6 +89,7 @@ private FileIndex( this.partitionPruner = partitionPruner; this.fileStatsIndex = new FileStatsIndex(path.toString(), rowType, metadataConfig); this.partitionBucketIdFunc = partitionBucketIdFunc; + this.metaClient = tableExists ? Option.of(HoodieTableMetaClient.builder().setBasePath(path.toString()).setConf(HadoopFSUtils.getStorageConf(hadoopConf)).build()) : Option.empty(); } /** @@ -158,8 +161,7 @@ public List getFilesInPartitions() { return Collections.emptyList(); } Map> filesInPartitions = FSUtils.getFilesInPartitions( - new HoodieFlinkEngineContext(hadoopConf), - new HoodieHadoopStorage(path, HadoopFSUtils.getStorageConf(hadoopConf)), metadataConfig, path.toString(), partitions); + new HoodieFlinkEngineContext(hadoopConf), metaClient.get(), metadataConfig, partitions); int totalFilesNum = filesInPartitions.values().stream().mapToInt(List::size).sum(); if (totalFilesNum < 1) { // returns early for empty table. @@ -232,9 +234,7 @@ public List getOrBuildPartitionPaths() { if (this.partitionPaths != null) { return this.partitionPaths; } - List allPartitionPaths = this.tableExists ? FSUtils.getAllPartitionPaths( - new HoodieFlinkEngineContext(hadoopConf), - new HoodieHadoopStorage(path, HadoopFSUtils.getStorageConf(hadoopConf)), metadataConfig, path.toString()) + List allPartitionPaths = this.tableExists ? FSUtils.getAllPartitionPaths(new HoodieFlinkEngineContext(hadoopConf), metaClient.get(), metadataConfig) : Collections.emptyList(); if (this.partitionPruner == null) { this.partitionPaths = allPartitionPaths; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java index 0b3ffeff7d5f4..16f3acec6a711 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.Pair; @@ -102,9 +103,11 @@ public String getIndexPartitionName() { public HoodieTableMetadata getMetadataTable() { // initialize the metadata table lazily if (this.metadataTable == null) { - this.metadataTable = HoodieTableMetadata.create( + HoodieHadoopStorage storage = new HoodieHadoopStorage(basePath, FlinkClientUtil.getHadoopConf()); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(storage.getConf()).build(); + this.metadataTable = metaClient.getTableFormat().getMetadataFactory().create( HoodieFlinkEngineContext.DEFAULT, - new HoodieHadoopStorage(basePath, FlinkClientUtil.getHadoopConf()), + storage, metadataConfig, basePath); } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index 6218df3dda1dc..30d7d8e46af58 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -434,7 +434,7 @@ private void assertNoDuplicateFile(Configuration conf) { Set> fileIdCommitTimeSet = new HashSet<>(); HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); HoodieStorage storage = metaClient.getStorage(); - FSUtils.getAllPartitionPaths(HoodieFlinkEngineContext.DEFAULT, metaClient.getStorage(), metaClient.getBasePath(), false).forEach( + FSUtils.getAllPartitionPaths(HoodieFlinkEngineContext.DEFAULT, metaClient, false).forEach( partition -> { try { storage.listDirectEntries(FSUtils.constructAbsolutePath(metaClient.getBasePath(), partition)) diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java index 191c0e7ac846c..97070b6266a63 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java @@ -290,7 +290,7 @@ public void testDropInvalidConfigs() { @Test public void testDefinedTableConfigs() { List> configProperties = HoodieTableConfig.definedTableConfigs(); - assertEquals(38, configProperties.size()); + assertEquals(39, configProperties.size()); configProperties.forEach(c -> { assertNotNull(c); assertFalse(c.doc().isEmpty()); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index 819ae0c1c5273..b1bfa61d65cae 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -97,8 +97,7 @@ protected List getPartitions(Option partitionsLimit) throws IOE // Using FSUtils.getFS here instead of metaClient.getFS() since we don't want to count these listStatus // calls in metrics as they are not part of normal HUDI operation. HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient.getStorage(), metaClient.getBasePath(), - HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS); + List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS); // Sort partition so we can pick last N partitions by default Collections.sort(partitionPaths); if (!partitionPaths.isEmpty()) { diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java index 0b31c0f6f4f74..4672de9bbbdcc 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java @@ -60,9 +60,9 @@ protected void deleteInstantFile(HoodieInstant instant) { } @Override - protected void transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata) { + protected HoodieInstant transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option metadata) { ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime())); - metaserverClient.transitionInstantState(databaseName, tableName, fromInstant, toInstant, + return metaserverClient.transitionInstantState(databaseName, tableName, fromInstant, toInstant, metadata.map(m -> convertMetadataToByteArray(m, metadataSerDeV2))); } diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClient.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClient.java index 83ee52eba20c4..f27c423510dfe 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClient.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClient.java @@ -45,7 +45,7 @@ public interface HoodieMetaserverClient extends Serializable, AutoCloseable { void createNewInstant(String db, String tb, HoodieInstant instant, Option content); - void transitionInstantState(String db, String tb, HoodieInstant fromInstant, HoodieInstant toInstant, Option content); + HoodieInstant transitionInstantState(String db, String tb, HoodieInstant fromInstant, HoodieInstant toInstant, Option content); void deleteInstant(String db, String tb, HoodieInstant instant); diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java index 409c3aeece409..3858f0a303688 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java @@ -136,11 +136,12 @@ public void createNewInstant(String db, String tb, HoodieInstant instant, Option } @Override - public void transitionInstantState(String db, String tb, HoodieInstant fromInstant, HoodieInstant toInstant, Option content) { + public HoodieInstant transitionInstantState(String db, String tb, HoodieInstant fromInstant, HoodieInstant toInstant, Option content) { exceptionWrapper(() -> this.client.transitionInstantState(db, tb, EntityConversions.toTHoodieInstant(fromInstant), EntityConversions.toTHoodieInstant(toInstant), getByteBuffer(content))).get(); + return toInstant; } @Override diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteTableCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteTableCommitActionExecutor.java index 772e388f9113a..4dc82a34064dc 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteTableCommitActionExecutor.java @@ -48,9 +48,8 @@ public WriteOperationType getWriteOperationType() { protected Map> getPartitionToReplacedFileIds(HoodieData writeStatuses) { HoodieEngineContext context = writeClient.getEngineContext(); List partitionPaths = FSUtils.getAllPartitionPaths(context, - table.getStorage(), - writeConfig.getMetadataConfig(), - table.getMetaClient().getBasePath()); + table.getMetaClient(), + writeConfig.getMetadataConfig()); if (partitionPaths == null || partitionPaths.isEmpty()) { return Collections.emptyMap(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 8c9b9d1ca803f..13a43fb1f8c98 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -326,6 +326,7 @@ class HoodieSparkSqlWriterInternal { .setRecordMergeStrategyId(recordMergeStrategyId) .setRecordMergeMode(RecordMergeMode.getValue(hoodieConfig.getString(HoodieWriteConfig.RECORD_MERGE_MODE))) .setMultipleBaseFileFormatsEnabled(hoodieConfig.getBoolean(HoodieTableConfig.MULTIPLE_BASE_FILE_FORMATS_ENABLE)) + .setTableFormat(hoodieConfig.getStringOrDefault(HoodieTableConfig.TABLE_FORMAT)) .initTable(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration), path) } @@ -433,7 +434,7 @@ class HoodieSparkSqlWriterInternal { val partitionsToDelete = if (parameters.contains(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key())) { val partitionColsToDelete = parameters(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key()).split(",") java.util.Arrays.asList(resolvePartitionWildcards(java.util.Arrays.asList(partitionColsToDelete: _*).asScala.toList, jsc, - tableMetaClient.getStorage, hoodieConfig, basePath.toString): _*) + tableMetaClient.getStorage, hoodieConfig, basePath.toString, tableMetaClient): _*) } else { val genericRecords = HoodieSparkUtils.createRdd(df, avroRecordName, avroRecordNamespace) genericRecords.map(gr => keyGenerator.getKey(gr).getPartitionPath).toJavaRDD().distinct().collect() @@ -597,13 +598,13 @@ class HoodieSparkSqlWriterInternal { * @return Pair of(boolean, table schema), where first entry will be true only if schema conversion is required. */ private def resolvePartitionWildcards(partitions: List[String], jsc: JavaSparkContext, - storage: HoodieStorage, cfg: HoodieConfig, basePath: String): List[String] = { + storage: HoodieStorage, cfg: HoodieConfig, basePath: String, metaClient: HoodieTableMetaClient): List[String] = { //find out if any of the input partitions have wildcards //note:spark-sql may url-encode special characters (* -> %2A) var (wildcardPartitions, fullPartitions) = partitions.partition(partition => partition.matches(".*(\\*|%2A).*")) val allPartitions = FSUtils.getAllPartitionPaths(new HoodieSparkEngineContext(jsc): HoodieEngineContext, - storage, HoodieMetadataConfig.newBuilder().fromProperties(cfg.getProps).build(), basePath) + metaClient, HoodieMetadataConfig.newBuilder().fromProperties(cfg.getProps).build()) if (fullPartitions.nonEmpty) { fullPartitions = fullPartitions.filter(partition => allPartitions.contains(partition)) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkBaseIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkBaseIndexSupport.scala index 7ff4e4936c136..d47c8785ab265 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkBaseIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkBaseIndexSupport.scala @@ -42,7 +42,7 @@ abstract class SparkBaseIndexSupport(spark: SparkSession, metaClient: HoodieTableMetaClient) { @transient protected lazy val engineCtx = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) @transient protected lazy val metadataTable: HoodieTableMetadata = - HoodieTableMetadata.create(engineCtx, metaClient.getStorage, metadataConfig, metaClient.getBasePath.toString) + metaClient.getTableFormat.getMetadataFactory.create(engineCtx, metaClient.getStorage, metadataConfig, metaClient.getBasePath.toString) def getIndexName: String diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index 69473ec355bb4..03871490f8a14 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -180,7 +180,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten def getPartitionPaths: Seq[String] = { val droppedPartitions = TimelineUtils.getDroppedPartitions(metaClient, org.apache.hudi.common.util.Option.empty(), org.apache.hudi.common.util.Option.empty()) - getAllPartitionPaths(spark, table, metaClient.getStorage) + getAllPartitionPaths(spark, table, metaClient) .filter(!droppedPartitions.contains(_)) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala index c365a2cfcece8..942adda6ff9ac 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala @@ -66,26 +66,25 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport { avroSchema.map(AvroConversionUtils.convertAvroSchemaToStructType) } - def getAllPartitionPaths(spark: SparkSession, table: CatalogTable, storage: HoodieStorage): Seq[String] = { + def getAllPartitionPaths(spark: SparkSession, table: CatalogTable, metaClient: HoodieTableMetaClient): Seq[String] = { val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) val metadataConfig = { val properties = TypedProperties.fromMap((spark.sessionState.conf.getAllConfs ++ table.storage.properties ++ table.properties).asJava) HoodieMetadataConfig.newBuilder.fromProperties(properties).build() } - FSUtils.getAllPartitionPaths(sparkEngine, storage, metadataConfig, getTableLocation(table, spark)).asScala.toSeq + FSUtils.getAllPartitionPaths(sparkEngine, metaClient, metadataConfig).asScala.toSeq } def getFilesInPartitions(spark: SparkSession, table: CatalogTable, - storage: HoodieStorage, + metaClient: HoodieTableMetaClient, partitionPaths: Seq[String]): Map[String, Seq[StoragePathInfo]] = { val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) val metadataConfig = { val properties = TypedProperties.fromMap((spark.sessionState.conf.getAllConfs ++ table.storage.properties ++ table.properties).asJava) HoodieMetadataConfig.newBuilder.fromProperties(properties).build() } - FSUtils.getFilesInPartitions(sparkEngine, storage, metadataConfig, getTableLocation(table, spark), - partitionPaths.toArray).asScala + FSUtils.getFilesInPartitions(sparkEngine, metaClient, metadataConfig, partitionPaths.toArray).asScala .map(e => (e._1, e._2.asScala.toSeq)) .toMap } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala index 777d88f1bf1a0..a713bdbebc6d2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.hudi.command -import org.apache.hudi.hadoop.fs.HadoopFSUtils -import org.apache.hudi.storage.HoodieStorageUtils - import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier @@ -87,9 +84,7 @@ case class RepairHoodieTableCommand(tableName: TableIdentifier, val total = partitionSpecsAndLocs.length val partitionList = partitionSpecsAndLocs.map(_._2.toString) val partitionStats = if (spark.sqlContext.conf.gatherFastStats && total > 0) { - HoodieSqlCommonUtils.getFilesInPartitions(spark, table, - HoodieStorageUtils.getStorage(partitionList.head, HadoopFSUtils.getStorageConf(spark.sessionState.newHadoopConf())), - partitionList) + HoodieSqlCommonUtils.getFilesInPartitions(spark, table, hoodieCatalogTable.metaClient, partitionList) .mapValues(statuses => PartitionStatistics(statuses.length, statuses.map(_.getLength).sum)) } else { Map.empty[String, PartitionStatistics] diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala index 0420de211d426..4b4f0e3261bd4 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/PartitionBucketIndexManager.scala @@ -146,7 +146,7 @@ class PartitionBucketIndexManager extends BaseProcedure val mdtEnable = metaClient.getStorage().exists(new StoragePath(metaClient.getBasePath, HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH)) // get all partition paths - val allPartitions = FSUtils.getAllPartitionPaths(context, metaClient.getStorage, metaClient.getBasePath, mdtEnable) + val allPartitions = FSUtils.getAllPartitionPaths(context, metaClient, mdtEnable) val usePartitionBucketIndexBefore = PartitionBucketIndexUtils.isPartitionSimpleBucketIndex(context.getStorageConf, basePath.toString) var partition2BucketWithLatestHashingConfig: util.Map[String, Integer] = null @@ -195,8 +195,8 @@ class PartitionBucketIndexManager extends BaseProcedure logInfo("Perform OVERWRITE with dry-run disabled.") val partitionsToRescale = rescalePartitionsMap.keys // get all fileSlices need to read - val allFilesMap = FSUtils.getFilesInPartitions(context, metaClient.getStorage(), HoodieMetadataConfig.newBuilder.enable(mdtEnable).build, - metaClient.getBasePath.toString, partitionsToRescale.map(relative => { + val allFilesMap = FSUtils.getFilesInPartitions(context, metaClient, HoodieMetadataConfig.newBuilder.enable(mdtEnable).build, + partitionsToRescale.map(relative => { new StoragePath(basePath, relative) }).map(storagePath => storagePath.toString).toArray) val files = allFilesMap.values().asScala.flatMap(x => x.asScala).toList diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala index 3cedd1a9cba3d..1f7ebbfde237a 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala @@ -63,7 +63,7 @@ class RepairMigratePartitionMetaProcedure extends BaseProcedure with ProcedureBu val metaClient = createMetaClient(jsc, tablePath) val engineContext: HoodieLocalEngineContext = new HoodieLocalEngineContext(metaClient.getStorageConf) - val partitionPaths: util.List[String] = FSUtils.getAllPartitionPaths(engineContext, metaClient.getStorage, tablePath, false) + val partitionPaths: util.List[String] = FSUtils.getAllPartitionPaths(engineContext, metaClient, false) val basePath: StoragePath = new StoragePath(tablePath) val rows = new util.ArrayList[Row](partitionPaths.size) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala index 1d3bfd19aaa57..fa259c5970b93 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala @@ -25,7 +25,6 @@ import org.apache.hudi.common.data.HoodieData import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{FileSlice, HoodieRecord} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieInstantTimeGenerator} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.metadata.{HoodieTableMetadata, HoodieTableMetadataUtil} import org.apache.hudi.storage.StoragePath @@ -35,7 +34,6 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.hudi.command.procedures.ShowColumnStatsOverlapProcedure.{MAX_VALUE_TYPE, MIN_VALUE_TYPE} import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} -import java.time.Instant import java.util import java.util.function.Supplier @@ -109,7 +107,7 @@ class ShowColumnStatsOverlapProcedure extends BaseProcedure with ProcedureBuilde val columnStatsIndex = new ColumnStatsIndexSupport(spark, schema, metadataConfig, metaClient) val fsView = buildFileSystemView(table) val engineCtx = new HoodieSparkEngineContext(jsc) - val metaTable = HoodieTableMetadata.create(engineCtx, metaClient.getStorage, metadataConfig, basePath) + val metaTable = metaClient.getTableFormat.getMetadataFactory.create(engineCtx, metaClient.getStorage, metadataConfig, basePath) val allFileSlices = getAllFileSlices(partitionsSeq, metaTable, fsView) val fileSlicesSizeByPartition = allFileSlices.groupBy(_.getPartitionPath).mapValues(_.size) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala index ff87543854bca..d2f7ead67ea41 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala @@ -115,7 +115,7 @@ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure wit instants = instants.filter(instant => predicate.test(maxInstant, instant.requestedTime)) } - val filteredTimeline = metaClient.getTimelineLayout.getTimelineFactory.createDefaultTimeline( + val filteredTimeline = metaClient.getTableFormat.getTimelineFactory.createDefaultTimeline( new JArrayList[HoodieInstant](instants.toList.asJava).stream(), metaClient.getActiveTimeline.getInstantReader) new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses) } @@ -148,7 +148,7 @@ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure wit maxInstant: String, merge: Boolean): JList[Row] = { var fileSliceStream: JStream[FileSlice] = JStream.empty() - val completionTimeQueryView =metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient) + val completionTimeQueryView =metaClient.getTableFormat.getTimelineFactory.createCompletionTimeQueryView(metaClient) if (merge) { partitions.foreach(p => fileSliceStream = JStream.concat(fileSliceStream, fsView.getLatestMergedFileSlicesBeforeOrOn(p, maxInstant))) } else { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowInvalidParquetProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowInvalidParquetProcedure.scala index e052f5b1fc5e0..395dc740dff78 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowInvalidParquetProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowInvalidParquetProcedure.scala @@ -22,7 +22,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.util.StringUtils import org.apache.hudi.hadoop.fs.HadoopFSUtils -import org.apache.hudi.metadata.HoodieTableMetadata +import org.apache.hudi.metadata.{HoodieTableMetadata, NativeTableMetadataFactory} import org.apache.hudi.storage.hadoop.HoodieHadoopStorage import collection.JavaConverters._ @@ -65,7 +65,7 @@ class ShowInvalidParquetProcedure extends BaseProcedure with ProcedureBuilder { val storageConf = HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()) val storage = new HoodieHadoopStorage(srcPath, storageConf) val metadataConfig = HoodieMetadataConfig.newBuilder.enable(false).build - val metadata = HoodieTableMetadata.create(new HoodieSparkEngineContext(jsc), storage, metadataConfig, srcPath) + val metadata = NativeTableMetadataFactory.getInstance().create(new HoodieSparkEngineContext(jsc), storage, metadataConfig, srcPath) val partitionPaths: java.util.List[String] = metadata.getPartitionPathWithPathPrefixes(partitions.split(",").toList.asJava) val instantsList = if (StringUtils.isNullOrEmpty(instants)) Array.empty[String] else instants.split(",") val fileStatus = partitionPaths.asScala.flatMap(part => { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala index 6abae9b598bec..47da3695372db 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala @@ -82,7 +82,7 @@ class ShowMetadataTableColumnStatsProcedure extends BaseProcedure with Procedure val fsView = buildFileSystemView(table, engineCtx) val allFileSlices: Set[FileSlice] = { if (partitionsSeq.isEmpty) { - val metaTable = HoodieTableMetadata.create(engineCtx, metaClient.getStorage, metadataConfig, basePath) + val metaTable = metaClient.getTableFormat.getMetadataFactory.create(engineCtx, metaClient.getStorage, metadataConfig, basePath) metaTable.getAllPartitionPaths .asScala .flatMap(path => fsView.getLatestFileSlices(path).iterator().asScala) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java index 9e4b8f169f1f8..72947e617e546 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java @@ -334,7 +334,7 @@ public void testSecondaryIndexRecordGenerationForMOR() throws IOException { .withIndexOptions(Collections.emptyMap()) .build(); HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(true).withSecondaryIndexParallelism(2).build(); - HoodieTableMetadata metadata = HoodieTableMetadata.create(engineContext, storage, metadataConfig, metaClient.getBasePath().toString()); + HoodieTableMetadata metadata = metaClient.getTableFormat().getMetadataFactory().create(engineContext, storage, metadataConfig, metaClient.getBasePath().toString()); HoodieTableFileSystemView metadataView = new HoodieTableFileSystemView(metadata, metaClient, metaClient.getActiveTimeline()); metadataView.loadAllPartitions(); List> partitionFileSlicePairs = new ArrayList<>(); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index d8f4c957c958e..e833919c30ac4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -418,7 +418,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), - FSUtils.getAllPartitionPaths(context, storage, basePath, false).stream() + FSUtils.getAllPartitionPaths(context, metaClient, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -436,7 +436,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), - FSUtils.getAllPartitionPaths(context, storage, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() + FSUtils.getAllPartitionPaths(context, metaClient, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -452,7 +452,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), - FSUtils.getAllPartitionPaths(context, storage, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() + FSUtils.getAllPartitionPaths(context, metaClient, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -469,7 +469,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), - FSUtils.getAllPartitionPaths(context, storage, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() + FSUtils.getAllPartitionPaths(context, metaClient, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -484,7 +484,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), - FSUtils.getAllPartitionPaths(context, storage, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() + FSUtils.getAllPartitionPaths(context, metaClient, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -501,7 +501,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), - FSUtils.getAllPartitionPaths(context, storage, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() + FSUtils.getAllPartitionPaths(context, metaClient, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java index 712c8ba707559..ad5ff84e6d4ce 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java @@ -809,7 +809,7 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws E HoodieTableMetaClient metadataMetaClient = createMetaClient(metadataTableBasePath); List metadataTablePartitions = FSUtils.getAllPartitionPaths( - engineContext, metadataMetaClient.getStorage(), metadataMetaClient.getBasePath(), false); + engineContext, metadataMetaClient, false); Option completedReplaceInstant = metadataMetaClient.reloadActiveTimeline().getCompletedReplaceTimeline().lastInstant(); assertTrue(completedReplaceInstant.isPresent()); @@ -1184,7 +1184,7 @@ public void testMetadataRollbackDuringInit() throws Exception { client.commit(newCommitTime2, writeStatuses); } - HoodieTableMetadata metadataReader = HoodieTableMetadata.create( + HoodieTableMetadata metadataReader = metaClient.getTableFormat().getMetadataFactory().create( context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); Map result = metadataReader .readRecordIndex(records1.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toList())); @@ -3634,7 +3634,7 @@ public void testDeleteWithRecordIndex() throws Exception { allRecords.addAll(secondBatchOfrecords); // RI should have created mappings for all the records inserted above - HoodieTableMetadata metadataReader = HoodieTableMetadata.create( + HoodieTableMetadata metadataReader = metaClient.getTableFormat().getMetadataFactory().create( context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); Map result = metadataReader .readRecordIndex(allRecords.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toList())); @@ -3651,7 +3651,7 @@ public void testDeleteWithRecordIndex() throws Exception { client.commit(deleteTime, jsc.parallelize(writeStatuses)); // RI should not return mappings for deleted records - metadataReader = HoodieTableMetadata.create( + metadataReader = metaClient.getTableFormat().getMetadataFactory().create( context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); result = metadataReader.readRecordIndex(allRecords.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toList())); assertEquals(allRecords.size() - recordsToDelete.size(), result.size(), "RI should not have mapping for deleted records"); @@ -3666,12 +3666,12 @@ public void testDeleteWithRecordIndex() throws Exception { List writeStatuses = client.delete(jsc.emptyRDD(), deleteTime).collect(); client.commit(deleteTime, jsc.parallelize(writeStatuses)); - HoodieTableMetadata metadataReader = HoodieTableMetadata.create( + HoodieTableMetadata metadataReader = metaClient.getTableFormat().getMetadataFactory().create( context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); assertTrue(metadataReader.getLatestCompactionTime().isPresent(), "Compaction should have taken place on MDT"); // RI should not return mappings for deleted records - metadataReader = HoodieTableMetadata.create(context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); + metadataReader = metaClient.getTableFormat().getMetadataFactory().create(context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); Map result = metadataReader.readRecordIndex(allRecords.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toList())); assertEquals(allRecords.size() - keysToDelete.size(), result.size(), "RI should not have mapping for deleted records"); result.keySet().forEach(mappingKey -> assertFalse(keysToDelete.contains(mappingKey), "RI should not have mapping for deleted records")); @@ -3682,7 +3682,7 @@ public void testDeleteWithRecordIndex() throws Exception { client.commit(reinsertTime, jsc.parallelize(writeStatuses)); // New mappings should have been created for re-inserted records and should map to the new commit time - metadataReader = HoodieTableMetadata.create(context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); + metadataReader = metaClient.getTableFormat().getMetadataFactory().create(context, storage, writeConfig.getMetadataConfig(), writeConfig.getBasePath()); result = metadataReader.readRecordIndex(allRecords.stream().map(HoodieRecord::getRecordKey).collect(Collectors.toList())); assertEquals(allRecords.size(), result.size(), "RI should have mappings for re-inserted records"); for (String reInsertedKey : keysToDelete) { @@ -3738,8 +3738,7 @@ private void validateMetadata(SparkRDDWriteClient testClient, Option ign // Metadata table has a fixed number of partitions // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. - List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, storage, getMetadataTableBasePath(basePath), - false); + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient, false); // Secondary index is enabled by default but no MDT partition corresponding to it is available final boolean isPartitionStatsEnabled; if (!metadataWriter.getEnabledPartitionTypes().contains(COLUMN_STATS)) { @@ -3916,7 +3915,7 @@ public static void validateMetadata(HoodieWriteConfig config, Option ign // Metadata table has a fixed number of partitions // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. - List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, storage, getMetadataTableBasePath(basePath), false); + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, metadataMetaClient, false); // check if the last instant is restore, then the metadata table should have only the partitions that are not deleted metaClient.reloadActiveTimeline().getReverseOrderedInstants().findFirst().ifPresent(instant -> { if (instant.getAction().equals(HoodieActiveTimeline.RESTORE_ACTION)) { @@ -4018,7 +4017,8 @@ private static HoodieBackedTableMetadataWriter, JavaRDD) v1 -> - HoodieTableMetadata.create(context, metaClient.getStorage(), config.getMetadataConfig(), config.getBasePath())) + metaClient.getTableFormat().getMetadataFactory().create(context, metaClient.getStorage(), config.getMetadataConfig(), config.getBasePath())) .getFileSystemView(basePath); assertFileSystemViews(config, enableMdt, storageType); @@ -224,7 +224,7 @@ private void assertFileSystemViews(HoodieWriteConfig writeConfig, boolean enable HoodieTableFileSystemView actualFileSystemView = (HoodieTableFileSystemView) FileSystemViewManager .createViewManager(context, writeConfig.getMetadataConfig(), viewStorageConfig, writeConfig.getCommonConfig(), (SerializableFunctionUnchecked) v1 -> - HoodieTableMetadata.create(context, metaClient.getStorage(), writeConfig.getMetadataConfig(), writeConfig.getBasePath())) + metaClient.getTableFormat().getMetadataFactory().create(context, metaClient.getStorage(), writeConfig.getMetadataConfig(), writeConfig.getBasePath())) .getFileSystemView(basePath); try { assertForFSVEquality(expectedFileSystemView, actualFileSystemView, enableMdt, Option.empty()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java index 6bb11c67ddce5..80b3455dc4076 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java @@ -395,21 +395,19 @@ private void validateFilesExistInCompactionPlan(String compactionInstant) { } private void validateFileListingInMetadataTable() { - List partitionPaths = FSUtils.getAllPartitionPaths(context(), hoodieStorage(), basePath(), false) + List partitionPaths = FSUtils.getAllPartitionPaths(context(), metaClient, false) .stream() .map(e -> new StoragePath(basePath(), e).toString()) .collect(Collectors.toList()); Map> filesFromStorage = FSUtils.getFilesInPartitions( context(), - hoodieStorage(), + metaClient, HoodieMetadataConfig.newBuilder().enable(false).build(), - basePath(), partitionPaths.toArray(new String[0])); Map> filesFromMetadataTable = FSUtils.getFilesInPartitions( context(), - hoodieStorage(), + metaClient, HoodieMetadataConfig.newBuilder().enable(true).build(), - basePath(), partitionPaths.toArray(new String[0])); assertEquals(filesFromStorage.size(), filesFromMetadataTable.size()); for (String partition : filesFromStorage.keySet()) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat b/hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat new file mode 100644 index 0000000000000..eb7e6220e5e68 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat @@ -0,0 +1,19 @@ +# +# 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. +# +org.apache.hudi.tableformat.TestTableFormat \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index b9150d3de0090..9a2c39f718b98 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -634,7 +634,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS metaClient = HoodieTableMetaClient.reload(metaClient) // Test getting partition paths in a subset of directories - val metadata = HoodieTableMetadata.create(context, + val metadata = metaClient.getTableFormat.getMetadataFactory.create(context, metaClient.getStorage, HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build(), metaClient.getBasePath.toString) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala new file mode 100644 index 0000000000000..ce770599ed3e8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterWithTestFormat.scala @@ -0,0 +1,717 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + + +import org.apache.hudi.client.SparkRDDWriteClient +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.model._ +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.testutils.HoodieTestDataGenerator +import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieIndexConfig, HoodieWriteConfig} +import org.apache.hudi.exception.{HoodieException, SchemaCompatibilityException} +import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode +import org.apache.hudi.functional.TestBootstrap +import org.apache.hudi.hadoop.fs.HadoopFSUtils +import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} +import org.apache.hudi.testutils.DataSourceTestUtils +import org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient + +import org.apache.avro.Schema +import org.apache.commons.io.FileUtils +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.{DataFrame, Row, SaveMode} +import org.apache.spark.sql.functions.{expr, lit} +import org.apache.spark.sql.hudi.command.SqlKeyGenerator +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider._ +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{spy, times, verify} +import org.scalatest.Assertions.assertThrows +import org.scalatest.Matchers.intercept + +import java.time.Instant +import java.util.{Collections, Date, UUID} + +import scala.collection.JavaConverters._ + +/** + * Test suite for SparkSqlWriter class with format as "test-format" that implements org.apache.hudi.common.table.HoodieTableFormat. + * All cases of using of {@link HoodieTimelineTimeZone.UTC} should be done in a separate test class {@link TestHoodieSparkSqlWriterUtc}. + * Otherwise UTC tests will generate infinite loops, if there is any initiated test with time zone that is greater then UTC+0. + * The reason is in a saved value in the heap of static {@link org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.lastInstantTime}. + */ +class TestHoodieSparkSqlWriterWithTestFormat extends HoodieSparkWriterTestBase { + + /** + * Local utility method for performing bulk insert tests. + * + * @param sortMode Bulk insert sort mode + * @param populateMetaFields Flag for populating meta fields + */ + def testBulkInsertWithSortMode(sortMode: BulkInsertSortMode, populateMetaFields: Boolean = true, enableOCCConfigs: Boolean = false): Unit = { + //create a new table + var fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") + .updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields)) + .updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), sortMode.name()) + .updated(HoodieTableConfig.TABLE_FORMAT.key, "test-format") + + if (enableOCCConfigs) { + fooTableModifier = fooTableModifier + .updated("hoodie.write.concurrency.mode", "optimistic_concurrency_control") + .updated("hoodie.clean.failed.writes.policy", "LAZY") + .updated("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.InProcessLockProvider") + } + + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val inserts = DataSourceTestUtils.generateRandomRows(1000) + + // add some updates so that preCombine kicks in + val toUpdateDataset = sqlContext.createDataFrame(DataSourceTestUtils.getUniqueRows(inserts, 40), structType) + val updates = DataSourceTestUtils.updateRowsWithUpdatedTs(toUpdateDataset) + val records = inserts.asScala.union(updates.asScala) + val recordsSeq = convertRowListToSeq(records.asJava) + val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) + // write to Hudi + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df) + + // collect all partition paths to issue read of parquet files + val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, + HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH) + // Check the entire dataset has all records still + val fullPartitionPaths = new Array[String](3) + for (i <- fullPartitionPaths.indices) { + fullPartitionPaths(i) = String.format("%s/%s/*", tempBasePath, partitions(i)) + } + // fetch all records from parquet files generated from write to hudi + val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2)) + if (!populateMetaFields) { + List(0, 1, 2, 3, 4).foreach(i => assertEquals(0, actualDf.select(HoodieRecord.HOODIE_META_COLUMNS.get(i)).filter(entry => !(entry.mkString(",").equals(""))).count())) + } + // remove metadata columns so that expected and actual DFs can be compared as is + val trimmedDf = dropMetaFields(actualDf) + assert(df.except(trimmedDf).count() == 0) + } + + /** + * Test case for throw hoodie exception when there already exist a table + * with different name with Append Save mode + */ + @Test + def testThrowExceptionAlreadyExistsWithAppendSaveMode(): Unit = { + //create a new table + val fooTableModifier = Map( + "path" -> tempBasePath, + HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, + "hoodie.datasource.write.recordkey.field" -> "uuid", + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + HoodieTableConfig.TABLE_FORMAT.key -> "test-format") + val dataFrame = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, dataFrame) + + //on same path try append with different("hoodie_bar_tbl") table name which should throw an exception + val barTableModifier = Map( + "path" -> tempBasePath, + HoodieWriteConfig.TBL_NAME.key -> "hoodie_bar_tbl", + "hoodie.datasource.write.recordkey.field" -> "uuid", + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4") + val dataFrame2 = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) + val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableModifier, dataFrame2)) + assert(tableAlreadyExistException.getMessage.contains("Config conflict")) + assert(tableAlreadyExistException.getMessage.contains(s"${HoodieWriteConfig.TBL_NAME.key}:\thoodie_bar_tbl\thoodie_foo_tbl")) + + //on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception + val deleteTableModifier = barTableModifier ++ Map(DataSourceWriteOptions.OPERATION.key -> "delete") + val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableModifier, dataFrame2)) + assert(tableAlreadyExistException.getMessage.contains("Config conflict")) + assert(tableAlreadyExistException.getMessage.contains(s"${HoodieWriteConfig.TBL_NAME.key}:\thoodie_bar_tbl\thoodie_foo_tbl")) + } + + /** + * Test case for Do not validate table config if save mode is set to Overwrite + */ + @Test + def testValidateTableConfigWithOverwriteSaveMode(): Unit = { + //create a new table + val tableModifier1 = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, + "hoodie.datasource.write.recordkey.field" -> "uuid", HoodieTableConfig.TABLE_FORMAT.key -> "test-format") + val dataFrame = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, tableModifier1, dataFrame) + + //on same path try write with different RECORDKEY_FIELD_NAME and Append SaveMode should throw an exception + val tableModifier2 = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, + "hoodie.datasource.write.recordkey.field" -> "ts", HoodieTableConfig.TABLE_FORMAT.key -> "test-format") + val dataFrame2 = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) + val hoodieException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, tableModifier2, dataFrame2)) + assert(hoodieException.getMessage.contains("Config conflict")) + assert(hoodieException.getMessage.contains(s"RecordKey:\tts\tuuid")) + + //on same path try write with different RECORDKEY_FIELD_NAME and Overwrite SaveMode should be successful. + assert(HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, tableModifier2, dataFrame2)._1) + } + + + /** + * Test case for each bulk insert sort mode + * + * @param sortMode Bulk insert sort mode + */ + @ParameterizedTest + @EnumSource(value = classOf[BulkInsertSortMode]) + def testBulkInsertForSortMode(sortMode: BulkInsertSortMode): Unit = { + testBulkInsertWithSortMode(sortMode, populateMetaFields = true) + } + + @Test + def testBulkInsertForSortModeWithOCC(): Unit = { + testBulkInsertWithSortMode(BulkInsertSortMode.GLOBAL_SORT, populateMetaFields = true, true) + } + + /** + * Test case for Bulk insert with populating meta fields or + * without populating meta fields. + * + * @param populateMetaFields Flag for populating meta fields + */ + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testBulkInsertForPopulateMetaFields(populateMetaFields: Boolean): Unit = { + testBulkInsertWithSortMode(BulkInsertSortMode.NONE, populateMetaFields) + } + + /** + * Test case for disable and enable meta fields. + */ + @Test + def testDisableAndEnableMetaFields(): Unit = { + testBulkInsertWithSortMode(BulkInsertSortMode.NONE, populateMetaFields = false) + //create a new table + val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") + .updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name()) + .updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), "true") + .updated(HoodieTableConfig.TABLE_FORMAT.key, "test-format") + + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val inserts = DataSourceTestUtils.generateRandomRows(1000) + val df = spark.createDataFrame(sc.parallelize(inserts.asScala.toSeq), structType) + try { + // write to Hudi + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df) + fail("Should have thrown exception") + } catch { + case e: HoodieException => assertTrue(e.getMessage.startsWith("Config conflict")) + case e: Exception => fail(e); + } + } + + /** + * Test case for drop duplicates row writing for bulk_insert. + */ + @Test + def testDropDuplicatesRowForBulkInsert(): Unit = { + try { + //create a new table + val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") + .updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "true") + .updated(HoodieTableConfig.TABLE_FORMAT.key, "test-format") + + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val records = DataSourceTestUtils.generateRandomRows(100) + val recordsSeq = convertRowListToSeq(records) + val df = spark.createDataFrame(spark.sparkContext.parallelize(recordsSeq), structType) + // write to Hudi + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df) + fail("Drop duplicates with bulk insert in row writing should have thrown exception") + } catch { + case e: HoodieException => assertTrue(e.getMessage.contains("Dropping duplicates with bulk_insert in row writer path is not supported yet")) + } + } + + /** + * Test case for insert dataset without precombine field. + */ + @Test + def testInsertDatasetWithoutPrecombineField(): Unit = { + + val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false") + .updated(HoodieTableConfig.TABLE_FORMAT.key, "test-format") + + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val records = DataSourceTestUtils.generateRandomRows(100) + val recordsSeq = convertRowListToSeq(records) + val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) + // write to Hudi + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df) + + // collect all partition paths to issue read of parquet files + val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, + HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH) + // Check the entire dataset has all records still + val fullPartitionPaths = new Array[String](3) + for (i <- fullPartitionPaths.indices) { + fullPartitionPaths(i) = String.format("%s/%s/*", tempBasePath, partitions(i)) + } + + // fetch all records from parquet files generated from write to hudi + val actualDf = spark.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2)) + // remove metadata columns so that expected and actual DFs can be compared as is + val trimmedDf = dropMetaFields(actualDf) + assert(df.except(trimmedDf).count() == 0) + } + + /** + * Test case for insert dataset without partitioning field + */ + @Test + def testInsertDatasetWithoutPartitionField(): Unit = { + val tableOpts = + commonTableModifier + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .updated(HoodieTableConfig.TABLE_FORMAT.key, "test-format") + + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val records = DataSourceTestUtils.generateRandomRows(1) + val recordsSeq = convertRowListToSeq(records) + val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) + + // try write to Hudi + assertThrows[HoodieException] { + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, tableOpts - DataSourceWriteOptions.PARTITIONPATH_FIELD.key, df) + } + } + + /** + * Test case for bulk insert dataset with datasource impl multiple rounds. + */ + @Test + def testBulkInsertDatasetWithDatasourceImplMultipleRounds(): Unit = { + + val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") + .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") + .updated(HoodieTableConfig.TABLE_FORMAT.key, "test-format") + val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, + HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH) + val fullPartitionPaths = new Array[String](3) + for (i <- 0 to 2) { + fullPartitionPaths(i) = String.format("%s/%s/*", tempBasePath, partitions(i)) + } + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + var totalExpectedDf = spark.createDataFrame(sc.emptyRDD[Row], structType) + for (_ <- 0 to 2) { + // generate the inserts + val records = DataSourceTestUtils.generateRandomRows(200) + val recordsSeq = convertRowListToSeq(records) + val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) + // write to Hudi + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df) + // Fetch records from entire dataset + val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2)) + // remove metadata columns so that expected and actual DFs can be compared as is + val trimmedDf = dropMetaFields(actualDf) + // find total df (union from multiple rounds) + totalExpectedDf = totalExpectedDf.union(df) + // find mismatch between actual and expected df + assert(totalExpectedDf.except(trimmedDf).count() == 0) + } + } + + /** + * Test cases for basic HoodieSparkSqlWriter functionality with datasource insert + * for different tableTypes, fileFormats and options for population meta fields. + * + * @param tableType Type of table + * @param baseFileFormat File format + * @param populateMetaFields Flag for populating meta fields + */ + @ParameterizedTest + @MethodSource(Array("testDatasourceInsert")) + def testDatasourceInsertForTableTypeBaseFileMetaFields(tableType: String, populateMetaFields: Boolean, baseFileFormat: String): Unit = { + val hoodieFooTableName = "hoodie_foo_tbl" + val fooTableModifier = Map("path" -> tempBasePath, + HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, + HoodieWriteConfig.BASE_FILE_FORMAT.key -> baseFileFormat, + DataSourceWriteOptions.TABLE_TYPE.key -> tableType, + HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "4", + DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", + HoodieTableConfig.POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields), + DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key() -> classOf[DefaultHoodieRecordPayload].getCanonicalName) + val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier) + // generate the inserts + val schema = DataSourceTestUtils.getStructTypeExampleSchema + val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) + val modifiedSchema = AvroConversionUtils.convertStructTypeToAvroSchema(structType, "trip", "example.schema") + val records = DataSourceTestUtils.generateRandomRows(100) + val recordsSeq = convertRowListToSeq(records) + val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) + initializeMetaClientForBootstrap(fooTableParams, tableType, addBootstrapPath = false, initBasePath = true) + val client = spy[SparkRDDWriteClient[_]](DataSourceUtils.createHoodieClient( + new JavaSparkContext(sc), modifiedSchema.toString, tempBasePath, hoodieFooTableName, + fooTableParams.asJava).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]) + + HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df, Option.empty, Option(client)) + // Verify that asynchronous compaction is not scheduled + verify(client, times(0)).scheduleCompaction(any()) + // Verify that HoodieWriteClient is closed correctly + verify(client, times(1)).close() + + // collect all partition paths to issue read of parquet files + val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, + HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH) + // Check the entire dataset has all records still + val fullPartitionPaths = new Array[String](3) + for (i <- fullPartitionPaths.indices) { + fullPartitionPaths(i) = String.format("%s/%s/*", tempBasePath, partitions(i)) + } + // fetch all records from parquet files generated from write to hudi + var actualDf: DataFrame = null + if (baseFileFormat.equalsIgnoreCase(HoodieFileFormat.PARQUET.name())) { + actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2)) + } else if (baseFileFormat.equalsIgnoreCase(HoodieFileFormat.ORC.name())) { + actualDf = sqlContext.read.orc(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2)) + } + // remove metadata columns so that expected and actual DFs can be compared as is + val trimmedDf = dropMetaFields(actualDf) + assert(df.except(trimmedDf).count() == 0) + } + + /** + * Test cases for HoodieSparkSqlWriter functionality with datasource bootstrap + * for different type of tables and table versions. + * + * @param tableType Type of table + * @param tableVersion Version of table + */ + @ParameterizedTest + @MethodSource(Array("bootstrapTestParams")) + def testWithDatasourceBootstrapForTableType(tableType: String, tableVersion: Int): Unit = { + val srcPath = java.nio.file.Files.createTempDirectory("hoodie_bootstrap_source_path") + try { + val sourceDF = TestBootstrap.generateTestRawTripDataset(Instant.now.toEpochMilli, 0, 100, Collections.emptyList(), sc, + spark.sqlContext) + // Write source data non-partitioned + sourceDF.write.format("parquet").mode(SaveMode.Overwrite).save(srcPath.toAbsolutePath.toString) + + val fooTableModifier = Map("path" -> tempBasePath, + HoodieBootstrapConfig.BASE_PATH.key -> srcPath.toAbsolutePath.toString, + HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, + DataSourceWriteOptions.TABLE_TYPE.key -> tableType, + HoodieBootstrapConfig.PARALLELISM_VALUE.key -> "4", + DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL, + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "", + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName, + DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key() -> classOf[DefaultHoodieRecordPayload].getCanonicalName, + "hoodie.write.table.version" -> tableVersion.toString, + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "false") + val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier) + initializeMetaClientForBootstrap(fooTableParams, tableType, addBootstrapPath = true, initBasePath = false) + + val client = spy[SparkRDDWriteClient[_]](DataSourceUtils.createHoodieClient( + new JavaSparkContext(sc), + null, + tempBasePath, + hoodieFooTableName, + fooTableParams.asJava).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]) + + HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableModifier, spark.emptyDataFrame, Option.empty, + Option.empty, Option(client)) + + // Verify that HoodieWriteClient is closed correctly + verify(client, times(1)).close() + + val ignoreResult = HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Ignore, fooTableModifier, spark.emptyDataFrame, Option.empty, + Option.empty, Option(client)) + assertFalse(ignoreResult) + verify(client, times(2)).close() + + // Assert the table version is adopted. + val metaClient = createMetaClient(spark, tempBasePath) + assertEquals(metaClient.getTableConfig.getTableVersion.versionCode(), tableVersion) + // fetch all records from parquet files generated from write to hudi + val actualDf = sqlContext.read.parquet(tempBasePath) + assert(actualDf.count == 100) + } finally { + FileUtils.deleteDirectory(srcPath.toFile) + } + } + + def initializeMetaClientForBootstrap(fooTableParams: Map[String, String], tableType: String, addBootstrapPath: Boolean, initBasePath: Boolean): Unit = { + // when metadata is enabled, directly instantiating write client using DataSourceUtils.createHoodieClient + // will hit a code which tries to instantiate meta client for data table. if table does not exist, it fails. + // hence doing an explicit instantiation here. + val tableMetaClientBuilder = HoodieTableMetaClient.newTableBuilder() + .setTableType(tableType) + .setTableName(hoodieFooTableName) + .setRecordKeyFields(fooTableParams(DataSourceWriteOptions.RECORDKEY_FIELD.key)) + .setBaseFileFormat(fooTableParams.getOrElse(HoodieWriteConfig.BASE_FILE_FORMAT.key, + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name)) + .setArchiveLogFolder(HoodieTableConfig.TIMELINE_HISTORY_PATH.defaultValue()) + .setPreCombineField(fooTableParams.getOrElse(DataSourceWriteOptions.PRECOMBINE_FIELD.key, null)) + .setPartitionFields(fooTableParams(DataSourceWriteOptions.PARTITIONPATH_FIELD.key)) + .setKeyGeneratorClassProp(fooTableParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.defaultValue())) + if (addBootstrapPath) { + tableMetaClientBuilder + .setBootstrapBasePath(fooTableParams(HoodieBootstrapConfig.BASE_PATH.key)) + } + if (initBasePath) { + tableMetaClientBuilder.initTable(HadoopFSUtils.getStorageConfWithCopy(sc.hadoopConfiguration), tempBasePath) + } + } + + @Test + def testNonpartitonedWithReuseTableConfig(): Unit = { + val _spark = spark + import _spark.implicits._ + val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + val options = Map( + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts", + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + + // case 1: When commit C1 specifies a key generator and commit C2 does not specify key generator + val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1") + + // NonpartitionedKeyGenerator is automatically inferred and used + df.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .mode(SaveMode.Overwrite).save(tablePath1) + + val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + // In first commit, we explicitly over-ride it to Nonpartitioned, where as in 2nd batch, since re-using of table configs + // come into play, no exception should be thrown even if we don't supply any key gen class. + df2.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .mode(SaveMode.Append).save(tablePath1) + } + + @Test + def testDefaultKeyGenToNonpartitoned(): Unit = { + val _spark = spark + import _spark.implicits._ + val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + val options = Map( + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt", + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + + // case 1: When commit C1 does not specify key generator and commit C2 specifies a key generator + val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1") + + df.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .mode(SaveMode.Overwrite).save(tablePath1) + + val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + // raise exception when NonpartitionedKeyGenerator is specified + val configConflictException = intercept[HoodieException] { + df2.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName) + .mode(SaveMode.Append).save(tablePath1) + } + assert(configConflictException.getMessage.contains("Config conflict")) + assert(configConflictException.getMessage.contains(s"KeyGenerator:\t${classOf[NonpartitionedKeyGenerator].getName}\t${classOf[SimpleKeyGenerator].getName}")) + } + + @Test + def testNoKeyGenToSimpleKeyGen(): Unit = { + val _spark = spark + import _spark.implicits._ + val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + val options = Map( + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt", + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + + // case 1: When commit C1 specifies a key generator and commkt C2 does not specify key generator + val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1") + + df.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .mode(SaveMode.Overwrite).save(tablePath1) + + val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + // No Exception Should be raised + try { + df2.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName) + .mode(SaveMode.Append).save(tablePath1) + } catch { + case _: Throwable => fail("Switching from no keygen to explicit SimpleKeyGenerator should not fail"); + } + } + + @Test + def testSimpleKeyGenToNoKeyGen(): Unit = { + val _spark = spark + import _spark.implicits._ + val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + val options = Map( + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt", + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + + // case 1: When commit C1 specifies a key generator and commkt C2 does not specify key generator + val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1") + + // the first write need to specify KEYGENERATOR_CLASS_NAME params + df.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName) + .mode(SaveMode.Overwrite).save(tablePath1) + + val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + // No Exception Should be raised when default keygen is used + try { + df2.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .mode(SaveMode.Append).save(tablePath1) + } catch { + case _: Throwable => fail("Switching from explicit SimpleKeyGenerator to default keygen should not fail"); + } + } + + @Test + def testGetOriginKeyGenerator(): Unit = { + // for dataframe write + val m1 = Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> classOf[ComplexKeyGenerator].getName, + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + val kg1 = HoodieWriterUtils.getOriginKeyGenerator(m1) + assertTrue(kg1 == classOf[ComplexKeyGenerator].getName) + + // for sql write + val m2 = Map( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getName, + SqlKeyGenerator.ORIGINAL_KEYGEN_CLASS_NAME -> classOf[SimpleKeyGenerator].getName, + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + val kg2 = HoodieWriterUtils.getOriginKeyGenerator(m2) + assertTrue(kg2 == classOf[SimpleKeyGenerator].getName) + } + + /** + * + * Test that you can't have consistent hashing bucket index on a COW table + * */ + @Test + def testCOWConsistentHashing(): Unit = { + val _spark = spark + import _spark.implicits._ + val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt") + val options = Map( + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts", + HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key -> "CONSISTENT_HASHING", + HoodieIndexConfig.INDEX_TYPE.key -> "BUCKET", + HoodieTableConfig.TABLE_FORMAT.key -> "test-format" + ) + + val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1") + val exc = intercept[HoodieException] { + df.write.format("hudi") + .options(options) + .option(HoodieWriteConfig.TBL_NAME.key, tableName1) + .mode(SaveMode.Overwrite).save(tablePath1) + } + assert(exc.getMessage.contains("Consistent hashing bucket index does not work with COW table. Use simple bucket index or an MOR table.")) + } + + private def fetchActualSchema(): Schema = { + val tableMetaClient = createMetaClient(spark, tempBasePath) + new TableSchemaResolver(tableMetaClient).getTableAvroSchema(false) + } +} + +object TestHoodieSparkSqlWriterWithTestFormat { + def testDatasourceInsert: java.util.stream.Stream[Arguments] = { + val scenarios = Array( + Seq("COPY_ON_WRITE", true), + Seq("COPY_ON_WRITE", false), + Seq("MERGE_ON_READ", true), + Seq("MERGE_ON_READ", false) + ) + + val parquetScenarios = scenarios.map { + _ :+ "parquet" + } + val orcScenarios = scenarios.map { + _ :+ "orc" + } + val targetScenarios = parquetScenarios ++ orcScenarios + + java.util.Arrays.stream(targetScenarios.map(as => Arguments.arguments(as.map(_.asInstanceOf[AnyRef]): _*))) + } + + def deletePartitionsWildcardTestParams(): java.util.stream.Stream[Arguments] = { + java.util.stream.Stream.of( + Arguments.arguments("*5/03/1*", Seq("2016/03/15")), + Arguments.arguments("2016/03/*", Seq("2015/03/16", "2015/03/17"))) + } + + def bootstrapTestParams(): java.util.stream.Stream[Arguments] = { + java.util.stream.Stream.of( + Arguments.arguments("MERGE_ON_READ", Integer.valueOf(8)), + Arguments.arguments("MERGE_ON_READ", Integer.valueOf(6)), + Arguments.arguments("COPY_ON_WRITE", Integer.valueOf(8)) + ) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 4bfe7ed8afb86..08b989bb3b57f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -202,8 +202,9 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup snapshot1.cache() assertEquals(300, snapshot1.count()) + metaClient = createMetaClient(spark, basePath) var partitionPaths = FSUtils.getAllPartitionPaths( - new HoodieSparkEngineContext(jsc), storage, HoodieMetadataConfig.newBuilder().build(), basePath) + new HoodieSparkEngineContext(jsc), metaClient, HoodieMetadataConfig.newBuilder().build()) assertTrue(partitionPaths.contains("100/rider-123")) assertTrue(partitionPaths.contains("200/rider-456")) @@ -226,8 +227,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup .mode(SaveMode.Overwrite) .save(basePath) - partitionPaths = FSUtils.getAllPartitionPaths( - new HoodieSparkEngineContext(jsc), storage, HoodieMetadataConfig.newBuilder().build(), basePath) + partitionPaths = FSUtils.getAllPartitionPaths(new HoodieSparkEngineContext(jsc), metaClient, HoodieMetadataConfig.newBuilder().build()) assertEquals(partitionPaths.size(), 1) assertEquals(partitionPaths.get(0), "") } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala index e7b9fc751a1d2..2e98d0b7d5918 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieMultipleBaseFileFormat.scala @@ -114,7 +114,7 @@ class TestHoodieMultipleBaseFileFormat extends HoodieSparkClientTestBase with Sp val viewManager: FileSystemViewManager = FileSystemViewManager.createViewManager( engineContext, metadataConfig, FileSystemViewStorageConfig.newBuilder.build, HoodieCommonConfig.newBuilder.build, - (mc: HoodieTableMetaClient) => HoodieTableMetadata.create(engineContext, mc.getStorage, metadataConfig, basePath)) + (mc: HoodieTableMetaClient) => metaClient.getTableFormat.getMetadataFactory.create(engineContext, mc.getStorage, metadataConfig, basePath)) val fsView: SyncableFileSystemView = viewManager.getFileSystemView(metaClient) val orcFiles = fsView.getAllBaseFiles(DEFAULT_SECOND_PARTITION_PATH).filter(bf => bf.getFileName.endsWith("orc")) assertTrue(orcFiles.count() > 0) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala index 5fdc2a5b0f34f..15b95960097da 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala @@ -401,7 +401,7 @@ object HoodieSparkSqlTestBase { engineContext, metadataConfig, FileSystemViewStorageConfig.newBuilder.build, HoodieCommonConfig.newBuilder.build, (_: HoodieTableMetaClient) => { - HoodieTableMetadata.create( + metaClient.getTableFormat.getMetadataFactory.create( engineContext, metaClient.getStorage, metadataConfig, metaClient.getBasePath.toString) } ) diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java index f453bf856660a..e6fee456585f2 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java @@ -123,8 +123,7 @@ public MessageType getStorageSchema(boolean includeMetadataField) { public List getAllPartitionPathsOnStorage() { HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getStorageConf()); return FSUtils.getAllPartitionPaths(engineContext, - metaClient.getStorage(), - config.getString(META_SYNC_BASE_PATH), + metaClient, config.getBoolean(META_SYNC_USE_FILE_LISTING_FROM_METADATA)); } diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileWriter.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileWriter.java index 773eae1db97b8..07533ee900879 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileWriter.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileWriter.java @@ -104,7 +104,7 @@ public StoragePath getManifestFolder(boolean useAbsolutePath) { @VisibleForTesting static Stream getLatestBaseFiles(boolean canUseMetadataTable, HoodieEngineContext engContext, HoodieTableMetaClient metaClient, boolean useAbsolutePath) { - List partitions = FSUtils.getAllPartitionPaths(engContext, metaClient.getStorage(), metaClient.getBasePath(), canUseMetadataTable); + List partitions = FSUtils.getAllPartitionPaths(engContext, metaClient, canUseMetadataTable); LOG.info("Retrieve all partitions: {}", partitions.size()); HoodieTableFileSystemView fsView = null; try { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index 1308687079a14..7dca20c9666fa 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -734,10 +734,9 @@ private boolean checkMetadataTableIsAvailable() { List validatePartitions(HoodieSparkEngineContext engineContext, StoragePath basePath, HoodieTableMetaClient metaClient) { // compare partitions HoodieTimeline completedTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); - List allPartitionPathsFromFS = getPartitionsFromFileSystem(engineContext, basePath, metaClient.getStorage(), - completedTimeline); + List allPartitionPathsFromFS = getPartitionsFromFileSystem(engineContext, metaClient, completedTimeline); - List allPartitionPathsMeta = getPartitionsFromMDT(engineContext, basePath, metaClient.getStorage()); + List allPartitionPathsMeta = getPartitionsFromMDT(engineContext, metaClient); Collections.sort(allPartitionPathsFromFS); Collections.sort(allPartitionPathsMeta); @@ -801,20 +800,18 @@ Option getPartitionCreationInstant(HoodieStorage storage, StoragePath ba } @VisibleForTesting - List getPartitionsFromMDT(HoodieEngineContext engineContext, StoragePath basePath, - HoodieStorage storage) { - return FSUtils.getAllPartitionPaths(engineContext, storage, basePath, true); + List getPartitionsFromMDT(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient) { + return FSUtils.getAllPartitionPaths(engineContext, metaClient, true); } @VisibleForTesting - List getPartitionsFromFileSystem(HoodieEngineContext engineContext, StoragePath basePath, - HoodieStorage storage, HoodieTimeline completedTimeline) { - List allPartitionPathsFromFS = FSUtils.getAllPartitionPaths(engineContext, storage, basePath, false); + List getPartitionsFromFileSystem(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, HoodieTimeline completedTimeline) { + List allPartitionPathsFromFS = FSUtils.getAllPartitionPaths(engineContext, metaClient, false); // ignore partitions created by uncommitted ingestion. return allPartitionPathsFromFS.stream().parallel().filter(part -> { HoodiePartitionMetadata hoodiePartitionMetadata = - new HoodiePartitionMetadata(storage, FSUtils.constructAbsolutePath(basePath, part)); + new HoodiePartitionMetadata(metaClient.getStorage(), FSUtils.constructAbsolutePath(metaClient.getBasePath().toString(), part)); Option instantOption = hoodiePartitionMetadata.readPartitionCreatedCommitTime(); if (instantOption.isPresent()) { String instantTime = instantOption.get(); @@ -1758,7 +1755,7 @@ public HoodieMetadataValidationContext( FileSystemViewStorageConfig viewConf = FileSystemViewStorageConfig.newBuilder().fromProperties(props).build(); ValidationUtils.checkArgument(viewConf.getStorageType().name().equals(viewStorageType), "View storage type not reflected"); HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(props).build(); - this.tableMetadata = HoodieTableMetadata.create( + this.tableMetadata = metaClient.getTableFormat().getMetadataFactory().create( engineContext, metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString()); this.fileSystemView = getFileSystemView(engineContext, metaClient, metadataConfig, viewConf, commonConfig); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index 0a09e6783417c..d42e0cccf866a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -184,7 +184,8 @@ private Option getLatestCommitTimestamp(HoodieTableMetaClient tableMetad private List getPartitions(HoodieEngineContext engineContext, Config cfg, HoodieStorage storage) { - return FSUtils.getAllPartitionPaths(engineContext, storage, cfg.sourceBasePath, true); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(cfg.sourceBasePath).setConf(storage.getConf()).build(); + return FSUtils.getAllPartitionPaths(engineContext, metaClient, true); } private void createSuccessTag(FileSystem fs, Config cfg) throws IOException { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/TableSizeStats.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/TableSizeStats.java index 90267c8c0b8d8..5ccc7ac7b1289 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/TableSizeStats.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/TableSizeStats.java @@ -278,7 +278,10 @@ private void logTableStats(String basePath, LocalDate[] dateInterval) throws IOE .build(); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); StorageConfiguration storageConf = HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration()); - HoodieTableMetadata tableMetadata = HoodieTableMetadata.create( + HoodieTableMetaClient metaClientLocal = HoodieTableMetaClient.builder() + .setBasePath(basePath) + .setConf(storageConf.newInstance()).build(); + HoodieTableMetadata tableMetadata = metaClientLocal.getTableFormat().getMetadataFactory().create( engineContext, new HoodieHadoopStorage(basePath, storageConf), metadataConfig, basePath); List allPartitions = tableMetadata.getAllPartitionPaths(); @@ -313,9 +316,6 @@ private void logTableStats(String basePath, LocalDate[] dateInterval) throws IOE || (endDate == null && (partitionDate.isEqual(startDate) || partitionDate.isAfter(startDate))) || (startDate == null && partitionDate.isBefore(endDate)) || (startDate != null && endDate != null && ((partitionDate.isEqual(startDate) || partitionDate.isAfter(startDate)) && partitionDate.isBefore(endDate)))) { - HoodieTableMetaClient metaClientLocal = HoodieTableMetaClient.builder() - .setBasePath(basePath) - .setConf(storageConf.newInstance()).build(); HoodieMetadataConfig metadataConfig1 = HoodieMetadataConfig.newBuilder() .enable(false) .build(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java index b1043efb363e1..59c9d9a8b2e7c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java @@ -32,7 +32,6 @@ import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hudi.timeline.service.TimelineService; import org.apache.hudi.utilities.UtilHelpers; @@ -95,12 +94,6 @@ private void setHostAddrFromSparkConf(SparkConf sparkConf) { public void run() throws IOException { JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - List allPartitionPaths = FSUtils.getAllPartitionPaths( - engineContext, new HoodieHadoopStorage(cfg.basePath, engineContext.getStorageConf()), - cfg.basePath, cfg.useFileListingFromMetadata); - Collections.shuffle(allPartitionPaths); - List selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions) - .collect(Collectors.toList()); if (!useExternalTimelineServer) { this.timelineServer.startService(); @@ -114,6 +107,11 @@ engineContext, new HoodieHadoopStorage(cfg.basePath, engineContext.getStorageCon .setConf(timelineServer.getStorageConf().newInstance()) .setBasePath(cfg.basePath) .setLoadActiveTimelineOnLoad(true).build(); + + List allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient, cfg.useFileListingFromMetadata); + Collections.shuffle(allPartitionPaths); + List selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions) + .collect(Collectors.toList()); SyncableFileSystemView fsView = new RemoteHoodieTableFileSystemView(this.hostAddr, cfg.serverPort, metaClient); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java index efd2bed0d1bb9..65261be899ace 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java @@ -36,6 +36,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.commit.BaseDatasetBulkInsertCommitActionExecutor; import org.apache.hudi.commit.HoodieStreamerDatasetBulkInsertCommitActionExecutor; +import org.apache.hudi.common.NativeTableFormat; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.HoodieTimeGeneratorConfig; @@ -456,6 +457,7 @@ HoodieTableMetaClient initializeEmptyTable(HoodieTableMetaClient.TableBuilder ta Boolean.parseBoolean(HIVE_STYLE_PARTITIONING_ENABLE.defaultValue()))) .setUrlEncodePartitioning(props.getBoolean(URL_ENCODE_PARTITIONING.key(), Boolean.parseBoolean(URL_ENCODE_PARTITIONING.defaultValue()))) + .setTableFormat(props.getProperty(HoodieTableConfig.TABLE_FORMAT.key(), NativeTableFormat.TABLE_FORMAT)) .initTable(storageConf, cfg.targetBasePath); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java index 5d578a73dce84..c698070d4fc70 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java @@ -1231,12 +1231,12 @@ void setPartitionCreationTime(Option partitionCreationTime) { } @Override - List getPartitionsFromFileSystem(HoodieEngineContext engineContext, StoragePath basePath, HoodieStorage storage, HoodieTimeline completedTimeline) { + List getPartitionsFromFileSystem(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, HoodieTimeline completedTimeline) { return fsPartitionsToReturn; } @Override - List getPartitionsFromMDT(HoodieEngineContext engineContext, StoragePath basePath, HoodieStorage storage) { + List getPartitionsFromMDT(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient) { return metadataPartitionsToReturn; } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index cfbd4170e1781..49252565d94a6 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -3197,8 +3197,7 @@ public void testBulkInsertWithUserDefinedPartitioner() throws Exception { syncAndAssertRecordCount(cfg, 1000, tableBasePath, "00000", 1); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(HoodieTestUtils.getDefaultStorageConf()).build(); - List partitions = FSUtils.getAllPartitionPaths( - new HoodieLocalEngineContext(metaClient.getStorageConf()), metaClient.getStorage(), metaClient.getBasePath(), false); + List partitions = FSUtils.getAllPartitionPaths(new HoodieLocalEngineContext(metaClient.getStorageConf()), metaClient, false); StorageConfiguration hadoopConf = metaClient.getStorageConf(); HoodieLocalEngineContext engContext = new HoodieLocalEngineContext(hadoopConf); HoodieTableFileSystemView fsView = HoodieTableFileSystemView.fileListingBasedFileSystemView(engContext, metaClient, From 4f3fabf9ca73900c4ac0572e1092d668402c3939 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Wed, 25 Jun 2025 16:45:05 -0700 Subject: [PATCH 02/11] Rebasing --- .../java/org/apache/hudi/client/WriteClientTestUtils.java | 4 ++-- .../common/table/timeline/HoodieInstantTimeGenerator.java | 1 - .../common/table/timeline/versioning/v2/ActiveTimelineV2.java | 4 ++-- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java index c3dfc2ea1d415..547484cf71413 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java @@ -57,8 +57,8 @@ public long generateTime(boolean skipLocking) { } @Override - public void consumeTime(boolean skipLocking, Consumer func) { - + public long consumeTime(boolean skipLocking, Consumer func) { + return 0; } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index 53a1f43c45407..6224b5a921354 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -32,7 +32,6 @@ import java.time.temporal.ChronoField; import java.time.temporal.TemporalAccessor; import java.util.Date; -import java.util.TimeZone; import java.util.concurrent.atomic.AtomicReference; import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java index 02fa180722d15..f2b1ee555f36d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java @@ -562,7 +562,7 @@ protected HoodieInstant transitionStateToComplete(boolean shouldLock, Hoodie ValidationUtils.checkArgument( metaClient.getStorage().exists(getInstantFileNamePath(fromInstantFileName)), "File " + getInstantFileNamePath(fromInstantFileName) + " does not exist!"); - String completionTime = HoodieInstantTimeGenerator.formatDateUTC(new Date(createCompleteFileInMetaPath(shouldLock, toInstant, metadata))); + String completionTime = HoodieInstantTimeGenerator.formatDateBasedOnTimeZone(new Date(createCompleteFileInMetaPath(shouldLock, toInstant, metadata))); return new HoodieInstant(toInstant.getState(), toInstant.getAction(), toInstant.requestedTime(), completionTime, instantComparator.requestedTimeOrderedComparator()); } } catch (IOException e) { @@ -746,7 +746,7 @@ protected long createCompleteFileInMetaPath(boolean shouldLock, HoodieInstan Option writerOption = getHoodieInstantWriterOption(this, metadata); TimeGenerator timeGenerator = TimeGenerators .getTimeGenerator(metaClient.getTimeGeneratorConfig(), metaClient.getStorageConf()); - timeGenerator.consumeTime(!shouldLock, currentTimeMillis -> { + return timeGenerator.consumeTime(!shouldLock, currentTimeMillis -> { String completionTime = HoodieInstantTimeGenerator.formatDateBasedOnTimeZone(new Date(currentTimeMillis)); String fileName = instantFileNameGenerator.getFileName(completionTime, instant); StoragePath fullPath = getInstantFileNamePath(fileName); From da763c885b2191c953ac3fbbc7c28377db1be883 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Wed, 25 Jun 2025 17:51:15 -0700 Subject: [PATCH 03/11] Wiring in the property in few more places --- .../apache/hudi/common/table/HoodieTableMetaClient.java | 8 ++++++-- .../java/org/apache/hudi/configuration/FlinkOptions.java | 7 +++++++ .../src/main/java/org/apache/hudi/util/StreamerUtil.java | 1 + .../connect/writers/KafkaConnectTransactionServices.java | 2 ++ .../main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala | 7 ++++++- .../spark/sql/catalyst/catalog/HoodieCatalogTable.scala | 1 + .../java/org/apache/hudi/cli/BootstrapExecutorUtils.java | 2 ++ .../apache/hudi/utilities/streamer/BootstrapExecutor.java | 1 + .../org/apache/hudi/utilities/streamer/StreamSync.java | 3 +-- 9 files changed, 27 insertions(+), 5 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 8a7f918272ca4..3426217b714cc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -87,7 +87,6 @@ import static org.apache.hudi.common.table.HoodieTableConfig.PAYLOAD_CLASS_NAME; import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGE_MODE; import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGE_STRATEGY_ID; -import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_FORMAT; import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_PATH; import static org.apache.hudi.common.table.HoodieTableConfig.VERSION; import static org.apache.hudi.common.util.ConfigUtils.containsConfigProperty; @@ -1275,6 +1274,7 @@ public TableBuilder fromMetaClient(HoodieTableMetaClient metaClient) { return setTableType(metaClient.getTableType()) .setTableName(metaClient.getTableConfig().getTableName()) .setTableVersion(metaClient.getTableConfig().getTableVersion()) + .setTableFormat(metaClient.getTableConfig().getTableFormat(metaClient.getTimelineLayoutVersion()).getName()) .setTimelinePath(metaClient.getTableConfig().getTimelinePath()) .setArchiveLogFolder(metaClient.getTableConfig().getTimelineHistoryPath()) .setRecordMergeMode(metaClient.getTableConfig().getRecordMergeMode()) @@ -1305,6 +1305,10 @@ public TableBuilder fromProperties(Properties properties) { setTableVersion(hoodieConfig.getInt(VERSION)); } + if (hoodieConfig.contains(HoodieTableConfig.TABLE_FORMAT)) { + setTableFormat(hoodieConfig.getString(HoodieTableConfig.TABLE_FORMAT)); + } + if (hoodieConfig.contains(TIMELINE_PATH)) { setTimelinePath(hoodieConfig.getString(TIMELINE_PATH)); } @@ -1541,7 +1545,7 @@ public Properties build() { tableConfig.setValue(HoodieTableConfig.RELATIVE_INDEX_DEFINITION_PATH, indexDefinitionPath); } if (null != tableFormat) { - tableConfig.setValue(TABLE_FORMAT, tableFormat); + tableConfig.setValue(HoodieTableConfig.TABLE_FORMAT, tableFormat); } return tableConfig.getProps(); } 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 d191bc9a82507..4b5287280530a 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 @@ -420,6 +420,13 @@ private FlinkOptions() { .defaultValue(HoodieWriteConfig.WRITE_TABLE_VERSION.defaultValue()) .withDescription("Table version produced by this writer."); + @AdvancedConfig + public static final ConfigOption WRITE_TABLE_FORMAT = ConfigOptions + .key(HoodieTableConfig.TABLE_FORMAT.key()) + .stringType() + .defaultValue(HoodieTableConfig.TABLE_FORMAT.defaultValue()) + .withDescription("Table format produced by this writer."); + /** * Flag to indicate whether to drop duplicates before insert/upsert. * By default false to gain extra performance. diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 0b8ac3c7bff35..00ccc42ac2e13 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 @@ -274,6 +274,7 @@ public static HoodieTableMetaClient initTableIfNotExists( .setTableType(conf.getString(FlinkOptions.TABLE_TYPE)) .setTableName(conf.getString(FlinkOptions.TABLE_NAME)) .setTableVersion(conf.getInteger(FlinkOptions.WRITE_TABLE_VERSION)) + .setTableFormat(conf.getString(FlinkOptions.WRITE_TABLE_FORMAT)) .setRecordMergeMode(getMergeMode(conf)) .setRecordMergeStrategyId(getMergeStrategyId(conf)) .setPayloadClassName(getPayloadClass(conf)) diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java index 30deffa77bcca..c7a4d6314c59b 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -100,6 +101,7 @@ public KafkaConnectTransactionServices(KafkaConnectConfigs connectConfigs) throw .setRecordKeyFields(recordKeyFields) .setPartitionFields(partitionColumns) .setTableVersion(writeConfig.getWriteVersion()) + .setTableFormat(connectConfigs.getStringOrDefault(HoodieTableConfig.TABLE_FORMAT)) .setKeyGeneratorClassProp(writeConfig.getKeyGeneratorClass()) .fromProperties(connectConfigs.getProps()) .initTable(storageConf.newInstance(), tableBasePath)); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 13a43fb1f8c98..383f825f712e3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -299,9 +299,11 @@ class HoodieSparkSqlWriterInternal { if (StringUtils.nonEmpty(hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME))) hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME) else KeyGeneratorType.getKeyGeneratorClassName(hoodieConfig) + val tableFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.TABLE_FORMAT) HoodieTableMetaClient.newTableBuilder() .setTableType(tableType) .setTableVersion(tableVersion) + .setTableFormat(tableFormat) .setDatabaseName(databaseName) .setTableName(tblName) .setBaseFileFormat(baseFileFormat) @@ -429,7 +431,8 @@ class HoodieSparkSqlWriterInternal { val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(TypedProperties.copy(hoodieConfig.getProps)) val tableMetaClient = HoodieTableMetaClient.builder .setConf(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration)) - .setBasePath(basePath.toString).build() + .setBasePath(basePath.toString) + .build() // Get list of partitions to delete val partitionsToDelete = if (parameters.contains(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key())) { val partitionColsToDelete = parameters(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key()).split(",") @@ -757,12 +760,14 @@ class HoodieSparkSqlWriterInternal { HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(), String.valueOf(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue()) )) + val tableFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.TABLE_FORMAT) HoodieTableMetaClient.newTableBuilder() .setTableType(HoodieTableType.valueOf(tableType)) .setTableName(tableName) .setRecordKeyFields(recordKeyFields) .setTableVersion(tableVersion) + .setTableFormat(tableFormat) .setArchiveLogFolder(archiveLogFolder) .setPayloadClassName(payloadClass) .setRecordMergeMode(RecordMergeMode.getValue(hoodieConfig.getString(HoodieWriteConfig.RECORD_MERGE_MODE))) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index 03871490f8a14..e4ffcac2c5429 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -231,6 +231,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten HoodieTableMetaClient.newTableBuilder() .fromProperties(properties) .setTableVersion(Integer.valueOf(getStringWithAltKeys(tableConfigs, HoodieWriteConfig.WRITE_TABLE_VERSION))) + .setTableFormat(getStringWithAltKeys(tableConfigs, HoodieTableConfig.TABLE_FORMAT)) .setDatabaseName(catalogDatabaseName) .setTableName(table.identifier.table) .setTableCreateSchema(schema.toString()) diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java index 3edeac283cbba..325e8c6a5019e 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java @@ -244,6 +244,8 @@ private void initializeTable() throws IOException { TIMELINE_HISTORY_PATH.key(), TIMELINE_HISTORY_PATH.defaultValue())) .setPayloadClassName(cfg.payloadClass) .setBaseFileFormat(cfg.baseFileFormat) + .setTableFormat(props.getString(HoodieTableConfig.TABLE_FORMAT.key(), + HoodieTableConfig.TABLE_FORMAT.defaultValue())) .setBootstrapIndexClass(cfg.bootstrapIndexClass) .setBootstrapBasePath(bootstrapBasePath) .setCDCEnabled(props.getBoolean(HoodieTableConfig.CDC_ENABLED.key(), diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java index ae697dff17303..749be2e83068a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/BootstrapExecutor.java @@ -209,6 +209,7 @@ private void initializeTable() throws IOException { .setRecordKeyFields(props.getString(RECORDKEY_FIELD_NAME.key())) .setPreCombineField(props.getString(PRECOMBINE_FIELD_NAME.key(), null)) .setTableVersion(ConfigUtils.getIntWithAltKeys(props, WRITE_TABLE_VERSION)) + .setTableFormat(props.getString(HoodieTableConfig.TABLE_FORMAT.key(), HoodieTableConfig.TABLE_FORMAT.defaultValue())) .setPopulateMetaFields(props.getBoolean( POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue())) .setArchiveLogFolder(props.getString( diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java index 65261be899ace..32e6945d3116a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java @@ -36,7 +36,6 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.commit.BaseDatasetBulkInsertCommitActionExecutor; import org.apache.hudi.commit.HoodieStreamerDatasetBulkInsertCommitActionExecutor; -import org.apache.hudi.common.NativeTableFormat; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.HoodieTimeGeneratorConfig; @@ -457,7 +456,7 @@ HoodieTableMetaClient initializeEmptyTable(HoodieTableMetaClient.TableBuilder ta Boolean.parseBoolean(HIVE_STYLE_PARTITIONING_ENABLE.defaultValue()))) .setUrlEncodePartitioning(props.getBoolean(URL_ENCODE_PARTITIONING.key(), Boolean.parseBoolean(URL_ENCODE_PARTITIONING.defaultValue()))) - .setTableFormat(props.getProperty(HoodieTableConfig.TABLE_FORMAT.key(), NativeTableFormat.TABLE_FORMAT)) + .setTableFormat(props.getProperty(HoodieTableConfig.TABLE_FORMAT.key(), HoodieTableConfig.TABLE_FORMAT.defaultValue())) .initTable(storageConf, cfg.targetBasePath); } From daa22ca0253f446342b6bdcb848c422e2928c67b Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Thu, 26 Jun 2025 15:10:26 -0700 Subject: [PATCH 04/11] Fix test failure --- .../apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala index 0ed710c9d0df5..53c9d49614fc1 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala @@ -144,6 +144,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase { |[hoodie.record.merge.strategy.id,eeb8d96f-b1e4-49fd-bbf8-28ac514178e5,null] |[hoodie.table.checksum,,] |[hoodie.table.create.schema,,] + |[hoodie.table.format,native,null] |[hoodie.table.initial.version,8,8] |[hoodie.table.keygenerator.type,NON_PARTITION,null] |[hoodie.table.name,,] From fb0e50b62abba08e53379e3294829c63fa3fc838 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Fri, 27 Jun 2025 07:00:12 -0700 Subject: [PATCH 05/11] Renaming the interface to HoodieTableFormat --- ...dieJavaClientOnCopyOnWriteStorageForTestFormat.java | 10 ++++------ ...Format => org.apache.hudi.common.HoodieTableFormat} | 0 .../{TableFormat.java => HoodieTableFormat.java} | 2 +- .../java/org/apache/hudi/common/NativeTableFormat.java | 2 +- .../apache/hudi/common/table/HoodieTableConfig.java | 8 ++++---- .../hudi/common/table/HoodieTableMetaClient.java | 6 +++--- .../org/apache/hudi/tableformat/TestTableFormat.java | 8 ++++---- ...Format => org.apache.hudi.common.HoodieTableFormat} | 0 8 files changed, 17 insertions(+), 19 deletions(-) rename hudi-client/hudi-java-client/src/test/resources/META-INF/services/{org.apache.hudi.common.TableFormat => org.apache.hudi.common.HoodieTableFormat} (100%) rename hudi-common/src/main/java/org/apache/hudi/common/{TableFormat.java => HoodieTableFormat.java} (98%) rename hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/{org.apache.hudi.common.TableFormat => org.apache.hudi.common.HoodieTableFormat} (100%) diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java index 7ee8c427050a9..4d3c8326c2b93 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorageForTestFormat.java @@ -19,7 +19,6 @@ package org.apache.hudi.client.functional; import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -36,11 +35,10 @@ protected void initMetaClient() throws IOException { if (basePath == null) { initPath(); } - TypedProperties properties = new TypedProperties(); - properties.put(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "false"); - properties.put(HoodieTableConfig.TABLE_FORMAT.key(), "test-format"); - properties.put(HoodieMetadataConfig.ENABLE.key(), "false"); - metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, properties); + storageConf.set(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), "false"); + storageConf.set(HoodieTableConfig.TABLE_FORMAT.key(), "test-format"); + storageConf.set(HoodieMetadataConfig.ENABLE.key(), "false"); + metaClient = HoodieTestUtils.init(storageConf, basePath, HoodieTableType.COPY_ON_WRITE); } @AfterAll diff --git a/hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat b/hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.HoodieTableFormat similarity index 100% rename from hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat rename to hudi-client/hudi-java-client/src/test/resources/META-INF/services/org.apache.hudi.common.HoodieTableFormat diff --git a/hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java similarity index 98% rename from hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java rename to hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java index 17fcf8233d310..b01ca22bacb76 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/TableFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java @@ -34,7 +34,7 @@ /** * External Table Format needs to implement this interface */ -public interface TableFormat extends Serializable { +public interface HoodieTableFormat extends Serializable { /** * Returns the name of the table format. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java index ecc13612a71eb..6c57ed0ab1677 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/NativeTableFormat.java @@ -24,7 +24,7 @@ import org.apache.hudi.metadata.NativeTableMetadataFactory; import org.apache.hudi.metadata.TableMetadataFactory; -public class NativeTableFormat implements TableFormat { +public class NativeTableFormat implements HoodieTableFormat { public static final String TABLE_FORMAT = "native"; private final TimelineLayoutVersion timelineLayoutVersion; 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 4da5603298062..6cd6f1b1b230d 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 @@ -18,7 +18,7 @@ package org.apache.hudi.common.table; -import org.apache.hudi.common.TableFormat; +import org.apache.hudi.common.HoodieTableFormat; import org.apache.hudi.common.NativeTableFormat; import org.apache.hudi.common.bootstrap.index.hfile.HFileBootstrapIndex; import org.apache.hudi.common.config.ConfigClassProperty; @@ -728,11 +728,11 @@ public Option getTimelineLayoutVersion() { : Option.empty(); } - public TableFormat getTableFormat(TimelineLayoutVersion layoutVersion) { + public HoodieTableFormat getTableFormat(TimelineLayoutVersion layoutVersion) { String tableFormat = getStringOrDefault(TABLE_FORMAT); if (!tableFormat.equals(NativeTableFormat.TABLE_FORMAT)) { - ServiceLoader loader = ServiceLoader.load(TableFormat.class); - for (TableFormat tableFormatImpl : loader) { + ServiceLoader loader = ServiceLoader.load(HoodieTableFormat.class); + for (HoodieTableFormat tableFormatImpl : loader) { if (getString(TABLE_FORMAT).equals(tableFormatImpl.getName())) { tableFormatImpl.init(props); return tableFormatImpl; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 3426217b714cc..b527a292dab39 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -18,8 +18,8 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.HoodieTableFormat; import org.apache.hudi.common.NativeTableFormat; -import org.apache.hudi.common.TableFormat; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetaserverConfig; @@ -167,7 +167,7 @@ public class HoodieTableMetaClient implements Serializable { protected HoodieMetaserverConfig metaserverConfig; private HoodieTimeGeneratorConfig timeGeneratorConfig; private Option indexMetadataOpt = Option.empty(); - private TableFormat tableFormat; + private HoodieTableFormat tableFormat; /** * Instantiate HoodieTableMetaClient. @@ -361,7 +361,7 @@ public StoragePath getTimelinePath() { return timelinePath; } - public TableFormat getTableFormat() { + public HoodieTableFormat getTableFormat() { return tableFormat; } diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java index 7f7f489483409..e8a16a6b56092 100644 --- a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java @@ -19,7 +19,7 @@ package org.apache.hudi.tableformat; import org.apache.hudi.avro.model.HoodieCleanMetadata; -import org.apache.hudi.common.TableFormat; +import org.apache.hudi.common.HoodieTableFormat; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -35,10 +35,10 @@ import java.util.concurrent.CopyOnWriteArrayList; /** - * Test implementation of TableFormat that records all Hoodie instants in memory. - * Used for functional testing of TableFormat. + * Test implementation of HoodieTableFormat that records all Hoodie instants in memory. + * Used for functional testing of HoodieTableFormat. */ -public class TestTableFormat implements TableFormat { +public class TestTableFormat implements HoodieTableFormat { private static final Map> RECORDED_INSTANTS = new ConcurrentHashMap<>(); diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat b/hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.HoodieTableFormat similarity index 100% rename from hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.TableFormat rename to hudi-spark-datasource/hudi-spark/src/test/resources/META-INF/services/org.apache.hudi.common.HoodieTableFormat From c23dcbd291427581a9e680b20d35339e18c477ac Mon Sep 17 00:00:00 2001 From: danny0405 Date: Sun, 29 Jun 2025 09:40:47 +0800 Subject: [PATCH 06/11] fix meta client creation in file index --- .../org/apache/hudi/source/FileIndex.java | 21 ++++++++++++------- .../hudi/source/IncrementalInputSplits.java | 9 ++++---- .../apache/hudi/table/HoodieTableSource.java | 12 +++++------ .../org/apache/hudi/source/TestFileIndex.java | 10 ++++++--- 4 files changed, 31 insertions(+), 21 deletions(-) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java index 82a2830bab202..e25723966d223 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java @@ -22,10 +22,8 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; -import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.index.bucket.BucketIdentifier; import org.apache.hudi.source.prune.ColumnStatsProbe; import org.apache.hudi.source.prune.PartitionPruners; @@ -71,13 +69,14 @@ public class FileIndex implements Serializable { private final ColumnStatsProbe colStatsProbe; // for probing column stats private final Function partitionBucketIdFunc; // for bucket pruning private List partitionPaths; // cache of partition paths - private final FileStatsIndex fileStatsIndex;// for data skipping - private final Option metaClient; + private final FileStatsIndex fileStatsIndex; // for data skipping + private final HoodieTableMetaClient metaClient; private FileIndex( StoragePath path, Configuration conf, RowType rowType, + HoodieTableMetaClient metaClient, ColumnStatsProbe colStatsProbe, PartitionPruners.PartitionPruner partitionPruner, Function partitionBucketIdFunc) { @@ -89,7 +88,7 @@ private FileIndex( this.partitionPruner = partitionPruner; this.fileStatsIndex = new FileStatsIndex(path.toString(), rowType, metadataConfig); this.partitionBucketIdFunc = partitionBucketIdFunc; - this.metaClient = tableExists ? Option.of(HoodieTableMetaClient.builder().setBasePath(path.toString()).setConf(HadoopFSUtils.getStorageConf(hadoopConf)).build()) : Option.empty(); + this.metaClient = metaClient; } /** @@ -161,7 +160,7 @@ public List getFilesInPartitions() { return Collections.emptyList(); } Map> filesInPartitions = FSUtils.getFilesInPartitions( - new HoodieFlinkEngineContext(hadoopConf), metaClient.get(), metadataConfig, partitions); + new HoodieFlinkEngineContext(hadoopConf), metaClient, metadataConfig, partitions); int totalFilesNum = filesInPartitions.values().stream().mapToInt(List::size).sum(); if (totalFilesNum < 1) { // returns early for empty table. @@ -234,7 +233,7 @@ public List getOrBuildPartitionPaths() { if (this.partitionPaths != null) { return this.partitionPaths; } - List allPartitionPaths = this.tableExists ? FSUtils.getAllPartitionPaths(new HoodieFlinkEngineContext(hadoopConf), metaClient.get(), metadataConfig) + List allPartitionPaths = this.tableExists ? FSUtils.getAllPartitionPaths(new HoodieFlinkEngineContext(hadoopConf), metaClient, metadataConfig) : Collections.emptyList(); if (this.partitionPruner == null) { this.partitionPaths = allPartitionPaths; @@ -288,6 +287,7 @@ public static class Builder { private StoragePath path; private Configuration conf; private RowType rowType; + private HoodieTableMetaClient metaClient; private ColumnStatsProbe columnStatsProbe; private PartitionPruners.PartitionPruner partitionPruner; private Function partitionBucketIdFunc; @@ -310,6 +310,11 @@ public Builder rowType(RowType rowType) { return this; } + public Builder metaClient(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + return this; + } + public Builder columnStatsProbe(ColumnStatsProbe columnStatsProbe) { this.columnStatsProbe = columnStatsProbe; return this; @@ -327,7 +332,7 @@ public Builder partitionBucketIdFunc(Function partitionBucketId public FileIndex build() { return new FileIndex(Objects.requireNonNull(path), Objects.requireNonNull(conf), Objects.requireNonNull(rowType), - columnStatsProbe, partitionPruner, partitionBucketIdFunc); + metaClient, columnStatsProbe, partitionPruner, partitionBucketIdFunc); } } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java index 60bef1f3a9294..80256c6dcd817 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java @@ -177,7 +177,7 @@ public Result inputSplits( final List fileInfoList; if (fullTableScan) { // scans the partitions and files directly. - FileIndex fileIndex = getFileIndex(); + FileIndex fileIndex = getFileIndex(metaClient); readPartitions = new TreeSet<>(fileIndex.getOrBuildPartitionPaths()); if (readPartitions.size() == 0) { LOG.warn("No partitions found for reading in user provided path."); @@ -208,7 +208,7 @@ public Result inputSplits( LOG.warn("Found deleted files in metadata, fall back to full table scan."); // fallback to full table scan // reading from the earliest, scans the partitions and files directly. - FileIndex fileIndex = getFileIndex(); + FileIndex fileIndex = getFileIndex(metaClient); readPartitions = new TreeSet<>(fileIndex.getOrBuildPartitionPaths()); if (readPartitions.size() == 0) { LOG.warn("No partitions found for reading in user provided path."); @@ -275,7 +275,7 @@ public Result inputSplits( if (instantRange.isEmpty()) { // reading from the earliest, scans the partitions and files directly. - FileIndex fileIndex = getFileIndex(); + FileIndex fileIndex = getFileIndex(metaClient); Set readPartitions = new TreeSet<>(fileIndex.getOrBuildPartitionPaths()); if (readPartitions.size() == 0) { @@ -411,11 +411,12 @@ private static Stream getFileSlices( : fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, endInstant); } - private FileIndex getFileIndex() { + private FileIndex getFileIndex(HoodieTableMetaClient metaClient) { return FileIndex.builder() .path(new StoragePath(path.toUri())) .conf(conf) .rowType(rowType) + .metaClient(metaClient) .partitionPruner(partitionPruner) .build(); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index 5b944682f22a3..2179e72321706 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -113,7 +113,6 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.StringJoiner; import java.util.concurrent.atomic.AtomicInteger; @@ -191,16 +190,16 @@ public HoodieTableSource( this.partitionKeys = partitionKeys; this.defaultPartName = defaultPartName; this.conf = conf; - this.predicates = Optional.ofNullable(predicates).orElse(Collections.emptyList()); + this.predicates = Option.ofNullable(predicates).orElse(Collections.emptyList()); this.columnStatsProbe = columnStatsProbe; this.partitionPruner = partitionPruner; this.dataBucketFunc = dataBucketFunc; - this.requiredPos = Optional.ofNullable(requiredPos).orElseGet(() -> IntStream.range(0, this.tableRowType.getFieldCount()).toArray()); - this.limit = Optional.ofNullable(limit).orElse(NO_LIMIT_CONSTANT); + this.requiredPos = Option.ofNullable(requiredPos).orElseGet(() -> IntStream.range(0, this.tableRowType.getFieldCount()).toArray()); + this.limit = Option.ofNullable(limit).orElse(NO_LIMIT_CONSTANT); this.hadoopConf = new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(conf)); - this.metaClient = Optional.ofNullable(metaClient).orElseGet(() -> StreamerUtil.metaClientForReader(conf, this.hadoopConf.unwrap())); + this.metaClient = Option.ofNullable(metaClient).orElseGet(() -> StreamerUtil.metaClientForReader(conf, this.hadoopConf.unwrap())); this.maxCompactionMemoryInBytes = StreamerUtil.getMaxCompactionMemoryInBytes(conf); - this.internalSchemaManager = Optional.ofNullable(internalSchemaManager).orElseGet(() -> InternalSchemaManager.get(this.conf, this.metaClient)); + this.internalSchemaManager = Option.ofNullable(internalSchemaManager).orElseGet(() -> InternalSchemaManager.get(this.conf, this.metaClient)); } @Override @@ -610,6 +609,7 @@ private FileIndex getOrBuildFileIndex() { .path(this.path) .conf(this.conf) .rowType(this.tableRowType) + .metaClient(metaClient) .columnStatsProbe(this.columnStatsProbe) .partitionPruner(this.partitionPruner) .partitionBucketIdFunc(PartitionBucketIdFunc.create(this.dataBucketFunc, diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java index 6d35785e1b30d..fadffd1346013 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java @@ -27,6 +27,7 @@ import org.apache.hudi.source.prune.PartitionPruners; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; +import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestData; @@ -82,7 +83,7 @@ void testFileListingUsingMetadata(boolean hiveStylePartitioning) throws Exceptio conf.setBoolean(HIVE_STYLE_PARTITIONING, hiveStylePartitioning); TestData.writeData(TestData.DATA_SET_INSERT, conf); FileIndex fileIndex = FileIndex.builder().path(new StoragePath(tempFile.getAbsolutePath())).conf(conf) - .rowType(TestConfigurations.ROW_TYPE).build(); + .rowType(TestConfigurations.ROW_TYPE).metaClient(StreamerUtil.createMetaClient(conf)).build(); List partitionKeys = Collections.singletonList("partition"); List> partitions = fileIndex.getPartitions(partitionKeys, PARTITION_DEFAULT_NAME.defaultValue(), @@ -106,7 +107,7 @@ void testFileListingUsingMetadataNonPartitionedTable() throws Exception { conf.setBoolean(METADATA_ENABLED, true); TestData.writeData(TestData.DATA_SET_INSERT, conf); FileIndex fileIndex = FileIndex.builder().path(new StoragePath(tempFile.getAbsolutePath())).conf(conf) - .rowType(TestConfigurations.ROW_TYPE).build(); + .rowType(TestConfigurations.ROW_TYPE).metaClient(StreamerUtil.createMetaClient(conf)).build(); List partitionKeys = Collections.singletonList(""); List> partitions = fileIndex.getPartitions(partitionKeys, PARTITION_DEFAULT_NAME.defaultValue(), false); @@ -147,7 +148,9 @@ void testFileListingWithDataSkipping() throws Exception { FileIndex fileIndex = FileIndex.builder() .path(new StoragePath(tempFile.getAbsolutePath())) - .conf(conf).rowType(TestConfigurations.ROW_TYPE_BIGINT) + .conf(conf) + .rowType(TestConfigurations.ROW_TYPE_BIGINT) + .metaClient(StreamerUtil.createMetaClient(conf)) .columnStatsProbe(ColumnStatsProbe.newInstance(Collections.singletonList(new CallExpression( FunctionIdentifier.of("greaterThan"), BuiltInFunctionDefinitions.GREATER_THAN, @@ -205,6 +208,7 @@ void testFileListingWithPartitionStatsPruning(HoodieTableType tableType) throws .path(new StoragePath(tempFile.getAbsolutePath())) .conf(conf) .rowType(TestConfigurations.ROW_TYPE) + .metaClient(StreamerUtil.createMetaClient(conf)) .partitionPruner(PartitionPruners.builder().rowType(TestConfigurations.ROW_TYPE).basePath(tempFile.getAbsolutePath()).conf(conf).columnStatsProbe(columnStatsProbe).build()) .build(); From 6dc7f2cd8d713526be6f2f0b025c635980a9f6aa Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 30 Jun 2025 18:26:02 -0700 Subject: [PATCH 07/11] Minor changes to RFC-93 based on PR --- rfc/rfc-93/rfc-93.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/rfc/rfc-93/rfc-93.md b/rfc/rfc-93/rfc-93.md index cefd3198aec01..3a5f0aef540b0 100644 --- a/rfc/rfc-93/rfc-93.md +++ b/rfc/rfc-93/rfc-93.md @@ -27,7 +27,7 @@ ## Status -JIRA: +JIRA: https://issues.apache.org/jira/browse/HUDI-9332 ## Abstract @@ -57,7 +57,7 @@ Some non-technical reasons: ## **Implementation** -The main implementation step here is to create abstraction called TableFormatPlugin which handles table format operations such as +The main implementation step here is to create abstraction called `HoodieTableFormat` which handles table format operations such as 1. Committing writes @@ -68,7 +68,8 @@ The main implementation step here is to create abstraction called TableFormatPlu 6. Rollbacks -The Hudi platform is responsible for managing the data path and can be configured with the table format plugin by default using Hudi native table format. Other table formats will have their own implementation of this abstraction. +The Hudi platform is responsible for managing the data path and can be configured with the table format plugin or by default it continues to use Hudi's native table format. +Other table formats will have their own implementation of this abstraction. ### Commit Protocol: @@ -80,7 +81,7 @@ Hudi uses the creation of an action-complete file (e:g .commit, .deltacommit, .c The Hudi timeline is still very much used for all internal operations and the table format's commit metadata will be an overlay on top of this. With this, the action is only completed when both the above steps are completed. The plugin provided timeline needs to fence the timeline ensuring the definition of complete stays consistent. This ensures the snapshot isolation is maintained. -There will be a hudi table-property "hudi.table.format.plugin" to identify the table format with default being "native". This allows consistency in plugin behaviors across all hudi writers. +There will be a hudi table-property "hudi.table.format" to identify the table format with default being "native". This allows consistency in plugin behaviors across all hudi writers. ### Metadata: From cf93d8d30683e0d345db815a91a1e01cfb875943 Mon Sep 17 00:00:00 2001 From: danny0405 Date: Tue, 1 Jul 2025 11:21:58 +0800 Subject: [PATCH 08/11] fix the file stats index meta client, fix the archived instants to be lazy --- .../versioning/v1/TimelineArchiverV1.java | 2 +- .../versioning/v2/TimelineArchiverV2.java | 8 +++--- .../apache/hudi/common/HoodieTableFormat.java | 3 ++- .../hudi/tableformat/TestTableFormat.java | 9 ++++--- .../org/apache/hudi/source/FileIndex.java | 2 +- .../hudi/source/prune/PartitionPruners.java | 22 ++++++++++----- .../hudi/source/stats/FileStatsIndex.java | 27 ++++++++++++------- .../source/stats/PartitionStatsIndex.java | 10 ++++--- .../apache/hudi/table/HoodieTableSource.java | 1 + .../source/TestIncrementalInputSplits.java | 2 +- .../source/stats/TestColumnStatsIndex.java | 13 ++++----- 11 files changed, 59 insertions(+), 40 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java index db2ed4f5799fc..a2ad333c3b602 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java @@ -401,7 +401,7 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant)); } // Call Table Format archive to allow archiving in table format. - table.getMetaClient().getTableFormat().archive(archivedInstants, table.getContext(), table.getMetaClient(), table.getViewManager()); + table.getMetaClient().getTableFormat().archive(() -> archivedInstants, table.getContext(), table.getMetaClient(), table.getViewManager()); return true; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java index 8803583bcd4ac..9a39a45ead6d9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java @@ -51,12 +51,12 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Consumer; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -118,9 +118,9 @@ public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) t deleteArchivedActions(instantsToArchive, context); // triggers compaction and cleaning only after archiving action this.timelineWriter.compactAndClean(context); - List archivedInstants = instantsToArchive.stream() - .map(action -> Stream.concat(action.getCompletedInstants().stream(), action.getPendingInstants().stream()).collect(Collectors.toList())) - .flatMap(Collection::stream).collect(Collectors.toList()); + Supplier> archivedInstants = () -> instantsToArchive.stream() + .flatMap(action -> Stream.concat(action.getCompletedInstants().stream(), action.getPendingInstants().stream())) + .collect(Collectors.toList()); // Call Table Format archive to allow archiving in table format. table.getMetaClient().getTableFormat().archive(archivedInstants, table.getContext(), table.getMetaClient(), table.getViewManager()); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java index b01ca22bacb76..01b018af9b1bf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/HoodieTableFormat.java @@ -30,6 +30,7 @@ import java.io.Serializable; import java.util.List; import java.util.Properties; +import java.util.function.Supplier; /** * External Table Format needs to implement this interface @@ -92,7 +93,7 @@ default void clean( * @param viewManager viewManager from HoodieTable. */ default void archive( - List archivedInstants, + Supplier> archivedInstants, HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { diff --git a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java index e8a16a6b56092..43d45af8ad14d 100644 --- a/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/tableformat/TestTableFormat.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Supplier; /** * Test implementation of HoodieTableFormat that records all Hoodie instants in memory. @@ -40,7 +41,7 @@ */ public class TestTableFormat implements HoodieTableFormat { - private static final Map> RECORDED_INSTANTS = new ConcurrentHashMap<>(); + private static final Map> RECORDED_INSTANTS = new ConcurrentHashMap<>(); public TestTableFormat() { } @@ -74,9 +75,9 @@ public void clean(HoodieCleanMetadata cleanMetadata, HoodieInstant completedInst } @Override - public void archive(List archivedInstants, HoodieEngineContext engineContext, - HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { - RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).removeAll(archivedInstants); + public void archive(Supplier> archivedInstants, HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, FileSystemViewManager viewManager) { + RECORDED_INSTANTS.get(metaClient.getBasePath().toString()).removeAll(archivedInstants.get()); } @Override diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java index e25723966d223..273bbb5810316 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java @@ -86,7 +86,7 @@ private FileIndex( this.metadataConfig = StreamerUtil.metadataConfig(conf); this.colStatsProbe = isDataSkippingFeasible(conf.get(FlinkOptions.READ_DATA_SKIPPING_ENABLED)) ? colStatsProbe : null; this.partitionPruner = partitionPruner; - this.fileStatsIndex = new FileStatsIndex(path.toString(), rowType, metadataConfig); + this.fileStatsIndex = new FileStatsIndex(path.toString(), rowType, conf, metaClient); this.partitionBucketIdFunc = partitionBucketIdFunc; this.metaClient = metaClient; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/prune/PartitionPruners.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/prune/PartitionPruners.java index 955f471df31c5..f80ba96bda88e 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/prune/PartitionPruners.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/prune/PartitionPruners.java @@ -18,7 +18,7 @@ package org.apache.hudi.source.prune; -import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.source.ExpressionEvaluators; import org.apache.hudi.source.ExpressionEvaluators.Evaluator; @@ -26,12 +26,13 @@ import org.apache.hudi.source.stats.PartitionStatsIndex; import org.apache.hudi.table.format.FilePathUtils; import org.apache.hudi.util.DataTypeUtils; -import org.apache.hudi.util.StreamerUtil; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import javax.annotation.Nullable; + import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -147,10 +148,11 @@ public static class ColumnStatsPartitionPruner implements PartitionPruner { public ColumnStatsPartitionPruner( RowType rowType, String basePath, - HoodieMetadataConfig metadataConfig, - ColumnStatsProbe probe) { + Configuration conf, + ColumnStatsProbe probe, + @Nullable HoodieTableMetaClient metaClient) { this.probe = probe; - this.partitionStatsIndex = new PartitionStatsIndex(basePath, rowType, metadataConfig); + this.partitionStatsIndex = new PartitionStatsIndex(basePath, rowType, conf, metaClient); } @Override @@ -190,6 +192,7 @@ public static Builder builder() { public static class Builder { private RowType rowType; private String basePath; + private HoodieTableMetaClient metaClient; private Configuration conf; private ColumnStatsProbe probe; private List partitionEvaluators; @@ -212,6 +215,11 @@ public Builder basePath(String basePath) { return this; } + public Builder metaClient(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + return this; + } + public Builder conf(Configuration conf) { this.conf = conf; return this; @@ -267,8 +275,8 @@ public PartitionPruner build() { if (probe != null && conf.get(FlinkOptions.READ_DATA_SKIPPING_ENABLED) && conf.get(FlinkOptions.METADATA_ENABLED)) { - columnStatsPruner = new ColumnStatsPartitionPruner(Objects.requireNonNull(rowType), Objects.requireNonNull(basePath), - StreamerUtil.metadataConfig(Objects.requireNonNull(conf)), probe); + columnStatsPruner = new ColumnStatsPartitionPruner(Objects.requireNonNull(rowType), Objects.requireNonNull(basePath), Objects.requireNonNull(conf), + probe, metaClient); } List partitionPruners = Stream.of(staticPruner, dynamicPruner, columnStatsPruner) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java index 16f3acec6a711..b01f4c11c5426 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/FileStatsIndex.java @@ -20,7 +20,6 @@ import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.client.common.HoodieFlinkEngineContext; -import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -34,13 +33,13 @@ import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.source.prune.ColumnStatsProbe; -import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hudi.util.AvroToRowDataConverters; import org.apache.hudi.util.DataTypeUtils; -import org.apache.hudi.util.FlinkClientUtil; import org.apache.hudi.util.RowDataProjection; +import org.apache.hudi.util.StreamerUtil; import org.apache.avro.generic.GenericRecord; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -83,16 +82,19 @@ public class FileStatsIndex implements ColumnStatsIndex { private static final Logger LOG = LoggerFactory.getLogger(FileStatsIndex.class); private final RowType rowType; private final String basePath; - private final HoodieMetadataConfig metadataConfig; + private final Configuration conf; + private HoodieTableMetaClient metaClient; private HoodieTableMetadata metadataTable; public FileStatsIndex( String basePath, RowType rowType, - HoodieMetadataConfig metadataConfig) { + Configuration conf, + @Nullable HoodieTableMetaClient metaClient) { this.basePath = basePath; this.rowType = rowType; - this.metadataConfig = metadataConfig; + this.conf = conf; + this.metaClient = metaClient; } @Override @@ -103,17 +105,22 @@ public String getIndexPartitionName() { public HoodieTableMetadata getMetadataTable() { // initialize the metadata table lazily if (this.metadataTable == null) { - HoodieHadoopStorage storage = new HoodieHadoopStorage(basePath, FlinkClientUtil.getHadoopConf()); - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(storage.getConf()).build(); + initMetaClient(); this.metadataTable = metaClient.getTableFormat().getMetadataFactory().create( HoodieFlinkEngineContext.DEFAULT, - storage, - metadataConfig, + metaClient.getStorage(), + StreamerUtil.metadataConfig(conf), basePath); } return this.metadataTable; } + private void initMetaClient() { + if (this.metaClient == null) { + this.metaClient = StreamerUtil.createMetaClient(conf); + } + } + @Override public Set computeCandidateFiles(ColumnStatsProbe probe, List allFiles) { if (probe == null) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/PartitionStatsIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/PartitionStatsIndex.java index e2f6431e77db8..1facae5671d95 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/PartitionStatsIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/PartitionStatsIndex.java @@ -19,12 +19,15 @@ package org.apache.hudi.source.stats; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; -import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.source.prune.ColumnStatsProbe; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.types.logical.RowType; +import javax.annotation.Nullable; + import java.util.List; import java.util.Set; @@ -37,8 +40,9 @@ public class PartitionStatsIndex extends FileStatsIndex { public PartitionStatsIndex( String basePath, RowType tableRowType, - HoodieMetadataConfig metadataConfig) { - super(basePath, tableRowType, metadataConfig); + Configuration conf, + @Nullable HoodieTableMetaClient metaClient) { + super(basePath, tableRowType, conf, metaClient); } @Override diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index 2179e72321706..a764d8f175cdd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -357,6 +357,7 @@ private PartitionPruners.PartitionPruner createPartitionPruner(List indexRows = indexSupport.readColumnStatsIndexByColumns(queryColumns); List results = indexRows.stream().map(Object::toString).sorted(String::compareTo).collect(Collectors.toList()); List expected = Arrays.asList( @@ -93,19 +94,15 @@ void testReadPartitionStatsIndex() throws Exception { void testTransposeColumnStatsIndex() throws Exception { final String path = tempFile.getAbsolutePath(); Configuration conf = TestConfigurations.getDefaultConf(path); - conf.setBoolean(FlinkOptions.METADATA_ENABLED, true); + conf.setBoolean(HoodieMetadataConfig.ENABLE.key(), true); conf.setBoolean(FlinkOptions.READ_DATA_SKIPPING_ENABLED, true); - conf.setString("hoodie.metadata.index.column.stats.enable", "true"); + conf.setBoolean(HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key(), true); - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() - .enable(true) - .withMetadataIndexColumnStats(true) - .build(); TestData.writeData(TestData.DATA_SET_INSERT, conf); // explicit query columns String[] queryColumns1 = {"uuid", "age"}; - FileStatsIndex indexSupport = new FileStatsIndex(path, TestConfigurations.ROW_TYPE, metadataConfig); + FileStatsIndex indexSupport = new FileStatsIndex(path, TestConfigurations.ROW_TYPE, conf, StreamerUtil.createMetaClient(conf)); List indexRows1 = indexSupport.readColumnStatsIndexByColumns(queryColumns1); Pair, String[]> transposedIndexTable1 = indexSupport.transposeColumnStatsIndex(indexRows1, queryColumns1); assertThat("The schema columns should sort by natural order", From 59cf10689bd2573835119c1c03d2d24818ab96db Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Tue, 1 Jul 2025 08:13:22 -0700 Subject: [PATCH 09/11] Minor test/code cleanup --- .../versioning/v1/TimelineArchiverV1.java | 3 +-- .../TestEightToSevenDowngradeHandler.java | 20 ++++++++++++++++++- .../TestSevenToEightUpgradeHandler.java | 2 -- .../source/stats/TestColumnStatsIndex.java | 4 ---- 4 files changed, 20 insertions(+), 9 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java index a2ad333c3b602..b8c94ab2804ae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java @@ -397,8 +397,7 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo // Therefore, the concurrency of deleting completed instants is temporarily disabled, // and instants are deleted in ascending order to prevent the occurrence of such holes. // See HUDI-7207 and #10325. - completedInstants.stream() - .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant)); + completedInstants.stream().forEach(activeTimeline::deleteInstantFileIfExists); } // Call Table Format archive to allow archiving in table format. table.getMetaClient().getTableFormat().archive(() -> archivedInstants, table.getContext(), table.getMetaClient(), table.getViewManager()); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java index a4f7c45b3ce02..7e0c85795afc6 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestEightToSevenDowngradeHandler.java @@ -134,16 +134,34 @@ void testDowngradeMetadataPartitions() { String baseTablePath = baseDir.toString(); HoodieStorage hoodieStorage = HoodieStorageUtils.getStorage(getDefaultStorageConf()); StoragePath basePath = new StoragePath(baseTablePath); + + HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); when(metaClient.getBasePath()).thenReturn(basePath); + when(metaClient.getTableConfig()).thenReturn(tableConfig); + when(metaClient.getStorage()).thenReturn(hoodieStorage); Map tablePropsToAdd = new HashMap<>(); try (MockedStatic mockedFSUtils = mockStatic(FSUtils.class); - MockedStatic mockedMetadataUtils = mockStatic(HoodieTableMetadataUtil.class)) { + MockedStatic mockedStaticMetaClient = mockStatic(HoodieTableMetaClient.class)) { StoragePath mdtBasePath = HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()); + + // Mock FSUtils.getAllPartitionPaths to return SAMPLE_METADATA_PATHS mockedFSUtils .when(() -> FSUtils.getAllPartitionPaths(context, metaClient, false)) .thenReturn(SAMPLE_METADATA_PATHS); + // Mock HoodieTableMetaClient.builder() to return a builder that returns a mock metaClient + HoodieTableMetaClient.Builder mockBuilder = mock(HoodieTableMetaClient.Builder.class); + when(mockBuilder.setBasePath(mdtBasePath.toUri().toString())).thenReturn(mockBuilder); + when(mockBuilder.setConf(hoodieStorage.getConf())).thenReturn(mockBuilder); + when(mockBuilder.build()).thenReturn(metaClient); + mockedStaticMetaClient.when(HoodieTableMetaClient::builder).thenReturn(mockBuilder); + + // Mock FSUtils.isTableExists to return true + mockedFSUtils + .when(() -> FSUtils.isTableExists(mdtBasePath.toString(), hoodieStorage)) + .thenReturn(true); + EightToSevenDowngradeHandler.downgradeMetadataPartitions(context, hoodieStorage, metaClient, tablePropsToAdd); assertTrue(tablePropsToAdd.containsKey(TABLE_METADATA_PARTITIONS)); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java index bcc9d897620f8..7a6dea543d8bb 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestSevenToEightUpgradeHandler.java @@ -61,8 +61,6 @@ class TestSevenToEightUpgradeHandler { private HoodieWriteConfig config; @Mock private HoodieTableConfig tableConfig; - @Mock - private SupportsUpgradeDowngrade upgradeDowngradeHelper; private SevenToEightUpgradeHandler upgradeHandler; diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/stats/TestColumnStatsIndex.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/stats/TestColumnStatsIndex.java index 53708b32e2405..e911f9fdcbcab 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/stats/TestColumnStatsIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/stats/TestColumnStatsIndex.java @@ -57,10 +57,6 @@ void testReadPartitionStatsIndex() throws Exception { conf.set(FlinkOptions.METADATA_ENABLED, true); conf.setString("hoodie.metadata.index.partition.stats.enable", "true"); conf.setString("hoodie.metadata.index.column.stats.enable", "true"); - HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() - .enable(true) - .withMetadataIndexColumnStats(true) - .build(); TestData.writeData(TestData.DATA_SET_INSERT, conf); String[] queryColumns = {"uuid", "age"}; From fc839775da18d0cda9f4136927042646621cab47 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Tue, 1 Jul 2025 18:01:18 -0700 Subject: [PATCH 10/11] Limiting handling to cases where non-native table format is used --- .../action/rollback/CopyOnWriteRollbackActionExecutor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java index 80e4e5b80732c..bd619c85616c7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.NativeTableFormat; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -76,7 +77,7 @@ protected List executeRollback(HoodieRollbackPlan hoodieRoll } // If instant is inflight but marked as completed in native format, delete the completed instant from storage. - if (instantToRollback.isInflight()) { + if (instantToRollback.isInflight() && !table.getMetaClient().getTableFormat().getName().equals(NativeTableFormat.TABLE_FORMAT)) { HoodieActiveTimeline activeTimelineForNativeFormat = table.getMetaClient().getActiveTimelineForNativeFormat(); Option instantToRollbackInNativeFormat = activeTimelineForNativeFormat.filter(instant -> instant.requestedTime().equals(instantToRollback.requestedTime())).lastInstant(); if (instantToRollbackInNativeFormat.isPresent() && instantToRollbackInNativeFormat.get().isCompleted()) { From 222d7b08a976998e5416ed6bd7c90abb1d2aee0e Mon Sep 17 00:00:00 2001 From: danny0405 Date: Wed, 2 Jul 2025 11:06:55 +0800 Subject: [PATCH 11/11] fix the reduadant completion time computation --- .../org/apache/hudi/client/WriteClientTestUtils.java | 3 +-- .../table/timeline/SkewAdjustingTimeGenerator.java | 3 +-- .../hudi/common/table/timeline/TimeGenerator.java | 4 +--- .../timeline/versioning/v2/ActiveTimelineV2.java | 12 ++++++++---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java index 547484cf71413..376da22edc98f 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/WriteClientTestUtils.java @@ -57,8 +57,7 @@ public long generateTime(boolean skipLocking) { } @Override - public long consumeTime(boolean skipLocking, Consumer func) { - return 0; + public void consumeTime(boolean skipLocking, Consumer func) { } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java index ce39c7b6ad751..0461ec2c43e96 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/SkewAdjustingTimeGenerator.java @@ -54,14 +54,13 @@ public long generateTime(boolean skipLocking) { } @Override - public long consumeTime(boolean skipLocking, Consumer func) { + public void consumeTime(boolean skipLocking, Consumer func) { try { if (!skipLocking) { lock(); } long currentTimeMillis = generateTime(true); func.accept(currentTimeMillis); - return currentTimeMillis; } finally { if (!skipLocking) { unlock(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java index c64ce043c2548..f797eb33c501b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java @@ -47,8 +47,6 @@ public interface TimeGenerator { * * @param skipLocking If this is triggered by another parent transaction, locking can be skipped. * @param func A consumer that takes a monotonically increasing timestamp. - * - * @return Return TrueTime as milliseconds in consumer function. */ - long consumeTime(boolean skipLocking, Consumer func); + void consumeTime(boolean skipLocking, Consumer func); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java index f2b1ee555f36d..b823c53580ddb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java @@ -62,6 +62,7 @@ import java.util.HashSet; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.TimelineUtils.getHoodieInstantWriterOption; @@ -562,7 +563,7 @@ protected HoodieInstant transitionStateToComplete(boolean shouldLock, Hoodie ValidationUtils.checkArgument( metaClient.getStorage().exists(getInstantFileNamePath(fromInstantFileName)), "File " + getInstantFileNamePath(fromInstantFileName) + " does not exist!"); - String completionTime = HoodieInstantTimeGenerator.formatDateBasedOnTimeZone(new Date(createCompleteFileInMetaPath(shouldLock, toInstant, metadata))); + String completionTime = createCompleteFileInMetaPath(shouldLock, toInstant, metadata); return new HoodieInstant(toInstant.getState(), toInstant.getAction(), toInstant.requestedTime(), completionTime, instantComparator.requestedTimeOrderedComparator()); } } catch (IOException e) { @@ -742,11 +743,12 @@ public void createFileInMetaPath(String filename, Option metadata, boolea } } - protected long createCompleteFileInMetaPath(boolean shouldLock, HoodieInstant instant, Option metadata) { + protected String createCompleteFileInMetaPath(boolean shouldLock, HoodieInstant instant, Option metadata) { Option writerOption = getHoodieInstantWriterOption(this, metadata); TimeGenerator timeGenerator = TimeGenerators .getTimeGenerator(metaClient.getTimeGeneratorConfig(), metaClient.getStorageConf()); - return timeGenerator.consumeTime(!shouldLock, currentTimeMillis -> { + final AtomicReference completionTimeRef = new AtomicReference<>(); + timeGenerator.consumeTime(!shouldLock, currentTimeMillis -> { String completionTime = HoodieInstantTimeGenerator.formatDateBasedOnTimeZone(new Date(currentTimeMillis)); String fileName = instantFileNameGenerator.getFileName(completionTime, instant); StoragePath fullPath = getInstantFileNamePath(fileName); @@ -755,8 +757,10 @@ protected long createCompleteFileInMetaPath(boolean shouldLock, HoodieInstan } else { metaClient.getStorage().createImmutableFileInPath(fullPath, writerOption); } - LOG.info("Created new file for toInstant ?{}", fullPath); + completionTimeRef.set(completionTime); + LOG.info("Created new file for toInstant: {}", fullPath); }); + return completionTimeRef.get(); } protected Option readDataFromPath(StoragePath detailPath) {