diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java index a9aefa2ab4744..d106d8375e7a8 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -25,7 +25,6 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; @@ -34,11 +33,15 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieBackedTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.spark.api.java.JavaSparkContext; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.shell.standard.ShellComponent; @@ -57,9 +60,6 @@ import java.util.Map; import java.util.Set; -import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; -import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT; - /** * CLI commands to operate on the Metadata Table. *

@@ -115,13 +115,13 @@ public String set(@ShellOption(value = {"--metadataDir"}, @ShellMethod(key = "metadata create", value = "Create the Metadata Table if it does not exist") public String create( @ShellOption(value = "--sparkMaster", defaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master - ) throws IOException { + ) throws Exception { HoodieCLI.getTableMetaClient(); Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); try { FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath); if (statuses.length > 0) { - throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") not empty."); + throw new RuntimeException("Metadata directory (" + metadataPath + ") not empty."); } } catch (FileNotFoundException e) { // Metadata directory does not exist yet @@ -131,28 +131,32 @@ public String create( HoodieTimer timer = HoodieTimer.start(); HoodieWriteConfig writeConfig = getWriteConfig(); initJavaSparkContext(Option.of(master)); - SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); - return String.format("Created Metadata Table in %s (duration=%.2f secs)", metadataPath, timer.endTimer() / 1000.0); + try (HoodieTableMetadataWriter writer = SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc))) { + return String.format("Created Metadata Table in %s (duration=%.2f secs)", metadataPath, timer.endTimer() / 1000.0); + } } @ShellMethod(key = "metadata delete", value = "Remove the Metadata Table") - public String delete() throws Exception { - HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient(); - Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); - try { - FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath); - if (statuses.length > 0) { - HoodieCLI.fs.delete(metadataPath, true); - } - } catch (FileNotFoundException e) { - // Metadata directory does not exist + public String delete(@ShellOption(value = "--backup", help = "Backup the metadata table before delete", defaultValue = "true", arity = 1) final boolean backup) throws Exception { + HoodieTableMetaClient dataMetaClient = HoodieCLI.getTableMetaClient(); + String backupPath = HoodieTableMetadataUtil.deleteMetadataTable(dataMetaClient, new HoodieSparkEngineContext(jsc), backup); + if (backup) { + return "Metadata Table has been deleted and backed up to " + backupPath; + } else { + return "Metadata Table has been deleted from " + getMetadataTableBasePath(HoodieCLI.basePath); } + } - LOG.info("Clear hoodie.table.metadata.partitions in hoodie.properties"); - HoodieTableConfig.delete(metaClient.getFs(), new Path(metaClient.getMetaPath()), new HashSet<>(Arrays - .asList(TABLE_METADATA_PARTITIONS.key(), TABLE_METADATA_PARTITIONS_INFLIGHT.key()))); - - return String.format("Removed Metadata Table from %s", metadataPath); + @ShellMethod(key = "metadata delete-record-index", value = "Delete the record index from Metadata Table") + public String deleteRecordIndex(@ShellOption(value = "--backup", help = "Backup the record index before delete", defaultValue = "true", arity = 1) final boolean backup) throws Exception { + HoodieTableMetaClient dataMetaClient = HoodieCLI.getTableMetaClient(); + String backupPath = HoodieTableMetadataUtil.deleteMetadataTablePartition(dataMetaClient, new HoodieSparkEngineContext(jsc), + MetadataPartitionType.RECORD_INDEX, backup); + if (backup) { + return "Record Index has been deleted from the Metadata Table and backed up to " + backupPath; + } else { + return "Record Index has been deleted from the Metadata Table"; + } } @ShellMethod(key = "metadata init", value = "Update the metadata table from commits since the creation") @@ -165,14 +169,16 @@ public String init(@ShellOption(value = "--sparkMaster", defaultValue = SparkUti HoodieCLI.fs.listStatus(metadataPath); } catch (FileNotFoundException e) { // Metadata directory does not exist - throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") does not exist."); + throw new RuntimeException("Metadata directory (" + metadataPath + ") does not exist."); } HoodieTimer timer = HoodieTimer.start(); if (!readOnly) { HoodieWriteConfig writeConfig = getWriteConfig(); initJavaSparkContext(Option.of(master)); - SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); + try (HoodieTableMetadataWriter writer = SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc))) { + // Empty + } } String action = readOnly ? "Opened" : "Initialized"; @@ -183,23 +189,23 @@ public String init(@ShellOption(value = "--sparkMaster", defaultValue = SparkUti public String stats() throws IOException { HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); - HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(HoodieCLI.conf), - config, HoodieCLI.basePath, "/tmp"); - Map stats = metadata.stats(); + try (HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(HoodieCLI.conf), + config, HoodieCLI.basePath)) { + Map stats = metadata.stats(); + + final List rows = new ArrayList<>(); + for (Map.Entry entry : stats.entrySet()) { + Comparable[] row = new Comparable[2]; + row[0] = entry.getKey(); + row[1] = entry.getValue(); + rows.add(row); + } - final List rows = new ArrayList<>(); - for (Map.Entry entry : stats.entrySet()) { - Comparable[] row = new Comparable[2]; - row[0] = entry.getKey(); - row[1] = entry.getValue(); - rows.add(row); + TableHeader header = new TableHeader() + .addTableHeaderField("stat key") + .addTableHeaderField("stat value"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", false, Integer.MAX_VALUE, false, rows); } - - TableHeader header = new TableHeader() - .addTableHeaderField("stat key") - .addTableHeaderField("stat value"); - return HoodiePrintHelper.print(header, new HashMap<>(), "", - false, Integer.MAX_VALUE, false, rows); } @ShellMethod(key = "metadata list-partitions", value = "List all partitions from metadata") @@ -209,27 +215,27 @@ public String listPartitions( HoodieCLI.getTableMetaClient(); initJavaSparkContext(Option.of(master)); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); - HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, - HoodieCLI.basePath, "/tmp"); + try (HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, + HoodieCLI.basePath)) { - if (!metadata.enabled()) { - return "[ERROR] Metadata Table not enabled/initialized\n\n"; - } + if (!metadata.enabled()) { + return "[ERROR] Metadata Table not enabled/initialized\n\n"; + } - HoodieTimer timer = HoodieTimer.start(); - List partitions = metadata.getAllPartitionPaths(); - LOG.debug("Took " + timer.endTimer() + " ms"); + HoodieTimer timer = HoodieTimer.start(); + List partitions = metadata.getAllPartitionPaths(); + LOG.debug("Took " + timer.endTimer() + " ms"); - final List rows = new ArrayList<>(); - partitions.stream().sorted(Comparator.reverseOrder()).forEach(p -> { - Comparable[] row = new Comparable[1]; - row[0] = p; - rows.add(row); - }); - - TableHeader header = new TableHeader().addTableHeaderField("partition"); - return HoodiePrintHelper.print(header, new HashMap<>(), "", - false, Integer.MAX_VALUE, false, rows); + final List rows = new ArrayList<>(); + partitions.stream().sorted(Comparator.reverseOrder()).forEach(p -> { + Comparable[] row = new Comparable[1]; + row[0] = p; + rows.add(row); + }); + + TableHeader header = new TableHeader().addTableHeaderField("partition"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", false, Integer.MAX_VALUE, false, rows); + } } @ShellMethod(key = "metadata list-files", value = "Print a list of all files in a partition from the metadata") @@ -237,32 +243,32 @@ public String listFiles( @ShellOption(value = {"--partition"}, help = "Name of the partition to list files", defaultValue = "") final String partition) throws IOException { HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); - HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); + try (HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata( + new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath)) { - if (!metaReader.enabled()) { - return "[ERROR] Metadata Table not enabled/initialized\n\n"; - } + if (!metaReader.enabled()) { + return "[ERROR] Metadata Table not enabled/initialized\n\n"; + } - Path partitionPath = new Path(HoodieCLI.basePath); - if (!StringUtils.isNullOrEmpty(partition)) { - partitionPath = new Path(HoodieCLI.basePath, partition); - } + Path partitionPath = new Path(HoodieCLI.basePath); + if (!StringUtils.isNullOrEmpty(partition)) { + partitionPath = new Path(HoodieCLI.basePath, partition); + } - HoodieTimer timer = HoodieTimer.start(); - FileStatus[] statuses = metaReader.getAllFilesInPartition(partitionPath); - LOG.debug("Took " + timer.endTimer() + " ms"); + HoodieTimer timer = HoodieTimer.start(); + FileStatus[] statuses = metaReader.getAllFilesInPartition(partitionPath); + LOG.debug("Took " + timer.endTimer() + " ms"); - final List rows = new ArrayList<>(); - Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(f -> { - Comparable[] row = new Comparable[1]; - row[0] = f; - rows.add(row); - }); - - TableHeader header = new TableHeader().addTableHeaderField("file path"); - return HoodiePrintHelper.print(header, new HashMap<>(), "", - false, Integer.MAX_VALUE, false, rows); + final List rows = new ArrayList<>(); + Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(f -> { + Comparable[] row = new Comparable[1]; + row[0] = f; + rows.add(row); + }); + + TableHeader header = new TableHeader().addTableHeaderField("file path"); + return HoodiePrintHelper.print(header, new HashMap<>(), "", false, Integer.MAX_VALUE, false, rows); + } } @ShellMethod(key = "metadata validate-files", value = "Validate all files in all partitions from the metadata") @@ -271,7 +277,7 @@ public String validateFiles( HoodieCLI.getTableMetaClient(); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); HoodieBackedTableMetadata metadataReader = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath, "/tmp"); + new HoodieLocalEngineContext(HoodieCLI.conf), config, HoodieCLI.basePath); if (!metadataReader.enabled()) { return "[ERROR] Metadata Table not enabled/initialized\n\n"; @@ -279,7 +285,7 @@ public String validateFiles( HoodieMetadataConfig fsConfig = HoodieMetadataConfig.newBuilder().enable(false).build(); HoodieBackedTableMetadata fsMetaReader = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(HoodieCLI.conf), fsConfig, HoodieCLI.basePath, "/tmp"); + new HoodieLocalEngineContext(HoodieCLI.conf), fsConfig, HoodieCLI.basePath); HoodieTimer timer = HoodieTimer.start(); List metadataPartitions = metadataReader.getAllPartitionPaths(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java index aa75ff29b8b27..d45c63eddbb66 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java @@ -104,7 +104,7 @@ public void init() throws Exception { metaClient.getHadoopConf(), config, context)) .withPartitionMetaFiles(DEFAULT_PARTITION_PATHS) .addCommit("100") - .withBaseFilesInPartitions(partitionAndFileId) + .withBaseFilesInPartitions(partitionAndFileId).getLeft() .addCommit("101"); hoodieTestTable.addCommit("102").withBaseFilesInPartitions(partitionAndFileId); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java index 98e258d0392d9..a1942a62312b4 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java @@ -104,9 +104,9 @@ public void init() throws Exception { metaClient.getHadoopConf(), config, context)) .withPartitionMetaFiles(DEFAULT_PARTITION_PATHS) .addCommit("100") - .withBaseFilesInPartitions(partitionAndFileId) + .withBaseFilesInPartitions(partitionAndFileId).getLeft() .addCommit("101") - .withBaseFilesInPartitions(partitionAndFileId) + .withBaseFilesInPartitions(partitionAndFileId).getLeft() .addInflightCommit("102") .withBaseFilesInPartitions(partitionAndFileId); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java index ff983d44ae780..4d1a0ec3fb748 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java @@ -77,13 +77,13 @@ public void init() throws Exception { HoodieTestTable.of(metaClient) .withPartitionMetaFiles(DEFAULT_PARTITION_PATHS) .addCommit("100") - .withBaseFilesInPartition(DEFAULT_FIRST_PARTITION_PATH, "file-1") - .withBaseFilesInPartition(DEFAULT_SECOND_PARTITION_PATH, "file-2") - .withBaseFilesInPartition(DEFAULT_THIRD_PARTITION_PATH, "file-3") + .withBaseFilesInPartition(DEFAULT_FIRST_PARTITION_PATH, "file-1").getLeft() + .withBaseFilesInPartition(DEFAULT_SECOND_PARTITION_PATH, "file-2").getLeft() + .withBaseFilesInPartition(DEFAULT_THIRD_PARTITION_PATH, "file-3").getLeft() .addInflightCommit("101") - .withBaseFilesInPartition(DEFAULT_FIRST_PARTITION_PATH, "file-1") - .withBaseFilesInPartition(DEFAULT_SECOND_PARTITION_PATH, "file-2") - .withBaseFilesInPartition(DEFAULT_THIRD_PARTITION_PATH, "file-3") + .withBaseFilesInPartition(DEFAULT_FIRST_PARTITION_PATH, "file-1").getLeft() + .withBaseFilesInPartition(DEFAULT_SECOND_PARTITION_PATH, "file-2").getLeft() + .withBaseFilesInPartition(DEFAULT_THIRD_PARTITION_PATH, "file-3").getLeft() .withMarkerFile(DEFAULT_FIRST_PARTITION_PATH, "file-1", IOType.MERGE) .withMarkerFile(DEFAULT_SECOND_PARTITION_PATH, "file-2", IOType.MERGE) .withMarkerFile(DEFAULT_THIRD_PARTITION_PATH, "file-3", IOType.MERGE); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java index e1fb98fb0a786..d158b096c38c6 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java @@ -95,9 +95,9 @@ public void testRollbackCommit() throws Exception { HoodieTestTable.of(metaClient) .withPartitionMetaFiles(DEFAULT_PARTITION_PATHS) .addCommit("100") - .withBaseFilesInPartitions(partitionAndFileId) + .withBaseFilesInPartitions(partitionAndFileId).getLeft() .addCommit("101") - .withBaseFilesInPartitions(partitionAndFileId) + .withBaseFilesInPartitions(partitionAndFileId).getLeft() .addCommit("102") .withBaseFilesInPartitions(partitionAndFileId); 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 95d868a43e094..897dcae291514 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 @@ -29,6 +29,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.heartbeat.HeartbeatUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -494,15 +495,16 @@ protected void runAnyPendingClustering(HoodieTable table) { /** * Write the HoodieCommitMetadata to metadata table if available. * - * @param table {@link HoodieTable} of interest. - * @param instantTime instant time of the commit. - * @param actionType action type of the commit. - * @param metadata instance of {@link HoodieCommitMetadata}. + * @param table {@link HoodieTable} of interest. + * @param instantTime instant time of the commit. + * @param actionType action type of the commit. + * @param metadata instance of {@link HoodieCommitMetadata}. + * @param writeStatuses Write statuses of the commit */ - protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { + protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata, HoodieData writeStatuses) { checkArgument(table.isTableServiceAction(actionType, instantTime), String.format("Unsupported action: %s.%s is not table service.", actionType, instantTime)); context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table: " + config.getTableName()); - table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime)); + table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, writeStatuses, instantTime)); } /** 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 644c87be4352b..e7f69b6e43958 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -33,6 +33,7 @@ import org.apache.hudi.client.utils.TransactionUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.config.HoodieCommonConfig; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -206,12 +207,13 @@ public abstract boolean commit(String instantTime, O writeStatuses, Option> partitionToReplacedFileIds, Option> extraPreCommitFunc); - public boolean commitStats(String instantTime, List stats, Option> extraMetadata, + public boolean commitStats(String instantTime, HoodieData writeStatuses, List stats, Option> extraMetadata, String commitActionType) { - return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap(), Option.empty()); + return commitStats(instantTime, writeStatuses, stats, extraMetadata, commitActionType, Collections.emptyMap(), Option.empty()); } - public boolean commitStats(String instantTime, List stats, Option> extraMetadata, + public boolean commitStats(String instantTime, HoodieData writeStatuses, List stats, + Option> extraMetadata, String commitActionType, Map> partitionToReplaceFileIds, Option> extraPreCommitFunc) { // Skip the empty commit if not allowed @@ -232,7 +234,7 @@ public boolean commitStats(String instantTime, List stats, Opti if (extraPreCommitFunc.isPresent()) { extraPreCommitFunc.get().accept(table.getMetaClient(), metadata); } - commit(table, commitActionType, instantTime, metadata, stats); + commit(table, commitActionType, instantTime, metadata, stats, writeStatuses); postCommit(table, metadata, instantTime, extraMetadata); LOG.info("Committed " + instantTime); releaseResources(instantTime); @@ -270,7 +272,7 @@ public boolean commitStats(String instantTime, List stats, Opti } protected void commit(HoodieTable table, String commitActionType, String instantTime, HoodieCommitMetadata metadata, - List stats) throws IOException { + List stats, HoodieData writeStatuses) throws IOException { LOG.info("Committing " + instantTime + " action " + commitActionType); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); // Finalize write @@ -281,7 +283,7 @@ protected void commit(HoodieTable table, String commitActionType, String instant saveInternalSchema(table, instantTime, metadata); } // update Metadata table - writeTableMetadata(table, instantTime, commitActionType, metadata); + writeTableMetadata(table, instantTime, commitActionType, metadata, writeStatuses); activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } @@ -344,17 +346,19 @@ protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata met /** * Write the HoodieCommitMetadata to metadata table if available. - * @param table {@link HoodieTable} of interest. - * @param instantTime instant time of the commit. - * @param actionType action type of the commit. - * @param metadata instance of {@link HoodieCommitMetadata}. + * + * @param table {@link HoodieTable} of interest. + * @param instantTime instant time of the commit. + * @param actionType action type of the commit. + * @param metadata instance of {@link HoodieCommitMetadata}. + * @param writeStatuses WriteStatuses for the completed action. */ - protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { + protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata, HoodieData writeStatuses) { if (table.isTableServiceAction(actionType, instantTime)) { - tableServiceClient.writeTableMetadata(table, instantTime, actionType, metadata); + tableServiceClient.writeTableMetadata(table, instantTime, actionType, metadata, writeStatuses); } else { context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table: " + config.getTableName()); - table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime)); + table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, writeStatuses, instantTime)); } } @@ -785,7 +789,6 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean skipLocking) t return clean(cleanInstantTime, true, false); } - /** * Clean up any stale/old files/data lying around (either on file storage or index storage) based on the * configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be @@ -1196,7 +1199,7 @@ protected final HoodieTable initTable(WriteOperationType operationType, Option *

  • Checking whether upgrade/downgrade is required
  • @@ -1469,7 +1472,7 @@ private void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient m // try to save history schemas FileBasedInternalSchemaStorageManager schemasManager = new FileBasedInternalSchemaStorageManager(metaClient); schemasManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(newSchema, historySchemaStr)); - commitStats(instantTime, Collections.emptyList(), Option.of(extraMeta), commitActionType); + commitStats(instantTime, context.emptyHoodieData(), Collections.emptyList(), Option.of(extraMeta), commitActionType); } private InternalSchema getInternalSchema(TableSchemaResolver schemaUtil) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 74eeeaf22f437..0a28a69c78478 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -47,7 +47,6 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.TimelineUtils; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; @@ -576,9 +575,8 @@ private Stream 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 (table.getMetaClient().getTableConfig().isMetadataTableEnabled()) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(), - config.getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) { + if (table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(), config.getBasePath())) { 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/client/utils/MetadataTableUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataTableUtils.java index b9f2b41d82b49..5e4ed6ac3d9d2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataTableUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataTableUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.utils; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.config.HoodieWriteConfig; @@ -34,10 +35,10 @@ public class MetadataTableUtils { * @param config Write configs. * @return {@code true} if using batch lookup; {@code false} otherwise. */ - public static boolean shouldUseBatchLookup(HoodieWriteConfig config) { + public static boolean shouldUseBatchLookup(HoodieTableConfig tableConfig, HoodieWriteConfig config) { FileSystemViewStorageType storageType = config.getClientSpecifiedViewStorageConfig().getStorageType(); - return config.getMetadataConfig().enabled() + return tableConfig.isMetadataTableAvailable() && !FileSystemViewStorageType.EMBEDDED_KV_STORE.equals(storageType) && !FileSystemViewStorageType.SPILLABLE_DISK.equals(storageType); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index 9e245e1f3a38d..7c730def11be8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -54,6 +54,7 @@ import static org.apache.hudi.index.HoodieIndex.IndexType.GLOBAL_SIMPLE; import static org.apache.hudi.index.HoodieIndex.IndexType.HBASE; import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; +import static org.apache.hudi.index.HoodieIndex.IndexType.RECORD_INDEX; import static org.apache.hudi.index.HoodieIndex.IndexType.SIMPLE; /** @@ -74,7 +75,7 @@ public class HoodieIndexConfig extends HoodieConfig { // Builder#getDefaultIndexType has already set it according to engine type .noDefaultValue() .withValidValues(HBASE.name(), INMEMORY.name(), BLOOM.name(), GLOBAL_BLOOM.name(), - SIMPLE.name(), GLOBAL_SIMPLE.name(), BUCKET.name(), FLINK_STATE.name()) + SIMPLE.name(), GLOBAL_SIMPLE.name(), BUCKET.name(), FLINK_STATE.name(), RECORD_INDEX.name()) .withDocumentation(HoodieIndex.IndexType.class); @@ -342,6 +343,21 @@ public class HoodieIndexConfig extends HoodieConfig { + "Specifically, if a file slice size is smaller than `hoodie.xxxx.max.file.size` * threshold, then it will be considered" + "as a merge candidate."); + public static final ConfigProperty RECORD_INDEX_USE_CACHING = ConfigProperty + .key("hoodie.record.index.use.caching") + .defaultValue("true") + .markAdvanced() + .withDocumentation("Only applies if index type is RECORD_INDEX." + + "When true, the input RDD will be cached to speed up index lookup by reducing IO " + + "for computing parallelism or affected partitions"); + + public static final ConfigProperty RECORD_INDEX_INPUT_STORAGE_LEVEL_VALUE = ConfigProperty + .key("hoodie.record.index.input.storage.level") + .defaultValue("MEMORY_AND_DISK_SER") + .markAdvanced() + .withDocumentation("Only applies when #recordIndexUseCaching is set. Determine what level of persistence is used to cache input RDDs. " + + "Refer to org.apache.spark.storage.StorageLevel for different values"); + /** * Deprecated configs. These are now part of {@link HoodieHBaseIndexConfig}. */ @@ -698,6 +714,16 @@ public Builder withRecordKeyField(String keyField) { return this; } + public Builder recordIndexUseCaching(boolean useCaching) { + hoodieIndexConfig.setValue(RECORD_INDEX_USE_CACHING, String.valueOf(useCaching)); + return this; + } + + public Builder withRecordIndexInputStorageLevel(String level) { + hoodieIndexConfig.setValue(RECORD_INDEX_INPUT_STORAGE_LEVEL_VALUE, level); + return this; + } + public HoodieIndexConfig build() { hoodieIndexConfig.setDefaultValue(INDEX_TYPE, getDefaultIndexType(engineType)); hoodieIndexConfig.setDefaults(HoodieIndexConfig.class.getName()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index cd81c4fda071e..eba9728777f19 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1961,6 +1961,10 @@ public String getBucketIndexHashFieldWithDefault() { return getStringOrDefault(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD, getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME)); } + public boolean getRecordIndexUseCaching() { + return getBoolean(HoodieIndexConfig.RECORD_INDEX_USE_CACHING); + } + /** * storage properties. */ @@ -2346,6 +2350,26 @@ public boolean isLogCompactionEnabledOnMetadata() { return getBoolean(HoodieMetadataConfig.ENABLE_LOG_COMPACTION_ON_METADATA_TABLE); } + public boolean isRecordIndexEnabled() { + return metadataConfig.enableRecordIndex(); + } + + public int getRecordIndexMinFileGroupCount() { + return metadataConfig.getRecordIndexMinFileGroupCount(); + } + + public int getRecordIndexMaxFileGroupCount() { + return metadataConfig.getRecordIndexMaxFileGroupCount(); + } + + public float getRecordIndexGrowthFactor() { + return metadataConfig.getRecordIndexGrowthFactor(); + } + + public int getRecordIndexMaxFileGroupSizeBytes() { + return metadataConfig.getRecordIndexMaxFileGroupSizeBytes(); + } + /** * Hoodie Client Lock Configs. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 92ea71c333a19..65bed9242085b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -193,7 +193,12 @@ public enum IndexType { + "choose bucket engine type, i.e., how buckets are generated.") BUCKET, @EnumFieldDescription("Internal Config for indexing based on Flink state.") - FLINK_STATE + FLINK_STATE, + + @EnumFieldDescription("Index which saves the record key to location mappings in the " + + "HUDI Metadata Table. Record index is a global index, enforcing key uniqueness across all " + + "partitions in the table. Supports sharding to achieve very high scale.") + RECORD_INDEX } @EnumDescription("Determines the type of bucketing or hashing to use when `hoodie.index.type`" diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 6653d348882d2..4a63541794365 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -54,6 +54,7 @@ import java.util.List; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getMetadataPartitionsNeedingWriteStatusTracking; /** * Base class for all write operations logically performed at the file group level. @@ -95,12 +96,18 @@ protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String this.writeSchema = overriddenSchema.orElseGet(() -> getWriteSchema(config)); this.writeSchemaWithMetaFields = HoodieAvroUtils.addMetadataFields(writeSchema, config.allowOperationMetadataField()); this.timer = HoodieTimer.start(); - this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), - !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); this.taskContextSupplier = taskContextSupplier; this.writeToken = makeWriteToken(); this.schemaOnReadEnabled = !isNullOrEmpty(hoodieTable.getConfig().getInternalSchema()); this.recordMerger = config.getRecordMerger(); + + // We need to track written records within WriteStatus in two cases: + // 1. When the HoodieIndex being used is not implicit with storage + // 2. If any of the metadata table partitions (record index, etc) which require written record tracking are enabled + final boolean trackSuccessRecords = !hoodieTable.getIndex().isImplicitWithStorage() + || getMetadataPartitionsNeedingWriteStatusTracking(config.getMetadataConfig(), hoodieTable.getMetaClient()); + this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), + trackSuccessRecords, config.getWriteStatusFailureFraction()); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 40f5ff4c0a672..aeaf1806936f1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -21,10 +21,10 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; import org.apache.hudi.avro.model.HoodieIndexPlan; -import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.BaseHoodieWriteClient; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.data.HoodieData; @@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -51,6 +52,7 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -59,8 +61,9 @@ import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hadoop.CachingPath; import org.apache.hudi.hadoop.SerializablePath; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -89,8 +92,6 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.getIndexInflightInstant; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeIndexPlan; -import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; -import static org.apache.hudi.metadata.HoodieMetadataWriteUtils.RECORD_KEY_FIELD_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.METADATA_INDEXER_TIME_SUFFIX; @@ -108,6 +109,15 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta public static final String METADATA_COMPACTION_TIME_SUFFIX = "001"; + // Virtual keys support for metadata table. This Field is + // from the metadata payload schema. + private static final String RECORD_KEY_FIELD_NAME = HoodieMetadataPayload.KEY_FIELD_NAME; + + // Average size of a record saved within the record index. + // Record index has a fixed size schema. This has been calculated based on experiments with default settings + // for block size (1MB), compression (GZ) and disabling the hudi metadata fields. + private static final int RECORD_INDEX_AVERAGE_RECORD_SIZE = 48; + protected HoodieWriteConfig metadataWriteConfig; protected HoodieWriteConfig dataWriteConfig; @@ -128,16 +138,13 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta * @param writeConfig Writer config * @param failedWritesCleaningPolicy Cleaning policy on failed writes * @param engineContext Engine context - * @param actionMetadata Optional action metadata to help decide initialize operations - * @param Action metadata types extending Avro generated SpecificRecordBase * @param inflightInstantTimestamp Timestamp of any instant in progress */ - protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, - HoodieWriteConfig writeConfig, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - HoodieEngineContext engineContext, - Option actionMetadata, - Option inflightInstantTimestamp) { + protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext engineContext, + Option inflightInstantTimestamp) { this.dataWriteConfig = writeConfig; this.engineContext = engineContext; this.hadoopConf = new SerializableConfiguration(hadoopConf); @@ -146,29 +153,32 @@ protected HoodieBackedTableMetadataWriter(Configu this.dataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build(); - if (dataMetaClient.getTableConfig().isMetadataTableEnabled() || writeConfig.isMetadataTableEnabled()) { + if (dataMetaClient.getTableConfig().isMetadataTableAvailable() || writeConfig.isMetadataTableEnabled()) { this.metadataWriteConfig = HoodieMetadataWriteUtils.createMetadataWriteConfig(writeConfig, failedWritesCleaningPolicy); try { enablePartitions(); initRegistry(); - initialized = initializeIfNeeded(dataMetaClient, actionMetadata, inflightInstantTimestamp); + initialized = initializeIfNeeded(dataMetaClient, inflightInstantTimestamp); } catch (IOException e) { - LOG.error("Failed to initialize MDT", e); + LOG.error("Failed to initialize metadata table", e); } } + ValidationUtils.checkArgument(!initialized || this.metadata != null, "MDT Reader should have been opened post initialization"); + } - if (initialized) { - // Since the MDT has been initialized, the reader should be available - try { - this.metadata = new HoodieBackedTableMetadata(engineContext, dataWriteConfig.getMetadataConfig(), - dataWriteConfig.getBasePath(), dataWriteConfig.getSpillableMapBasePath()); - this.metadataMetaClient = metadata.getMetadataMetaClient(); - } catch (Exception e) { - throw new HoodieException("Could not open MDT for reads", e); - } + private void initMetadataReader() { + if (this.metadata != null) { + this.metadata.close(); + } + + try { + this.metadata = new HoodieBackedTableMetadata(engineContext, dataWriteConfig.getMetadataConfig(), dataWriteConfig.getBasePath()); + this.metadataMetaClient = metadata.getMetadataMetaClient(); + } catch (Exception e) { + throw new HoodieException("Could not open MDT for reads", e); } } @@ -177,15 +187,18 @@ protected HoodieBackedTableMetadataWriter(Configu */ private void enablePartitions() { final HoodieMetadataConfig metadataConfig = dataWriteConfig.getMetadataConfig(); - if (dataWriteConfig.isMetadataTableEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.FILES)) { + if (dataWriteConfig.isMetadataTableEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.FILES)) { this.enabledPartitionTypes.add(MetadataPartitionType.FILES); } - if (metadataConfig.isBloomFilterIndexEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.BLOOM_FILTERS)) { + if (metadataConfig.isBloomFilterIndexEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.BLOOM_FILTERS)) { this.enabledPartitionTypes.add(MetadataPartitionType.BLOOM_FILTERS); } - if (metadataConfig.isColumnStatsIndexEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.COLUMN_STATS)) { + if (metadataConfig.isColumnStatsIndexEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.COLUMN_STATS)) { this.enabledPartitionTypes.add(MetadataPartitionType.COLUMN_STATS); } + if (dataWriteConfig.isRecordIndexEnabled() || dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.RECORD_INDEX)) { + this.enabledPartitionTypes.add(MetadataPartitionType.RECORD_INDEX); + } } protected abstract void initRegistry(); @@ -206,19 +219,16 @@ public List getEnabledPartitionTypes() { * Initialize the metadata table if needed. * * @param dataMetaClient - meta client for the data table - * @param actionMetadata - optional action metadata * @param inflightInstantTimestamp - timestamp of an instant in progress on the dataset - * @param - action metadata types extending Avro generated SpecificRecordBase * @throws IOException on errors */ - protected boolean initializeIfNeeded(HoodieTableMetaClient dataMetaClient, - Option actionMetadata, - Option inflightInstantTimestamp) throws IOException { + protected boolean initializeIfNeeded(HoodieTableMetaClient dataMetaClient, + Option inflightInstantTimestamp) throws IOException { HoodieTimer timer = HoodieTimer.start(); List partitionsToInit = new ArrayList<>(MetadataPartitionType.values().length); try { - boolean exists = metadataTableExists(dataMetaClient, actionMetadata); + boolean exists = metadataTableExists(dataMetaClient); if (!exists) { // FILES partition is always required partitionsToInit.add(MetadataPartitionType.FILES); @@ -235,7 +245,9 @@ protected boolean initializeIfNeeded(HoodieTableM } if (partitionsToInit.isEmpty()) { - // No partitions to initialize + // No partitions left to initialize, since all the metadata enabled partitions are either initialized before + // or current in the process of initialization. + initMetadataReader(); return true; } @@ -257,9 +269,8 @@ protected boolean initializeIfNeeded(HoodieTableM } } - private boolean metadataTableExists(HoodieTableMetaClient dataMetaClient, - Option actionMetadata) throws IOException { - boolean exists = dataMetaClient.getTableConfig().isMetadataTableEnabled(); + private boolean metadataTableExists(HoodieTableMetaClient dataMetaClient) throws IOException { + boolean exists = dataMetaClient.getTableConfig().isMetadataTableAvailable(); boolean reInitialize = false; // If the un-synced instants have been archived, then @@ -272,14 +283,12 @@ private boolean metadataTableExists(HoodieTableMe metadataMetaClient = initializeMetaClient(); } } catch (TableNotFoundException e) { - // Table not found, initialize the metadata table. - metadataMetaClient = initializeMetaClient(); + return false; } - final Option latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); - reInitialize = isBootstrapNeeded(latestMetadataInstant, actionMetadata); + reInitialize = isBootstrapNeeded(latestMetadataInstant); } if (reInitialize) { @@ -303,8 +312,7 @@ private boolean metadataTableExists(HoodieTableMe * * @return True if the initialization is not needed, False otherwise */ - private boolean isBootstrapNeeded(Option latestMetadataInstant, - Option actionMetadata) { + private boolean isBootstrapNeeded(Option latestMetadataInstant) { if (!latestMetadataInstant.isPresent()) { LOG.warn("Metadata Table will need to be re-initialized as no instants were found"); return true; @@ -315,60 +323,6 @@ private boolean isBootstrapNeeded(Option dataMetaClient.getTableConfig().isMetadataPartitionAvailable((metadataPartition))); metadataMetaClient = initializeMetaClient(); // Get a complete list of files and partitions from the file system or from already initialized FILES partition of MDT - boolean filesPartitionAvailable = dataMetaClient.getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.FILES); List partitionInfoList = filesPartitionAvailable ? listAllPartitionsFromMDT(initializationTime) : listAllPartitionsFromFilesystem(initializationTime); Map> partitionToFilesMap = partitionInfoList.stream() .map(p -> { @@ -408,18 +370,30 @@ private boolean initializeFromFilesystem(String initializationTime, List> fileGroupCountAndRecordsPair; - switch (partitionType) { - case FILES: - fileGroupCountAndRecordsPair = initializeFilesPartition(initializationTime, partitionInfoList); - break; - case BLOOM_FILTERS: - fileGroupCountAndRecordsPair = initializeBloomFiltersPartition(initializationTime, partitionToFilesMap); - break; - case COLUMN_STATS: - fileGroupCountAndRecordsPair = initializeColumnStatsPartition(partitionToFilesMap); - break; - default: - throw new HoodieMetadataException("Unsupported MDT partition type: " + partitionType); + try { + switch (partitionType) { + case FILES: + fileGroupCountAndRecordsPair = initializeFilesPartition(partitionInfoList); + break; + case BLOOM_FILTERS: + fileGroupCountAndRecordsPair = initializeBloomFiltersPartition(initializationTime, partitionToFilesMap); + break; + case COLUMN_STATS: + fileGroupCountAndRecordsPair = initializeColumnStatsPartition(partitionToFilesMap); + break; + case RECORD_INDEX: + fileGroupCountAndRecordsPair = initializeRecordIndexPartition(); + break; + default: + throw new HoodieMetadataException("Unsupported MDT partition type: " + partitionType); + } + } catch (Exception e) { + String metricKey = partitionType.getPartitionPath() + "_" + HoodieMetadataMetrics.BOOTSTRAP_ERR_STR; + metrics.ifPresent(m -> m.setMetric(metricKey, 1)); + LOG.error("Bootstrap on " + partitionType.getPartitionPath() + " partition failed for " + + metadataMetaClient.getBasePath(), e); + throw new HoodieMetadataException(partitionType.getPartitionPath() + + " bootstrap failed for " + metadataMetaClient.getBasePath(), e); } // Generate the file groups @@ -432,6 +406,8 @@ private boolean initializeFromFilesystem(String initializationTime, List> initializeBloomFiltersPartition( return Pair.of(fileGroupCount, records); } - private Pair> initializeFilesPartition(String createInstantTime, List partitionInfoList) { + private Pair> initializeRecordIndexPartition() throws IOException { + final HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(dataMetaClient, + dataMetaClient.getActiveTimeline(), metadata); + + // MOR tables are not supported + if (!dataMetaClient.getTableType().equals(HoodieTableType.COPY_ON_WRITE)) { + throw new HoodieMetadataException("Only COW tables are supported with record index"); + } + + // Collect the list of latest base files present in each partition + List partitions = metadata.getAllPartitionPaths(); + final List> partitionBaseFilePairs = new ArrayList<>(); + for (String partition : partitions) { + partitionBaseFilePairs.addAll(fsView.getLatestBaseFiles(partition) + .map(basefile -> Pair.of(partition, basefile.getFileName())).collect(Collectors.toList())); + } + + LOG.info("Initializing record index from " + partitionBaseFilePairs.size() + " base files in " + + partitions.size() + " partitions"); + + // Collect record keys from the files in parallel + HoodieData records = readRecordKeysFromBaseFiles(engineContext, partitionBaseFilePairs); + records.persist("MEMORY_AND_DISK_SER"); + final long recordCount = records.count(); + + // Initialize the file groups + final int fileGroupCount = HoodieTableMetadataUtil.estimateFileGroupCount(MetadataPartitionType.RECORD_INDEX, recordCount, + RECORD_INDEX_AVERAGE_RECORD_SIZE, dataWriteConfig.getRecordIndexMinFileGroupCount(), + dataWriteConfig.getRecordIndexMaxFileGroupCount(), dataWriteConfig.getRecordIndexGrowthFactor(), + dataWriteConfig.getRecordIndexMaxFileGroupSizeBytes()); + + LOG.info(String.format("Initializing record index with %d mappings and %d file groups.", recordCount, fileGroupCount)); + return Pair.of(fileGroupCount, records); + } + + /** + * Read the record keys from base files in partitions and return records. + */ + private HoodieData readRecordKeysFromBaseFiles(HoodieEngineContext engineContext, + List> partitionBaseFilePairs) { + if (partitionBaseFilePairs.isEmpty()) { + return engineContext.emptyHoodieData(); + } + + engineContext.setJobStatus(this.getClass().getSimpleName(), "Record Index: reading record keys from base files"); + return engineContext.parallelize(partitionBaseFilePairs, partitionBaseFilePairs.size()).flatMap(p -> { + final String partition = p.getKey(); + final String filename = p.getValue(); + Path dataFilePath = new Path(dataWriteConfig.getBasePath(), partition + Path.SEPARATOR + filename); + + final String fileId = FSUtils.getFileId(filename); + final String instantTime = FSUtils.getCommitTime(filename); + HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(HoodieRecord.HoodieRecordType.AVRO).getFileReader(hadoopConf.get(), dataFilePath); + ClosableIterator recordKeyIterator = reader.getRecordKeyIterator(); + + return new ClosableIterator() { + @Override + public void close() { + recordKeyIterator.close(); + } + + @Override + public boolean hasNext() { + return recordKeyIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + return HoodieMetadataPayload.createRecordIndexUpdate(recordKeyIterator.next(), partition, fileId, + instantTime); + } + }; + }); + } + + private Pair> initializeFilesPartition(List partitionInfoList) { // FILES partition uses a single file group final int fileGroupCount = 1; List partitions = partitionInfoList.stream().map(p -> HoodieTableMetadataUtil.getPartitionIdentifier(p.getRelativePath())) .collect(Collectors.toList()); final int totalDataFilesCount = partitionInfoList.stream().mapToInt(DirectoryInfo::getTotalFiles).sum(); - LOG.info("Committing " + partitions.size() + " partitions and " + totalDataFilesCount + " files to metadata"); //pwason reword + LOG.info("Committing total {} partitions and {} files to metadata", partitions.size(), totalDataFilesCount); // Record which saves the list of all partitions HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(partitions); @@ -494,14 +545,8 @@ private Pair> initializeFilesPartition(String engineContext.setJobStatus(this.getClass().getSimpleName(), "Creating records for MDT FILES partition"); HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { Map fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); - // filter for files that are part of the completed commits - Map validFileNameToSizeMap = fileNameToSizeMap.entrySet().stream().filter(fileSizePair -> { - String commitTime = FSUtils.getCommitTime(fileSizePair.getKey()); - return HoodieTimeline.compareTimestamps(commitTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, createInstantTime); - }).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - return HoodieMetadataPayload.createPartitionFilesRecord( - HoodieTableMetadataUtil.getPartitionIdentifier(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty()); + HoodieTableMetadataUtil.getPartitionIdentifier(partitionInfo.getRelativePath()), Option.of(fileNameToSizeMap), Option.empty()); }); ValidationUtils.checkState(fileListRecords.count() == partitions.size()); @@ -602,10 +647,6 @@ private List listAllPartitionsFromFilesystem(String initializatio */ private List listAllPartitionsFromMDT(String initializationTime) throws IOException { List dirinfoList = new LinkedList<>(); - if (metadata == null) { - this.metadata = new HoodieBackedTableMetadata(engineContext, dataWriteConfig.getMetadataConfig(), - dataWriteConfig.getBasePath(), dataWriteConfig.getSpillableMapBasePath()); - } List allPartitionPaths = metadata.getAllPartitionPaths().stream() .map(partitionPath -> dataWriteConfig.getBasePath() + "/" + partitionPath).collect(Collectors.toList()); Map partitionFileMap = metadata.getAllFilesInPartitions(allPartitionPaths); @@ -637,7 +678,7 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata fs.delete(partitionPath, true); ValidationUtils.checkState(!fs.exists(partitionPath), "Failed to delete MDT partition " + metadataPartition); } - } catch (FileNotFoundException e) { + } catch (FileNotFoundException ignored) { // If the partition did not exist yet, it will be created below } @@ -759,7 +800,6 @@ private interface ConvertMetadataFunction { */ private void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction) { Set partitionsToUpdate = getMetadataPartitionsToUpdate(); - Set inflightIndexes = getInflightMetadataPartitions(dataMetaClient.getTableConfig()); if (initialized && metadata != null) { // convert metadata and filter only the entries whose partition path are in partitionsToUpdate @@ -819,9 +859,18 @@ public void buildMetadataPartitions(HoodieEngineContext engineContext, List HoodieTableMetadataUtil.convertMetadataToRecords( - engineContext, commitMetadata, instantTime, getRecordsGenerationParams())); + public void update(HoodieCommitMetadata commitMetadata, HoodieData writeStatus, String instantTime) { + processAndCommit(instantTime, () -> { + Map> partitionToRecordMap = + HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, commitMetadata, instantTime, getRecordsGenerationParams()); + + // Updates for record index are created by parsing the WriteStatus which is a hudi-client object. Hence, we cannot yet move this code + // to the HoodieTableMetadataUtil class in hudi-common. + if (writeStatus != null && !writeStatus.isEmpty()) { + partitionToRecordMap.put(MetadataPartitionType.RECORD_INDEX, getRecordIndexUpdates(writeStatus)); + } + return partitionToRecordMap; + }); closeInternal(); } @@ -911,9 +960,7 @@ public void close() throws Exception { protected void bulkCommit( String instantTime, MetadataPartitionType partitionType, HoodieData records, int fileGroupCount) { - Map> partitionRecordsMap = new HashMap<>(); - partitionRecordsMap.put(partitionType, records); - commit(instantTime, partitionRecordsMap); + commit(instantTime, Collections.singletonMap(partitionType, records)); } /** @@ -983,7 +1030,7 @@ public void performTableServices(Option inFlightInstantTimestamp) { LOG.info("Latest deltacommit time found is " + latestDeltacommitTime + ", running clean operations."); cleanIfNecessary(writeClient, latestDeltacommitTime); - // Do timeline validation before scheduling compaction/logcompaction operations. + // Do timeline validation before scheduling compaction/logCompaction operations. if (validateTimelineBeforeSchedulingCompaction(inFlightInstantTimestamp, latestDeltacommitTime)) { compactIfNecessary(writeClient, latestDeltacommitTime); } @@ -1093,19 +1140,64 @@ protected boolean validateTimelineBeforeSchedulingCompaction(Option inFl } // Check if there are any pending compaction or log compaction instants in the timeline. - // If pending compact/logcompaction operations are found abort scheduling new compaction/logcompaction operations. + // If pending compact/logCompaction operations are found abort scheduling new compaction/logCompaction operations. Option pendingLogCompactionInstant = metadataMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline().firstInstant(); Option pendingCompactionInstant = metadataMetaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); if (pendingLogCompactionInstant.isPresent() || pendingCompactionInstant.isPresent()) { - LOG.warn(String.format("Not scheduling compaction or logcompaction, since a pending compaction instant %s or logcompaction %s instant is present", + LOG.warn(String.format("Not scheduling compaction or logCompaction, since a pending compaction instant %s or logCompaction %s instant is present", pendingCompactionInstant, pendingLogCompactionInstant)); return false; } return true; } + /** + * Return records that represent update to the record index due to write operation on the dataset. + * + * @param writeStatuses {@code WriteStatus} from the write operation + */ + private HoodieData getRecordIndexUpdates(HoodieData writeStatuses) { + return writeStatuses.flatMap(writeStatus -> { + List recordList = new LinkedList<>(); + for (HoodieRecord writtenRecord : writeStatus.getWrittenRecords()) { + if (!writeStatus.isErrored(writtenRecord.getKey())) { + HoodieRecord hoodieRecord; + HoodieKey key = writtenRecord.getKey(); + Option newLocation = writtenRecord.getNewLocation(); + if (newLocation.isPresent()) { + if (writtenRecord.getCurrentLocation() != null) { + // This is an update, no need to update index if the location has not changed + // newLocation should have the same fileID as currentLocation. The instantTimes differ as newLocation's + // instantTime refers to the current commit which was completed. + if (!writtenRecord.getCurrentLocation().getFileId().equals(newLocation.get().getFileId())) { + final String msg = String.format("Detected update in location of record with key %s from %s " + + " to %s. The fileID should not change.", + writtenRecord.getKey(), writtenRecord.getCurrentLocation(), newLocation.get()); + LOG.error(msg); + throw new HoodieMetadataException(msg); + } else { + // TODO: This may be required for clustering use-cases where record location changes + continue; + } + } + + hoodieRecord = HoodieMetadataPayload.createRecordIndexUpdate(key.getRecordKey(), key.getPartitionPath(), + newLocation.get().getFileId(), newLocation.get().getInstantTime()); + } else { + // Delete existing index for a deleted record + hoodieRecord = HoodieMetadataPayload.createRecordIndexDelete(key.getRecordKey()); + } + + recordList.add(hoodieRecord); + } + } + + return recordList.iterator(); + }); + } + protected void closeInternal() { try { close(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index d8e1779f1bd15..b809a4999e400 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -23,6 +23,8 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.BaseHoodieWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.util.Option; @@ -58,7 +60,7 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { * @param commitMetadata commit metadata of the operation of interest. * @param instantTime instant time of the commit. */ - void update(HoodieCommitMetadata commitMetadata, String instantTime); + void update(HoodieCommitMetadata commitMetadata, HoodieData writeStatuses, String instantTime); /** * Update the metadata table due to a CLEAN operation. 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 3abbfcdedb456..d31a5c44e0781 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 @@ -54,7 +54,6 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; @@ -83,7 +82,6 @@ import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.avro.Schema; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -146,8 +144,7 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(config.getMetadataConfig().getProps()) .build(); - this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(), - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue()); + this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath()); this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata); this.metaClient = metaClient; @@ -891,8 +888,7 @@ public HoodieEngineContext getContext() { * @return instance of {@link HoodieTableMetadataWriter} */ public final Option getMetadataWriter(String triggeringInstantTimestamp) { - return getMetadataWriter( - triggeringInstantTimestamp, EAGER, Option.empty()); + return getMetadataWriter(triggeringInstantTimestamp, EAGER); } /** @@ -922,22 +918,15 @@ public boolean isTableServiceAction(String actionType, String instantTime) { * @return An instance of {@link HoodieTableMetadataWriter}. */ public Option getIndexingMetadataWriter(String triggeringInstantTimestamp) { - return getMetadataWriter(triggeringInstantTimestamp, LAZY, Option.empty()); + return getMetadataWriter(triggeringInstantTimestamp, LAZY); } /** * Gets the metadata writer for regular writes. * * @param triggeringInstantTimestamp The instant that is triggering this metadata write. - * @param actionMetadata Optional action metadata. - * @param Action metadata type. * @return An instance of {@link HoodieTableMetadataWriter}. */ - public Option getMetadataWriter( - String triggeringInstantTimestamp, Option actionMetadata) { - return getMetadataWriter(triggeringInstantTimestamp, EAGER, actionMetadata); - } - /** * Get Table metadata writer. *

    @@ -952,10 +941,9 @@ public Option getMetad * @param failedWritesCleaningPolicy Cleaning policy on failed writes * @return instance of {@link HoodieTableMetadataWriter} */ - protected Option getMetadataWriter( + protected Option getMetadataWriter( String triggeringInstantTimestamp, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - Option actionMetadata) { + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy) { // Each engine is expected to override this and // provide the actual metadata writer, if enabled. return Option.empty(); @@ -1042,7 +1030,7 @@ private void clearMetadataTablePartitionsConfig(Option pa LOG.info("Clear hoodie.table.metadata.partitions in hoodie.properties"); metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), EMPTY_STRING); HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); - } else if (partitions.remove(partitionType.get().getPartitionPath())) { + } else if (partitionType.isPresent() && partitions.remove(partitionType.get().getPartitionPath())) { metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", partitions)); HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index 19c3d0ed38698..1800031f1a8c3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -24,9 +24,10 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -53,10 +54,11 @@ public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, /** * Writes commits metadata to table metadata. + * * @param metadata commit metadata of interest. */ - protected final void writeTableMetadata(HoodieCommitMetadata metadata, String actionType) { - table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, instantTime)); + protected final void writeTableMetadata(HoodieCommitMetadata metadata, HoodieData writeStatus, String actionType) { + table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, writeStatus, instantTime)); } /** @@ -72,7 +74,7 @@ protected final void writeTableMetadata(HoodieCleanMetadata metadata, String ins * @param metadata rollback metadata of interest. */ protected final void writeTableMetadata(HoodieRollbackMetadata metadata) { - table.getMetadataWriter(instantTime, Option.of(metadata)).ifPresent(w -> w.update(metadata, instantTime)); + table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, instantTime)); } /** @@ -80,6 +82,6 @@ protected final void writeTableMetadata(HoodieRollbackMetadata metadata) { * @param metadata restore metadata of interest. */ protected final void writeTableMetadata(HoodieRestoreMetadata metadata) { - table.getMetadataWriter(instantTime, Option.of(metadata)).ifPresent(w -> w.update(metadata, instantTime)); + table.getMetadataWriter(instantTime).ifPresent(w -> w.update(metadata, instantTime)); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index f579224ff9b89..c04f1ba8f2147 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -261,7 +261,7 @@ public HoodieCleanMetadata execute() { } } table.getMetaClient().reloadActiveTimeline(); - if (config.isMetadataTableEnabled()) { + if (table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { table.getHoodieView().sync(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index d855edc3569ce..be949fedb37e9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -105,7 +105,7 @@ public CleanPlanner(HoodieEngineContext context, HoodieTable hoodieT .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); // load all partitions in advance if necessary. - if (shouldUseBatchLookup(config)) { + if (shouldUseBatchLookup(hoodieTable.getMetaClient().getTableConfig(), config)) { LOG.info("Load all partitions and files into file system view in advance."); fileSystemView.loadAllPartitions(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index b68630916b55a..1e1a53942e4c8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableConfig; @@ -35,7 +36,6 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; @@ -103,8 +103,6 @@ public RunIndexActionExecutor(HoodieEngineContext context, HoodieWriteConfig con @Override public Option execute() { - HoodieTimer indexTimer = HoodieTimer.start(); - HoodieInstant indexInstant = validateAndGetIndexInstant(); // read HoodieIndexPlan HoodieIndexPlan indexPlan; @@ -298,15 +296,8 @@ private static List getCompletedArchivedAndActiveInstantsAfter(St } private void updateMetadataPartitionsTableConfig(HoodieTableMetaClient metaClient, Set metadataPartitions) { - // remove from inflight and update completed indexes - Set inflightPartitions = getInflightMetadataPartitions(metaClient.getTableConfig()); - Set completedPartitions = metaClient.getTableConfig().getMetadataPartitions(); - inflightPartitions.removeAll(metadataPartitions); - completedPartitions.addAll(metadataPartitions); - // update table config - metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); - metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); - HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); + metadataPartitions.forEach(metadataPartition -> metaClient.getTableConfig().setMetadataPartitionState( + metaClient, MetadataPartitionType.valueOf(metadataPartition.toUpperCase(Locale.ROOT)), true)); } /** @@ -351,6 +342,7 @@ public void run() { .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); instant = currentInstant.orElse(instant); // so that timeline is not reloaded very frequently + // TODO: HUDI-6371: this does not handle the case that the commit has indeed failed. Maybe use HB detection here. Thread.sleep(TIMELINE_RELOAD_INTERVAL_MILLIS); } catch (InterruptedException e) { throw new HoodieIndexException(String.format("Thread interrupted while running indexing check for instant: %s", instant), e); @@ -374,10 +366,14 @@ public void run() { case HoodieTimeline.COMMIT_ACTION: case HoodieTimeline.DELTA_COMMIT_ACTION: case HoodieTimeline.REPLACE_COMMIT_ACTION: + // TODO: HUDI-6372: Record index requires WriteStatus which cannot be read from the HoodieCommitMetadata. So if the original commit has not + // written to the MDT then we cannot sync that commit here. + if (metaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.RECORD_INDEX)) { + throw new HoodieIndexException(String.format("Cannot sync completed instant %s to metadata table as record index is enabled", instant)); + } HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); - // do not trigger any table service as partition is not fully built out yet - metadataWriter.update(commitMetadata, instant.getTimestamp()); + metadataWriter.update(commitMetadata, HoodieListData.eager(Collections.emptyList()), instant.getTimestamp()); break; case CLEAN_ACTION: HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); 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 c59ebcbfcd98f..29da31b478cbb 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 @@ -108,7 +108,7 @@ public HoodieSavepointMetadata execute() { // (2) using direct file system listing: we parallelize the partition listing so that // each partition can be listed on the file system concurrently through Spark. // Note that - if (shouldUseBatchLookup(config)) { + if (shouldUseBatchLookup(table.getMetaClient().getTableConfig(), config)) { latestFilesMap = view.getAllLatestFileSlicesBeforeOrOn(instantTime).entrySet().stream() .collect(Collectors.toMap( Map.Entry::getKey, diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java index d5e453fbd4a6b..f586a375e3ccf 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java @@ -22,6 +22,7 @@ import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.WriteOperationType; @@ -33,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -78,7 +80,7 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy partitionToFilesNameLengthMap, bootstrap, createInflightCommit); if (writer != null && !createInflightCommit) { writer.performTableServices(Option.of(commitTime)); - writer.update(commitMetadata, commitTime); + writer.update(commitMetadata, HoodieListData.eager(Collections.emptyList()), commitTime); } return commitMetadata; } @@ -87,7 +89,7 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { super.moveInflightCommitToComplete(instantTime, metadata); if (writer != null) { - writer.update(metadata, instantTime); + writer.update(metadata, HoodieListData.eager(Collections.emptyList()), instantTime); } return this; } @@ -95,7 +97,7 @@ public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCo public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata, boolean ignoreWriter) throws IOException { super.moveInflightCommitToComplete(instantTime, metadata); if (!ignoreWriter && writer != null) { - writer.update(metadata, instantTime); + writer.update(metadata, HoodieListData.eager(Collections.emptyList()), instantTime); } return this; } @@ -104,7 +106,7 @@ public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCo public HoodieTestTable moveInflightCompactionToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { super.moveInflightCompactionToComplete(instantTime, metadata); if (writer != null) { - writer.update(metadata, instantTime); + writer.update(metadata, HoodieListData.eager(Collections.emptyList()), instantTime); } return this; } @@ -121,7 +123,7 @@ public HoodieCleanMetadata doClean(String commitTime, Map parti public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception { super.addCompaction(instantTime, commitMetadata); if (writer != null) { - writer.update(commitMetadata, instantTime); + writer.update(commitMetadata, HoodieListData.eager(Collections.emptyList()), instantTime); } return this; } @@ -152,7 +154,7 @@ public HoodieTestTable addReplaceCommit( HoodieReplaceCommitMetadata completeReplaceMetadata) throws Exception { super.addReplaceCommit(instantTime, requestedReplaceMetadata, inflightReplaceMetadata, completeReplaceMetadata); if (writer != null) { - writer.update(completeReplaceMetadata, instantTime); + writer.update(completeReplaceMetadata, HoodieListData.eager(Collections.emptyList()), instantTime); } return this; } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java index b2cf06bf96f97..f21b7241435b2 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -96,7 +97,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable tab // commit to data table after committing to metadata table. // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. - writeTableMetadata(table, compactionCommitTime, compactionInstant.getAction(), metadata); + writeTableMetadata(table, compactionCommitTime, compactionInstant.getAction(), metadata, context.emptyHoodieData()); LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); } finally { @@ -166,7 +167,7 @@ protected void completeLogCompaction(HoodieCommitMetadata metadata, preCommit(metadata); finalizeWrite(table, logCompactionCommitTime, writeStats); // commit to data table after committing to metadata table. - writeTableMetadata(table, logCompactionCommitTime, HoodieTimeline.LOG_COMPACTION_ACTION, metadata); + writeTableMetadata(table, logCompactionCommitTime, HoodieTimeline.LOG_COMPACTION_ACTION, metadata, context.emptyHoodieData()); LOG.info("Committing Log Compaction " + logCompactionCommitTime + ". Finished with result " + metadata); CompactHelpers.getInstance().completeInflightLogCompaction(table, logCompactionCommitTime, metadata); } finally { @@ -207,7 +208,8 @@ protected void completeClustering( // commit to data table after committing to metadata table. // We take the lock here to ensure all writes to metadata table happens within a single lock (single writer). // Because more than one write to metadata table will result in conflicts since all of them updates the same partition. - writeTableMetadata(table, clusteringCommitTime, clusteringInstant.getAction(), metadata); + writeTableMetadata(table, clusteringCommitTime, clusteringInstant.getAction(), metadata, context.emptyHoodieData()); + LOG.info("Committing Clustering {} finished with result {}.", clusteringCommitTime, metadata); table.getActiveTimeline().transitionReplaceInflightToComplete( HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), @@ -249,9 +251,9 @@ public HoodieFlinkTable getHoodieTable() { } @Override - public void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { + public void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata, HoodieData writeStatuses) { try (HoodieBackedTableMetadataWriter metadataWriter = initMetadataWriter(Option.empty())) { - metadataWriter.update(metadata, instantTime); + metadataWriter.update(metadata, writeStatuses, instantTime); } catch (Exception e) { throw new HoodieException("Failed to update metadata", e); } @@ -271,7 +273,7 @@ protected void preCommit(HoodieCommitMetadata metadata) { */ private HoodieBackedTableMetadataWriter initMetadataWriter(Option latestPendingInstant) { return (HoodieBackedTableMetadataWriter) FlinkHoodieBackedTableMetadataWriter.create( - FlinkClientUtil.getHadoopConf(), this.config, HoodieFlinkEngineContext.DEFAULT, Option.empty(), latestPendingInstant); + FlinkClientUtil.getHadoopConf(), this.config, HoodieFlinkEngineContext.DEFAULT, latestPendingInstant); } public void initMetadataTable() { 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 49d873719b6bc..f30af6d8892e6 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 @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.client.utils.TransactionUtils; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; @@ -109,7 +110,7 @@ public boolean commit(String instantTime, List writeStatuses, Optio .values().stream() .map(duplicates -> duplicates.stream().reduce(WriteStatMerger::merge).get()) .collect(Collectors.toList()); - return commitStats(instantTime, merged, extraMetadata, commitActionType, partitionToReplacedFileIds, extraPreCommitFunc); + return commitStats(instantTime, HoodieListData.eager(writeStatuses), merged, extraMetadata, commitActionType, partitionToReplacedFileIds, extraPreCommitFunc); } @Override @@ -303,8 +304,8 @@ public void preTxn(HoodieTableMetaClient metaClient) { } @Override - protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { - tableServiceClient.writeTableMetadata(table, instantTime, actionType, metadata); + protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata, HoodieData writeStatuses) { + tableServiceClient.writeTableMetadata(table, instantTime, actionType, metadata, writeStatuses); } /** diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 8f13315245b1b..7dd32e2916ec7 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -34,11 +34,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.conf.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -51,48 +47,36 @@ */ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { - private static final Logger LOG = LoggerFactory.getLogger(FlinkHoodieBackedTableMetadataWriter.class); - private transient BaseHoodieWriteClient writeClient; public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) { - return create(conf, writeConfig, context, Option.empty()); - } - - public static HoodieTableMetadataWriter create(Configuration conf, - HoodieWriteConfig writeConfig, - HoodieEngineContext context, - Option actionMetadata) { - return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, EAGER, context, actionMetadata, Option.empty()); + return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, EAGER, context, Option.empty()); } - public static HoodieTableMetadataWriter create(Configuration conf, - HoodieWriteConfig writeConfig, - HoodieEngineContext context, - Option actionMetadata, - Option inFlightInstantTimestamp) { + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieEngineContext context, + Option inFlightInstantTimestamp) { return new FlinkHoodieBackedTableMetadataWriter( - conf, writeConfig, EAGER, context, actionMetadata, inFlightInstantTimestamp); + conf, writeConfig, EAGER, context, inFlightInstantTimestamp); } - public static HoodieTableMetadataWriter create(Configuration conf, - HoodieWriteConfig writeConfig, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - HoodieEngineContext context, - Option actionMetadata, - Option inFlightInstantTimestamp) { + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext context, + Option inFlightInstantTimestamp) { return new FlinkHoodieBackedTableMetadataWriter( - conf, writeConfig, failedWritesCleaningPolicy, context, actionMetadata, inFlightInstantTimestamp); + conf, writeConfig, failedWritesCleaningPolicy, context, inFlightInstantTimestamp); } - FlinkHoodieBackedTableMetadataWriter(Configuration hadoopConf, - HoodieWriteConfig writeConfig, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - HoodieEngineContext engineContext, - Option actionMetadata, - Option inFlightInstantTimestamp) { - super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, actionMetadata, inFlightInstantTimestamp); + FlinkHoodieBackedTableMetadataWriter(Configuration hadoopConf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext engineContext, + Option inFlightInstantTimestamp) { + super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, inFlightInstantTimestamp); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 1314c8596c8d5..430062238a178 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -39,8 +39,6 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.avro.specific.SpecificRecordBase; - import java.util.List; /** @@ -99,14 +97,13 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con * @return instance of {@link HoodieTableMetadataWriter} */ @Override - protected Option getMetadataWriter( + protected Option getMetadataWriter( String triggeringInstantTimestamp, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - Option actionMetadata) { - if (config.isMetadataTableEnabled()) { + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy) { + if (config.isMetadataTableEnabled() || getMetaClient().getTableConfig().isMetadataTableAvailable()) { return Option.of(FlinkHoodieBackedTableMetadataWriter.create( context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context, - actionMetadata, Option.of(triggeringInstantTimestamp))); + Option.of(triggeringInstantTimestamp))); } else { return Option.empty(); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java index bbb4034dff3bc..a44ca7ee419b2 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -19,6 +19,8 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -129,7 +131,7 @@ protected String getCommitActionType() { @Override protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { - commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList())); + commit(extraMetadata, HoodieListData.eager(result.getWriteStatuses()), result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList())); } protected void setCommitMetadata(HoodieWriteMetadata> result) { @@ -138,7 +140,8 @@ protected void setCommitMetadata(HoodieWriteMetadata> result) extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()))); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { + protected void commit(Option> extraMetadata, HoodieData writeStatuses, HoodieWriteMetadata> result, + List writeStats) { String actionType = getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); result.setCommitted(true); @@ -150,7 +153,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = result.getCommitMetadata().get(); - writeTableMetadata(metadata, actionType); + writeTableMetadata(metadata, writeStatuses, actionType); activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index 997dd5d84e361..118685fb29a18 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -87,7 +87,7 @@ public boolean commit(String instantTime, Map> partitionToReplacedFileIds, Option> extraPreCommitFunc) { List writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()); - return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds, + return commitStats(instantTime, HoodieListData.eager(writeStatuses), writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds, extraPreCommitFunc); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index b4eca05abdbd5..668a3e2e56b6d 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -208,7 +208,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = result.getCommitMetadata().get(); - writeTableMetadata(metadata, actionType); + writeTableMetadata(metadata, HoodieListData.eager(result.getWriteStatuses()), actionType); activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java index c0fb8a723516e..7836f484981e3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java @@ -40,7 +40,6 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieLogCompactException; import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -202,7 +201,7 @@ public HoodieWriteMetadata> cluster(String clusteringInstan LOG.info("Starting clustering at " + clusteringInstant); HoodieWriteMetadata> writeMetadata = table.cluster(context, clusteringInstant); HoodieWriteMetadata> clusteringMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses())); - // Validation has to be done after cloning. if not, it could result in dereferencing the write status twice which means clustering could get executed twice. + // Validation has to be done after cloning. if not, it could result in referencing the write status twice which means clustering could get executed twice. validateClusteringCommit(clusteringMetadata, clusteringInstant, table); // Publish file creation metrics for clustering. @@ -301,24 +300,10 @@ private void validateClusteringCommit(HoodieWriteMetadata> private void updateTableMetadata(HoodieTable table, HoodieCommitMetadata commitMetadata, HoodieInstant hoodieInstant) { - boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction(), hoodieInstant.getTimestamp()); // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. table.getMetadataWriter(hoodieInstant.getTimestamp()) - .ifPresent(writer -> ((HoodieTableMetadataWriter) writer).update(commitMetadata, hoodieInstant.getTimestamp())); - } - - /** - * Initialize the metadata table if needed. Creating the metadata table writer - * will trigger the initial bootstrapping from the data table. - * - * @param inFlightInstantTimestamp - The in-flight action responsible for the metadata table initialization - */ - protected void initializeMetadataTable(Option inFlightInstantTimestamp) { - if (config.isMetadataTableEnabled()) { - SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, - context, Option.empty(), inFlightInstantTimestamp); - } + .ifPresent(writer -> ((HoodieTableMetadataWriter) writer).update(commitMetadata, context.emptyHoodieData(), hoodieInstant.getTimestamp())); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index aee2846f14b71..1d82229d5418b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -100,7 +100,7 @@ public boolean commit(String instantTime, JavaRDD writeStatuses, Op Option> extraPreCommitFunc) { context.setJobStatus(this.getClass().getSimpleName(), "Committing stats: " + config.getTableName()); List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); - return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds, extraPreCommitFunc); + return commitStats(instantTime, HoodieJavaRDD.of(writeStatuses), writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds, extraPreCommitFunc); } @Override @@ -343,7 +343,7 @@ private void initializeMetadataTable(Option inFlightInstantTimestamp) { } try (HoodieTableMetadataWriter writer = SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, - context, Option.empty(), inFlightInstantTimestamp)) { + context, inFlightInstantTimestamp)) { if (writer.isInitialized()) { writer.performTableServices(inFlightInstantTimestamp); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java index 03360fd035b76..532187092590d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java @@ -70,6 +70,8 @@ public static HoodieIndex createIndex(HoodieWriteConfig config) { default: throw new HoodieIndexException("Unknown bucket index engine type: " + config.getBucketIndexEngineType()); } + case RECORD_INDEX: + return new SparkMetadataTableRecordIndex(config); default: throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); } @@ -96,6 +98,8 @@ public static boolean isGlobalIndex(HoodieWriteConfig config) { return true; case BUCKET: return false; + case RECORD_INDEX: + return true; default: return createIndex(config).isGlobal(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java new file mode 100644 index 0000000000000..452f1ce20bfdb --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java @@ -0,0 +1,238 @@ +/* + * 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.index; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; +import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import scala.Tuple2; + +/** + * Hoodie Index implementation backed by the record index present in the Metadata Table. + */ +public class SparkMetadataTableRecordIndex extends HoodieIndex { + + private static final Logger LOG = LoggerFactory.getLogger(SparkMetadataTableRecordIndex.class); + // The index to fallback upon when record index is not initialized yet. + // This should be a global index like record index so that the behavior of tagging across partitions is not changed. + private static final HoodieIndex.IndexType FALLBACK_INDEX_TYPE = IndexType.GLOBAL_SIMPLE; + + public SparkMetadataTableRecordIndex(HoodieWriteConfig config) { + super(config); + } + + @Override + public HoodieData> tagLocation(HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { + int fileGroupSize; + try { + ValidationUtils.checkState(hoodieTable.getMetaClient().getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.RECORD_INDEX)); + fileGroupSize = hoodieTable.getMetadataTable().getNumFileGroupsForPartition(MetadataPartitionType.RECORD_INDEX); + ValidationUtils.checkState(fileGroupSize > 0, "Record index should have at least one file group"); + } catch (TableNotFoundException | IllegalStateException e) { + // This means that record index has not been initialized. + LOG.warn(String.format("Record index not initialized so falling back to %s for tagging records", FALLBACK_INDEX_TYPE.name())); + + // Fallback to another index so that tagLocation is still accurate and there are no duplicates. + HoodieWriteConfig otherConfig = HoodieWriteConfig.newBuilder().withProperties(config.getProps()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(FALLBACK_INDEX_TYPE).build()).build(); + HoodieIndex fallbackIndex = SparkHoodieIndexFactory.createIndex(otherConfig); + + // Fallback index needs to be a global index like record index + ValidationUtils.checkArgument(fallbackIndex.isGlobal(), "Fallback index needs to be a global index like record index"); + + return fallbackIndex.tagLocation(records, context, hoodieTable); + } + + final int numFileGroups = fileGroupSize; + + if (config.getRecordIndexUseCaching()) { + records.persist(new HoodieConfig(config.getProps()).getString(HoodieIndexConfig.RECORD_INDEX_INPUT_STORAGE_LEVEL_VALUE)); + } + + // Partition the record keys to lookup such that each partition looks up one record index shard + JavaRDD partitionedKeyRDD = HoodieJavaRDD.getJavaRDD(records) + .map(HoodieRecord::getRecordKey) + .keyBy(k -> HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(k, numFileGroups)) + .partitionBy(new PartitionIdPassthrough(numFileGroups)) + .map(t -> t._2); + ValidationUtils.checkState(partitionedKeyRDD.getNumPartitions() <= numFileGroups); + + // Lookup the keys in the record index + HoodiePairData keyToLocationPairRDD = + HoodieJavaPairRDD.of(partitionedKeyRDD.mapPartitionsToPair(new RecordIndexFileGroupLookupFunction(hoodieTable))); + + // Tag the incoming records, as inserts or updates, by joining with existing record keys + HoodieData> taggedRecords = tagLocationBackToRecords(keyToLocationPairRDD, records); + + // The number of partitions in the taggedRecords is expected to the maximum of the partitions in + // keyToLocationPairRDD and records RDD. + + if (config.getRecordIndexUseCaching()) { + records.unpersist(); + } + + return taggedRecords; + } + + @Override + public HoodieData updateLocation(HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable) { + // This is a no-op as metadata record index updates are automatically maintained within the metadata table. + return writeStatuses; + } + + @Override + public boolean rollbackCommit(String instantTime) { + // Only those deltacommits which have a valid completed commit on the dataset are read. Since, the instantTime + // is being rolled back on the dataset, we will anyway not load the records from the deltacommit. + // + // In other words, there is no need to rollback anything here as the MDT rollback mechanism will take care of it. + return true; + } + + @Override + public boolean isGlobal() { + return true; + } + + @Override + public boolean canIndexLogFiles() { + return false; + } + + @Override + public boolean isImplicitWithStorage() { + return false; + } + + private HoodieData> tagLocationBackToRecords( + HoodiePairData keyFilenamePair, + HoodieData> records) { + HoodiePairData> keyRecordPairs = + records.mapToPair(record -> ImmutablePair.of(record.getRecordKey(), record)); + // Here as the records might have more data than keyFilenamePairs (some row keys' not found in record index), + // we will do left outer join. + return keyRecordPairs.leftOuterJoin(keyFilenamePair).values() + .map(v -> { + HoodieRecord record = v.getLeft(); + Option location = Option.ofNullable(v.getRight().orElse(null)); + if (!location.isPresent()) { + // No location found. + return record; + } + // Ensure the partitionPath is also set correctly in the key + if (!record.getPartitionPath().equals(location.get().getPartitionPath())) { + record = createNewHoodieRecord(record, location.get()); + } + + // Perform the tagging. Not using HoodieIndexUtils.getTaggedRecord to prevent an additional copy which is not necessary for this index. + record.unseal(); + record.setCurrentLocation(location.get()); + record.seal(); + return record; + }); + } + + private HoodieRecord createNewHoodieRecord(HoodieRecord oldRecord, HoodieRecordGlobalLocation location) { + HoodieKey recordKey = new HoodieKey(oldRecord.getRecordKey(), location.getPartitionPath()); + return config.getRecordMerger().getRecordType() == HoodieRecord.HoodieRecordType.AVRO + ? new HoodieAvroRecord(recordKey, (HoodieRecordPayload) oldRecord.getData()) + : ((HoodieSparkRecord) oldRecord).newInstance(); + } + + /** + * Function that lookups a list of keys in a single shard of the record index + */ + private static class RecordIndexFileGroupLookupFunction implements PairFlatMapFunction, String, HoodieRecordGlobalLocation> { + private final HoodieTable hoodieTable; + + public RecordIndexFileGroupLookupFunction(HoodieTable hoodieTable) { + this.hoodieTable = hoodieTable; + } + + @Override + public Iterator> call(Iterator recordKeyIterator) { + List keysToLookup = new ArrayList<>(); + recordKeyIterator.forEachRemaining(keysToLookup::add); + + // recordIndexInfo object only contains records that are present in record_index. + Map recordIndexInfo = hoodieTable.getMetadataTable().readRecordIndex(keysToLookup); + return recordIndexInfo.entrySet().stream() + .map(e -> new Tuple2<>(e.getKey(), e.getValue())).iterator(); + } + } + + /** + * A dummy partitioner for use with records whose partition ids have been pre-computed (i.e. for + * use on RDDs of (Int, Row) pairs where the Int is a partition id in the expected range). + * + * NOTE: This is a workaround for SPARK-39391, which moved the PartitionIdPassthrough from + * {@link org.apache.spark.sql.execution.ShuffledRowRDD} to {@link Partitioner}. + */ + private class PartitionIdPassthrough extends Partitioner { + + private final int numPartitions; + + public PartitionIdPassthrough(int numPartitions) { + this.numPartitions = numPartitions; + } + + @Override + public int numPartitions() { + return numPartitions; + } + + @Override + public int getPartition(Object key) { + return (int) key; + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index bae30d8b67358..2a22eacea8c5a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieFileFormat; @@ -97,6 +98,22 @@ public ClosableIterator> getRecordIterator(Schema read }); } + @Override + public ClosableIterator getRecordKeyIterator() throws IOException { + Schema schema = HoodieAvroUtils.getRecordKeySchema(); + ClosableIterator iterator = getInternalRowIterator(schema, schema); + StructType structType = HoodieInternalRowUtils.getCachedSchema(schema); + UnsafeProjection projection = HoodieInternalRowUtils.getCachedUnsafeProjection(structType, structType); + + return new CloseableMappingIterator<>(iterator, data -> { + // NOTE: We have to do [[UnsafeProjection]] of incoming [[InternalRow]] to convert + // it to [[UnsafeRow]] holding just raw bytes + UnsafeRow unsafeRow = projection.apply(data); + HoodieSparkRecord record = unsafeCast(new HoodieSparkRecord(unsafeRow)); + return record.getRecordKey(); + }); + } + private ClosableIterator getInternalRowIterator(Schema readerSchema, Schema requestedSchema) throws IOException { if (requestedSchema == null) { requestedSchema = readerSchema; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 84e0671f8c776..eab0c436248a3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -37,15 +37,13 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.metrics.DistributedRegistry; - -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.spark.api.java.JavaRDD; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -67,42 +65,38 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad * @param conf * @param writeConfig * @param context - * @param actionMetadata * @param inflightInstantTimestamp Timestamp of an instant which is in-progress. This instant is ignored while * attempting to bootstrap the table. * @return An instance of the {@code HoodieTableMetadataWriter} */ - public static HoodieTableMetadataWriter create(Configuration conf, - HoodieWriteConfig writeConfig, - HoodieEngineContext context, - Option actionMetadata, - Option inflightInstantTimestamp) { + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieEngineContext context, + Option inflightInstantTimestamp) { return new SparkHoodieBackedTableMetadataWriter( - conf, writeConfig, EAGER, context, actionMetadata, inflightInstantTimestamp); + conf, writeConfig, EAGER, context, inflightInstantTimestamp); } - public static HoodieTableMetadataWriter create(Configuration conf, - HoodieWriteConfig writeConfig, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - HoodieEngineContext context, - Option actionMetadata, - Option inflightInstantTimestamp) { + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext context, + Option inflightInstantTimestamp) { return new SparkHoodieBackedTableMetadataWriter( - conf, writeConfig, failedWritesCleaningPolicy, context, actionMetadata, inflightInstantTimestamp); + conf, writeConfig, failedWritesCleaningPolicy, context, inflightInstantTimestamp); } public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) { - return create(conf, writeConfig, context, Option.empty(), Option.empty()); + return create(conf, writeConfig, context, Option.empty()); } - SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf, - HoodieWriteConfig writeConfig, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - HoodieEngineContext engineContext, - Option actionMetadata, - Option inflightInstantTimestamp) { - super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, actionMetadata, inflightInstantTimestamp); + SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext engineContext, + Option inflightInstantTimestamp) { + super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, inflightInstantTimestamp); } @Override @@ -130,10 +124,8 @@ protected void commit(String instantTime, Map records, int fileGroupCount) { - Map> partitionRecordsMap = new HashMap<>(); - partitionRecordsMap.put(partitionType, records); SparkHoodieMetadataBulkInsertPartitioner partitioner = new SparkHoodieMetadataBulkInsertPartitioner(fileGroupCount); - commitInternal(instantTime, partitionRecordsMap, Option.of(partitioner)); + commitInternal(instantTime, Collections.singletonMap(partitionType, records), Option.of(partitioner)); } private void commitInternal(String instantTime, Map> partitionRecordsMap, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java index efdbb8ac3d0b7..e005f0ef7afc8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieMetadataBulkInsertPartitioner.java @@ -24,11 +24,11 @@ import java.util.List; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaRDD; - import scala.Tuple2; /** @@ -89,7 +89,7 @@ public JavaRDD repartitionRecords(JavaRDD records, i // FileGroupPartitioner returns a fixed number of partition as part of numPartitions(). In the special case that recordsRDD has fewer // records than fileGroupCount, some of these partitions (corresponding to fileGroups) will not have any data. // But we still need to return a fileID for use within {@code BulkInsertMapFunction} - fileIds.add(""); + fileIds.add(StringUtils.EMPTY_STRING); } return fileIds.iterator(); }, true).collect(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index d6796a7a4d489..f7d314aa8bb2d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -35,8 +35,6 @@ import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; - -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.fs.Path; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; @@ -90,17 +88,16 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con * @return instance of {@link HoodieTableMetadataWriter} */ @Override - protected Option getMetadataWriter( + protected Option getMetadataWriter( String triggeringInstantTimestamp, - HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, - Option actionMetadata) { - if (config.isMetadataTableEnabled()) { + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy) { + if (config.isMetadataTableEnabled() || metaClient.getTableConfig().isMetadataTableAvailable()) { // Create the metadata table writer. First time after the upgrade this creation might trigger // metadata table bootstrapping. Bootstrapping process could fail and checking the table // existence after the creation is needed. final HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context, - actionMetadata, Option.of(triggeringInstantTimestamp)); + Option.of(triggeringInstantTimestamp)); // even with metadata enabled, some index could have been disabled // delete metadata partitions corresponding to such indexes deleteMetadataIndexIfNecessary(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 820963e87466f..eae577d8b9dab 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -251,7 +251,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); metadata.setOperationType(operationType); - writeTableMetadata(metadata, actionType); + writeTableMetadata(metadata, result.getWriteStatuses(), actionType); try { activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index b8d62760136be..d19dd069ee247 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -307,7 +307,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta try { HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = result.getCommitMetadata().get(); - writeTableMetadata(metadata, actionType); + writeTableMetadata(metadata, result.getWriteStatuses(), actionType); activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); 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 96933c261a819..6b3b7eca95b13 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 @@ -525,9 +525,9 @@ public void testFailedRollbackCommit( testTable.withPartitionMetaFiles(p1, p2, p3) .addCommit(commitTime1) - .withBaseFilesInPartitions(partitionAndFileId1) + .withBaseFilesInPartitions(partitionAndFileId1).getLeft() .addCommit(commitTime2) - .withBaseFilesInPartitions(partitionAndFileId2) + .withBaseFilesInPartitions(partitionAndFileId2).getLeft() .addInflightCommit(commitTime3) .withBaseFilesInPartitions(partitionAndFileId3); @@ -726,9 +726,9 @@ public void testRollbackWithRequestedRollbackPlan(boolean enableMetadataTable, b testTable.withPartitionMetaFiles(p1, p2) .addCommit(commitTime1) - .withBaseFilesInPartitions(partitionAndFileId1) + .withBaseFilesInPartitions(partitionAndFileId1).getLeft() .addCommit(commitTime2) - .withBaseFilesInPartitions(partitionAndFileId2) + .withBaseFilesInPartitions(partitionAndFileId2).getLeft() .addInflightCommit(commitTime3) .withBaseFilesInPartitions(partitionAndFileId3); @@ -814,9 +814,9 @@ public void testFallbackToListingBasedRollbackForCompletedInstant() throws Excep HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, SparkHoodieBackedTableMetadataWriter.create(metaClient.getHadoopConf(), config, context)); testTable.withPartitionMetaFiles(p1, p2, p3) .addCommit(commitTime1) - .withBaseFilesInPartitions(partitionAndFileId1) + .withBaseFilesInPartitions(partitionAndFileId1).getLeft() .addCommit(commitTime2) - .withBaseFilesInPartitions(partitionAndFileId2) + .withBaseFilesInPartitions(partitionAndFileId2).getLeft() .addCommit(commitTime3) .withBaseFilesInPartitions(partitionAndFileId3); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java index 96a519cc11f29..00f5ed1f018a5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.data.HoodieData.HoodieDataCacheKey; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -133,7 +134,7 @@ void testWriteClientReleaseResourcesShouldOnlyUnpersistRelevantRdds(HoodieTableT writeClient.startCommitWithTime(instant1); List writeStatuses = writeClient.insert(writeRecords, instant1).collect(); assertNoWriteErrors(writeStatuses); - writeClient.commitStats(instant1, writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + writeClient.commitStats(instant1, HoodieListData.eager(writeStatuses), writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); writeClient.close(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java index c137154ae0f92..654fea9313884 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.functional; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; @@ -278,7 +279,8 @@ private List writeData(JavaRDD records, String commit } org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); if (doCommit) { - boolean success = writeClient.commitStats(commitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); + boolean success = writeClient.commitStats(commitTime, HoodieListData.eager(writeStatues), writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType()); Assertions.assertTrue(success); } metaClient = HoodieTableMetaClient.reload(metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index a3fd7775229cc..375839c1b730f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -48,6 +48,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -81,11 +82,14 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.metadata.FileSystemBackedTableMetadata; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; +import org.apache.hudi.metadata.HoodieMetadataFileSystemView; import org.apache.hudi.metadata.HoodieMetadataLogRecordReader; import org.apache.hudi.metadata.HoodieMetadataMetrics; import org.apache.hudi.metadata.HoodieMetadataPayload; @@ -152,9 +156,13 @@ import static org.apache.hudi.common.model.WriteOperationType.DELETE; import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; +import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.getNextCommitTime; +import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS; import static org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.METADATA_COMPACTION_TIME_SUFFIX; +import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataTable; import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS; import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; import static org.apache.hudi.metadata.MetadataPartitionType.FILES; @@ -341,6 +349,7 @@ public void testTurnOffMetadataIndexAfterEnable() throws Exception { assertTrue(tableConfig.getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath())); } + @Disabled("HUDI-6397") @Test public void testTurnOffMetadataTableAfterEnable() throws Exception { init(COPY_ON_WRITE, true); @@ -356,7 +365,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { // Sync to metadata table metaClient.reloadActiveTimeline(); HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - Option metadataWriter = table.getMetadataWriter(instant1, Option.of(hoodieCommitMetadata)); + Option metadataWriter = table.getMetadataWriter(instant1); validateMetadata(testTable, true); assertTrue(metadataWriter.isPresent()); @@ -374,7 +383,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { HoodieCommitMetadata hoodieCommitMetadata2 = doWriteOperationWithMeta(testTable, instant2, INSERT); metaClient.reloadActiveTimeline(); HoodieTable table2 = HoodieSparkTable.create(writeConfig2, context, metaClient); - Option metadataWriter2 = table2.getMetadataWriter(instant2, Option.of(hoodieCommitMetadata2)); + Option metadataWriter2 = table2.getMetadataWriter(instant2); assertFalse(metadataWriter2.isPresent()); HoodieTableConfig hoodieTableConfig2 = @@ -382,7 +391,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { assertEquals(Collections.emptySet(), hoodieTableConfig2.getMetadataPartitions()); // Assert metadata table folder is deleted assertFalse(metaClient.getFs().exists( - new Path(HoodieTableMetadata.getMetadataTableBasePath(writeConfig2.getBasePath())))); + new Path(getMetadataTableBasePath(writeConfig2.getBasePath())))); // Enable metadata table again and initialize metadata table through // HoodieTable.getMetadataWriter() function @@ -396,7 +405,7 @@ public void testTurnOffMetadataTableAfterEnable() throws Exception { HoodieCommitMetadata hoodieCommitMetadata3 = doWriteOperationWithMeta(testTable, instant3, INSERT); metaClient.reloadActiveTimeline(); HoodieTable table3 = HoodieSparkTable.create(writeConfig3, context, metaClient); - Option metadataWriter3 = table3.getMetadataWriter(instant3, Option.of(hoodieCommitMetadata3)); + Option metadataWriter3 = table3.getMetadataWriter(instant3); validateMetadata(testTable, true); assertTrue(metadataWriter3.isPresent()); HoodieTableConfig hoodieTableConfig3 = @@ -596,6 +605,7 @@ public void testMetadataInsertUpsertClean(HoodieTableType tableType) throws Exce @Test public void testMetadataInsertUpsertCleanNonPartitioned() throws Exception { init(COPY_ON_WRITE); + testTable.setNonPartitioned(); doWriteOperationNonPartitioned(testTable, "0000001", INSERT); doWriteOperationNonPartitioned(testTable, "0000002", UPSERT); testTable.doCleanBasedOnCommits("0000003", Arrays.asList("0000001")); @@ -618,8 +628,6 @@ public void testInsertUpsertCluster(HoodieTableType tableType) throws Exception /** * Tests that table services in data table won't trigger table services in metadata table. - * - * @throws Exception */ @Test public void testMetadataTableServices() throws Exception { @@ -738,8 +746,6 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws I /** * Tests that virtual key configs are honored in base files after compaction in metadata table. - * - * @throws Exception */ @Test public void testVirtualKeysInBaseFiles() throws Exception { @@ -860,7 +866,7 @@ public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompact validateMetadata(testTable); // Fetch compaction Commit file and rename to some other file. completed compaction meta file should have some serialized info that table interprets // for future upserts. so, renaming the file here to some temp name and later renaming it back to same name. - java.nio.file.Path parentPath = Paths.get(metadataTableBasePath, HoodieTableMetaClient.METAFOLDER_NAME); + java.nio.file.Path parentPath = Paths.get(metadataTableBasePath, METAFOLDER_NAME); java.nio.file.Path metaFilePath = parentPath.resolve(metadataCompactionInstant + HoodieTimeline.COMMIT_EXTENSION); java.nio.file.Path tempFilePath = FileCreateUtils.renameFileToTemp(metaFilePath, metadataCompactionInstant); metaClient.reloadActiveTimeline(); @@ -893,7 +899,7 @@ public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompact // Fetch compaction Commit file and rename to some other file. completed compaction meta file should have some serialized info that table interprets // for future upserts. so, renaming the file here to some temp name and later renaming it back to same name. - parentPath = Paths.get(metadataTableBasePath, HoodieTableMetaClient.METAFOLDER_NAME); + parentPath = Paths.get(metadataTableBasePath, METAFOLDER_NAME); metaFilePath = parentPath.resolve(metadataCompactionInstant + HoodieTimeline.COMMIT_EXTENSION); tempFilePath = FileCreateUtils.renameFileToTemp(metaFilePath, metadataCompactionInstant); @@ -1059,7 +1065,6 @@ public void testMetadataRecordKeyExcludeFromPayload(final HoodieTableType tableT * @param metadataMetaClient - Metadata meta client * @param latestCommitTimestamp - Latest commit timestamp * @param enableMetaFields - Enable meta fields for the table records - * @throws IOException */ private void verifyMetadataRecordKeyExcludeFromPayloadLogFiles(HoodieTable table, HoodieTableMetaClient metadataMetaClient, String latestCommitTimestamp, @@ -1098,7 +1103,6 @@ private void verifyMetadataRecordKeyExcludeFromPayloadLogFiles(HoodieTable table * @param table * @param logFiles - Metadata table log files to be verified * @param enableMetaFields - Enable meta fields for records - * @throws IOException */ private void verifyMetadataRawRecords(HoodieTable table, List logFiles, boolean enableMetaFields) throws IOException { for (HoodieLogFile logFile : logFiles) { @@ -1262,6 +1266,7 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { @Test public void testRollbackOperationsNonPartitioned() throws Exception { init(COPY_ON_WRITE); + testTable.setNonPartitioned(); doWriteInsertAndUpsertNonPartitioned(testTable); // trigger an upsert @@ -1396,6 +1401,9 @@ public void testSync(HoodieTableType tableType) throws Exception { @MethodSource("tableTypeAndEnableOperationArgs") public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType, boolean nonPartitionedDataset) throws Exception { init(tableType, true, true, true, false); + if (nonPartitionedDataset) { + testTable.setNonPartitioned(); + } long baseCommitTime = Long.parseLong(HoodieActiveTimeline.createNewInstantTime()); for (int i = 1; i < 25; i += 7) { long commitTime1 = getNextCommitTime(baseCommitTime); @@ -1419,6 +1427,87 @@ public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType, bool validateMetadata(testTable, emptyList(), nonPartitionedDataset); } + /** + * First attempt at bootstrap failed but the file slices get created. The next bootstrap should continue successfully. + */ + @Test + public void testFailedBootstrap() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Config with 5 fileGroups for record index + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .withEnableRecordIndex(true) + .withRecordIndexFileGroupCount(5, 5) + .build()) + .build(); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + // Write + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 100); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Metadata table should exist + final Path metadataTablePath = new Path(getMetadataTableBasePath(writeConfig.getBasePath())); + assertTrue(fs.exists(metadataTablePath)); + metaClient = HoodieTableMetaClient.reload(metaClient); + assertTrue(metaClient.getTableConfig().isMetadataTableAvailable()); + + // File groups should be created as in the config + HoodieBackedTableMetadata metadataReader = (HoodieBackedTableMetadata) metadata(client); + assertEquals(HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataReader.getMetadataMetaClient(), Option.empty(), + MetadataPartitionType.FILES.getPartitionPath()).size(), 1); + assertEquals(HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataReader.getMetadataMetaClient(), Option.empty(), + MetadataPartitionType.RECORD_INDEX.getPartitionPath()).size(), 5); + } + + // remove the MDT partition from dataset to simulate failed bootstrap + Properties updateProperties = new Properties(); + updateProperties.setProperty(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), ""); + HoodieTableConfig.update(fs, new Path(basePath + Path.SEPARATOR + METAFOLDER_NAME), + updateProperties); + + metaClient = HoodieTableMetaClient.reload(metaClient); + assertFalse(metaClient.getTableConfig().isMetadataTableAvailable()); + + // Config with 3 fileGroups for record index + writeConfig = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .withEnableRecordIndex(true) + .withRecordIndexFileGroupCount(3, 3) + .build()) + .build(); + + // Another commit which should bootstrap the MDT again + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + // Write + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 100); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Metadata table is recreated, during bootstrapping of metadata table. + metaClient = HoodieTableMetaClient.reload(metaClient); + assertTrue(metaClient.getTableConfig().isMetadataTableAvailable()); + validateMetadata(client); + + // File groups should be created as in the config + HoodieBackedTableMetadata metadataReader = (HoodieBackedTableMetadata) metadata(client); + assertEquals(HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataReader.getMetadataMetaClient(), Option.empty(), + MetadataPartitionType.FILES.getPartitionPath()).size(), 1); + assertEquals(HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataReader.getMetadataMetaClient(), Option.empty(), + MetadataPartitionType.RECORD_INDEX.getPartitionPath()).size(), 3); + } + } + // Some operations are not feasible with test table infra. hence using write client to test those cases. /** @@ -1458,8 +1547,6 @@ public void testFirstCommitRollback(HoodieTableType tableType) throws Exception * Tests the metadata payload spurious deletes. * Lets say a commit was applied to metadata table, and later was explicitly got rolled back. Due to spark task failures, there could be more files in rollback * metadata when compared to the original commit metadata. When payload consistency check is enabled, it will throw exception. If not, it will succeed. - * - * @throws Exception */ @ParameterizedTest @ValueSource(booleans = {true, false}) @@ -1508,7 +1595,7 @@ public void testColStatsPrefixLookup() throws IOException { initTimelineService(); initMetaClient(tableType); initTestDataGenerator(); - metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + metadataTableBasePath = getMetadataTableBasePath(basePath); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); // disable small file handling so that every insert goes to a new file group. @@ -1686,7 +1773,6 @@ public void testEagerRollbackinMDT() throws IOException { * * @param engineContext - Engine context * @param writeConfig - Write config - * @throws IOException */ private void testTableOperationsImpl(HoodieSparkEngineContext engineContext, HoodieWriteConfig writeConfig) throws IOException { try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { @@ -1737,6 +1823,7 @@ private void testTableOperationsImpl(HoodieSparkEngineContext engineContext, Hoo records = dataGen.generateUpdates(newCommitTime, 5); writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); assertNoWriteErrors(writeStatuses); + validateMetadata(client); // Compaction if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { @@ -1837,8 +1924,6 @@ public void testMetadataMultiWriter() throws Exception { /** * Tests that when inline cleaning is enabled and with auto commit set to true, there is no double locking. * bcoz, auto clean is triggered within post commit which is already happening within a lock. - * - * @throws Exception */ @Test public void testMultiWriterForDoubleLocking() throws Exception { @@ -1853,6 +1938,7 @@ public void testMultiWriterForDoubleLocking() throws Exception { .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(true).retainCommits(4) .build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).build()) .withAutoCommit(false) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class).build()) @@ -1956,7 +2042,6 @@ public void testReattemptOfFailedClusteringCommit() throws Exception { /** * Validates that if an instant is completed in MDT, but crashed before commiting to DT, MDT compaction should not kick in based on the instant time * since its not complete in DT yet. - * @throws Exception */ @Test public void testMDTCompactionWithFailedCommits() throws Exception { @@ -2041,8 +2126,6 @@ public void testMetadataReadWithNoCompletedCommits() throws Exception { /** * Ensure that the reader only reads completed instants. - * - * @throws IOException */ @Test public void testReader() throws Exception { @@ -2251,12 +2334,9 @@ public void testUpgradeDowngrade() throws IOException { /** * When table needs to be upgraded and when multi writer is enabled, hudi rolls back partial commits. Upgrade itself is happening * within a lock and hence rollback should not lock again. - * - * @throws IOException - * @throws InterruptedException */ @Test - public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, InterruptedException { + public void testRollbackDuringUpgradeForDoubleLocking() throws IOException { init(HoodieTableType.COPY_ON_WRITE, false); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); @@ -2324,8 +2404,6 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException, Inte /** * Tests rollback of a commit which has new partitions which is not present in hudi table prior to the commit being rolled back. - * - * @throws Exception */ @Test public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Exception { @@ -2359,7 +2437,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = HoodieTimeline.makeCommitFileName(newCommitTime); - assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + METAFOLDER_NAME, commitInstantFileName), false)); } @@ -2375,6 +2453,125 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRollbackPendingCommitWithRecordIndex(boolean performUpsert) throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + // Enable Record index and set index type to record index. + Properties props = new Properties(); + props.setProperty(HoodieMetadataConfig.RECORD_INDEX_ENABLE_PROP.key(), "true"); + props.setProperty(HoodieIndexConfig.INDEX_TYPE.key(), "RECORD_INDEX"); + HoodieWriteConfig cfg = getWriteConfigBuilder(true, true, false) + .withProps(props).build(); + // Initialize write client. + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + + // Insert first batch + String commitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // create pending commits scenario by disabling auto commit + HoodieWriteConfig autoCommitDisabled = getWriteConfigBuilder(false, true, false) + .withProps(props).build(); + // initialize second client, that will stop short of producing the commit. + SparkRDDWriteClient client2 = getHoodieWriteClient(autoCommitDisabled); + // Insert second batch + commitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + writeStatuses = client2.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // delete the metadata table partitions to check, whether rollback of pending commit succeeds and + // metadata table partitions are rebootstrapped. + metadataWriter.dropMetadataPartitions(Arrays.asList(MetadataPartitionType.RECORD_INDEX, FILES)); + assertFalse(fs.exists(new Path(getMetadataTableBasePath(basePath) + + Path.SEPARATOR + FILES.getPartitionPath()))); + assertFalse(fs.exists(new Path(getMetadataTableBasePath(basePath) + + Path.SEPARATOR + MetadataPartitionType.RECORD_INDEX.getPartitionPath()))); + + // Insert/upsert third batch of records + client = getHoodieWriteClient(cfg); + commitTime = HoodieActiveTimeline.createNewInstantTime(); + if (performUpsert) { + records = dataGen.generateUpdates(commitTime, 100); + records.addAll(dataGen.generateInserts(commitTime, 20)); + client.startCommitWithTime(commitTime); + writeStatuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect(); + } else { + records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + } + assertNoWriteErrors(writeStatuses); + assertTrue(fs.exists(new Path(basePath + Path.SEPARATOR + METAFOLDER_NAME))); + metaClient = HoodieTableMetaClient.reload(metaClient); + assertFalse(metaClient.getActiveTimeline().filterCompletedInstants().filterCompletedInstants().findInstantsAfterOrEquals(commitTime, 1).empty()); + + assertTrue(fs.exists(new Path(getMetadataTableBasePath(basePath) + + Path.SEPARATOR + FILES.getPartitionPath()))); + assertTrue(fs.exists(new Path(getMetadataTableBasePath(basePath) + + Path.SEPARATOR + MetadataPartitionType.RECORD_INDEX.getPartitionPath()))); + + } + + /** + * Hoodie.properties indicates metadata is enabled, however metadata folder is missing. + */ + @Test + public void testBootstrapWithTableNotFound() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // create initial commit + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false).build(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + // Write + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 1); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + + final Path metadataTablePath = new Path(getMetadataTableBasePath(writeConfig.getBasePath())); + assertTrue(fs.exists(metadataTablePath), "metadata table should exist."); + + deleteMetadataTable(metaClient, context, false); + assertFalse(fs.exists(metadataTablePath), "metadata table should not exist after being deleted."); + + writeConfig = getWriteConfigBuilder(true, true, false).build(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + // Write + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 1); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + } + + // Metadata table is recreated, during bootstrapping of metadata table. + assertTrue(fs.exists(metadataTablePath)); + } + + /** + * Test bootstrap when the dataset is empty + */ + @Test + public void testbootstrapWithEmptyCommit() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, true).build(); + initWriteConfigAndMetatableWriter(writeConfig, true); + testTable.doWriteOperation(HoodieActiveTimeline.createNewInstantTime(), INSERT, Collections.EMPTY_LIST, 0); + syncTableMetadata(writeConfig); + validateMetadata(testTable); + } + @Test public void testDeletePartitions() throws Exception { init(HoodieTableType.COPY_ON_WRITE); @@ -2459,7 +2656,7 @@ public void testErrorCases() throws Exception { // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = HoodieTimeline.makeCommitFileName(newCommitTime); - assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + METAFOLDER_NAME, commitInstantFileName), false)); } @@ -2558,6 +2755,48 @@ public void testGetFileGroupIndexFromFileId() { assertEquals(HoodieTableMetadataUtil.getFileGroupPrefix("some-file-id-2"), "some-file-id-2"); } + @Test + public void testDuplicatesDuringRecordIndexBootstrap() throws Exception { + init(HoodieTableType.COPY_ON_WRITE, true); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + List commitTimestamps = new ArrayList<>(); + HoodieWriteConfig customConfig = getWriteConfigBuilder(true, true, false) + .build(); + + List recordsFirstBatch = new ArrayList<>(); + String firstCommitTime = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, customConfig)) { + // Create a commit, with record index disabled + List insertRecords = dataGen.generateInserts(firstCommitTime, 100); + recordsFirstBatch.addAll(insertRecords); + // To test duplicates during bootstrap, insert duplicates in the first batch. + recordsFirstBatch.addAll(insertRecords); + client.startCommitWithTime(firstCommitTime); + List writeStatuses = client.insert(jsc.parallelize(recordsFirstBatch, 1), firstCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + commitTimestamps.add(firstCommitTime); + } + assertEquals(false, fs.exists(new Path(metaClient.getMetaPath(), "metadata/record_index"))); + + // bootstrap record index + customConfig = getWriteConfigBuilder(false, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .enableMetrics(false) + .ignoreSpuriousDeletes(false) + .withEnableRecordIndex(true) + .build()) + .build(); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, customConfig)) { + // Create a commit, with record index enabled + String secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List recordsSecondBatch = dataGen.generateInserts(secondCommitTime, 100); + client.startCommitWithTime(secondCommitTime); + assertThrows(HoodieException.class, () -> client.insert(jsc.parallelize(recordsSecondBatch, 1), secondCommitTime)); + } + } + private void doPreBootstrapOperations(HoodieTestTable testTable) throws Exception { doPreBootstrapOperations(testTable, "0000001", "0000002"); } @@ -2613,6 +2852,132 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex. .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } + @Test + public void testClusterOperationOnMainTable() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + // Disable small file handling that way multiple files are created for small batches. + Properties props = new Properties(); + props.setProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0"); + HoodieWriteConfig cfg = getWriteConfigBuilder(true, true, false) + .withProps(props).build(); + // Initialize metadata writers. + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + + // Insert one batch 0000001 + String commitTime = "0000001"; + List records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Insert second batch 0000002 + commitTime = "0000002"; + records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Schedule clustering operation 0000003 + HoodieWriteConfig clusterWriteCfg = getWriteConfigBuilder(true, true, false) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withInlineClusteringNumCommits(0) + .build()) + .build(); + SparkRDDWriteClient clusteringClient = getHoodieWriteClient(clusterWriteCfg); + clusteringClient.scheduleTableService("0000003", Option.empty(), TableServiceType.CLUSTER); + + // Execute pending clustering operation + clusteringClient.cluster("0000003", true); + + // verify metadata table + validateMetadata(client); + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTableMetadata tableMetadata = metadata(client); + HoodieMetadataFileSystemView metadataFileSystemView = new HoodieMetadataFileSystemView( + metaClient, metaClient.reloadActiveTimeline(), tableMetadata); + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); + tableMetadata.getAllPartitionPaths().forEach(partition -> { + List fileNamesFromMetadataFileListing = metadataFileSystemView.getLatestBaseFiles(partition) + .map(baseFile -> baseFile.getFileName()) + .sorted() + .collect(Collectors.toList()); + List fileNamesFromBaseFileView = fsView.getLatestBaseFiles(partition) + .map(baseFile -> baseFile.getFileName()) + .sorted() + .collect(Collectors.toList()); + assertEquals(fileNamesFromBaseFileView, fileNamesFromMetadataFileListing); + }); + } + + @Test + public void testOutOfOrderCommits() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + // Disable small file handling that way multiple files are created for small batches. + Properties props = new Properties(); + props.setProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0"); + HoodieWriteConfig cfg = getWriteConfigBuilder(true, true, false) + .withProps(props).build(); + // Initialize metadata writers. + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + + // Insert one batch 0000001 + String commitTime = "0000001"; + List records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Insert second batch 0000002 + commitTime = "0000002"; + records = dataGen.generateInserts(commitTime, 100); + client.startCommitWithTime(commitTime); + writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Schedule clustering operation 0000003 + HoodieWriteConfig clusterWriteCfg = getWriteConfigBuilder(true, true, false) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withInlineClusteringNumCommits(0) + .build()) + .build(); + SparkRDDWriteClient clusteringClient = getHoodieWriteClient(clusterWriteCfg); + clusteringClient.scheduleTableService("0000003", Option.empty(), TableServiceType.CLUSTER); + + // Insert second batch 0000004 + commitTime = "0000004"; + records = dataGen.generateInserts(commitTime, 100); + client = getHoodieWriteClient(cfg); + client.startCommitWithTime(commitTime); + writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // verify metadata table + validateMetadata(client); + + // Execute compaction on metadata table. + SparkHoodieBackedTableMetadataWriter metadataWriter = (SparkHoodieBackedTableMetadataWriter) + SparkHoodieBackedTableMetadataWriter.create(hadoopConf, client.getConfig(), context); + Properties metadataProps = metadataWriter.getWriteConfig().getProps(); + metadataProps.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "3"); + HoodieWriteConfig metadataWriteConfig = HoodieWriteConfig.newBuilder() + .withProperties(metadataProps).build(); + SparkRDDWriteClient metadataWriteClient = new SparkRDDWriteClient(context, metadataWriteConfig, true); + final String compactionInstantTime = HoodieTableMetadataUtil.createCompactionTimestamp(commitTime); + assertTrue(metadataWriteClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())); + metadataWriteClient.compact(compactionInstantTime); + + // verify metadata table + validateMetadata(client); + + // Execute pending clustering operation + clusteringClient = getHoodieWriteClient(clusterWriteCfg); + clusteringClient.cluster("0000003", true); + + // verify metadata table + validateMetadata(client); + } + private void validateMetadata(SparkRDDWriteClient testClient) throws IOException { validateMetadata(testClient, Option.empty()); } @@ -2628,6 +2993,7 @@ private void validateMetadata(SparkRDDWriteClient testClient, Option ign client = testClient; } + metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTableMetadata tableMetadata = metadata(client); assertNotNull(tableMetadata, "MetadataReader should have been initialized"); if (!config.isMetadataTableEnabled()) { @@ -2650,7 +3016,6 @@ private void validateMetadata(SparkRDDWriteClient testClient, Option ign assertTrue(fsPartitions.equals(metadataPartitions), "Partitions should match"); // Files within each partition should match - metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(config, engineContext); TableFileSystemView tableView = table.getHoodieView(); List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList()); @@ -2750,7 +3115,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, HoodieTableMetadata.getMetadataTableBasePath(basePath), + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, getMetadataTableBasePath(basePath), false, false); assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size()); @@ -2818,9 +3183,6 @@ private void verifyMetadataColumnStatsRecords(List logFiles) thro /** * Returns the list of all files in the dataset by iterating over the metadata table. - * - * @throws IOException - * @throws IllegalArgumentException */ private List getAllFiles(HoodieTableMetadata metadata) throws Exception { List allfiles = new LinkedList<>(); @@ -2840,8 +3202,7 @@ private HoodieBackedTableMetadataWriter metadataWriter(SparkRDDWriteClient clien private HoodieTableMetadata metadata(SparkRDDWriteClient client) { HoodieWriteConfig clientConfig = client.getConfig(); - return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getMetadataConfig(), clientConfig.getBasePath(), - clientConfig.getSpillableMapBasePath()); + return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getMetadataConfig(), clientConfig.getBasePath()); } private void changeTableVersion(HoodieTableVersion version) throws IOException { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 418cad3cec4e5..0d601d786b7fe 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -114,7 +114,7 @@ public void testMultiReaderForHoodieBackedTableMetadata(boolean reuse) throws Ex HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE; init(tableType); testTable.doWriteOperation("000001", INSERT, emptyList(), asList("p1"), 1); - HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), reuse); + HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), reuse); assertTrue(tableMetadata.enabled()); List metadataPartitions = tableMetadata.getAllPartitionPaths(); String partition = metadataPartitions.get(0); @@ -157,8 +157,7 @@ private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception private void verifyBaseMetadataTable(boolean reuseMetadataReaders) throws IOException { HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata( - context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), - writeConfig.getSpillableMapBasePath(), reuseMetadataReaders); + context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), reuseMetadataReaders); assertTrue(tableMetadata.enabled()); List fsPartitionPaths = testTable.getAllPartitionPaths(); List fsPartitions = new ArrayList<>(); @@ -197,7 +196,7 @@ public void testMetadataTableKeyGenerator(final HoodieTableType tableType) throw init(tableType); HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, - writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false); + writeConfig.getMetadataConfig(), writeConfig.getBasePath(), false); assertEquals(HoodieTableMetadataKeyGenerator.class.getCanonicalName(), tableMetadata.getMetadataMetaClient().getTableConfig().getKeyGeneratorClassName()); @@ -211,7 +210,7 @@ public void testMetadataTableKeyGenerator(final HoodieTableType tableType) throw public void testNotExistPartition(final HoodieTableType tableType) throws Exception { init(tableType); HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, - writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false); + writeConfig.getMetadataConfig(), writeConfig.getBasePath(), false); FileStatus[] allFilesInPartition = tableMetadata.getAllFilesInPartition(new Path(writeConfig.getBasePath() + "dummy")); assertEquals(allFilesInPartition.length, 0); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 4ad50121e72c6..1edfb2133a628 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; @@ -57,6 +58,7 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner; +import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; @@ -70,6 +72,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.ArrayList; @@ -79,6 +82,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -114,13 +118,17 @@ private static Stream indexTypeParams() { {IndexType.GLOBAL_SIMPLE, false, true}, {IndexType.GLOBAL_SIMPLE, false, false}, {IndexType.BUCKET, false, true}, - {IndexType.BUCKET, false, false} + {IndexType.BUCKET, false, false}, + {IndexType.RECORD_INDEX, false, true}, + {IndexType.RECORD_INDEX, true, true}, + {IndexType.RECORD_INDEX, true, false} }; return Stream.of(data).map(Arguments::of); } private HoodieIndex index; private HoodieWriteConfig config; + private final Random random = new Random(); private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception { setUp(indexType, populateMetaFields, enableMetadataIndex, true); @@ -135,6 +143,14 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean enab HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(indexType) .fromProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withIndexType(indexType); + + HoodieMetadataConfig.Builder metadataConfigBuilder = HoodieMetadataConfig.newBuilder() + .withMetadataIndexBloomFilter(enableMetadataIndex) + .withMetadataIndexColumnStats(enableMetadataIndex); + if (indexType == IndexType.RECORD_INDEX) { + metadataConfigBuilder.withEnableRecordIndex(true); + } + config = getConfigBuilder() .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withSchema(RawTripTestPayload.JSON_DATA_SCHEMA_STR) @@ -144,10 +160,7 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean enab .withRollbackUsingMarkers(rollbackUsingMarkers) .withIndexConfig(indexBuilder.build()) .withAutoCommit(false) - .withMetadataConfig(HoodieMetadataConfig.newBuilder() - .withMetadataIndexBloomFilter(enableMetadataIndex) - .withMetadataIndexColumnStats(enableMetadataIndex) - .build()) + .withMetadataConfig(metadataConfigBuilder.build()) .withLayoutConfig(HoodieLayoutConfig.newBuilder().fromProperties(indexBuilder.build().getProps()) .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build()) .build(); @@ -185,11 +198,35 @@ private static List getInserts() throws IOException { new RawTripTestPayload(recordStr4).toHoodieRecord()); } + private static List getInsertsBatch2() throws IOException { + String recordStr1 = "{\"_row_key\":\"005\",\"time\":\"2016-01-31T00:00:01.000Z\",\"number\":5}"; + String recordStr2 = "{\"_row_key\":\"006\",\"time\":\"2016-01-31T00:00:02.000Z\",\"number\":6}"; + String recordStr3 = "{\"_row_key\":\"007\",\"time\":\"2016-01-31T00:00:03.000Z\",\"number\":7}"; + String recordStr4 = "{\"_row_key\":\"008\",\"time\":\"2017-01-31T00:00:04.000Z\",\"number\":8}"; + return Arrays.asList( + new RawTripTestPayload(recordStr1).toHoodieRecord(), + new RawTripTestPayload(recordStr2).toHoodieRecord(), + new RawTripTestPayload(recordStr3).toHoodieRecord(), + new RawTripTestPayload(recordStr4).toHoodieRecord()); + } + + private static List getUpdates() throws IOException { + String recordStr1 = "{\"_row_key\":\"001\",\"time\":\"2016-01-31T00:00:01.000Z\",\"number\":5}"; + String recordStr2 = "{\"_row_key\":\"002\",\"time\":\"2016-01-31T00:00:02.000Z\",\"number\":6}"; + String recordStr3 = "{\"_row_key\":\"003\",\"time\":\"2016-01-31T00:00:03.000Z\",\"number\":7}"; + String recordStr4 = "{\"_row_key\":\"004\",\"time\":\"2017-01-31T00:00:04.000Z\",\"number\":8}"; + return new ArrayList<>(Arrays.asList( + new RawTripTestPayload(recordStr1).toHoodieRecord(), + new RawTripTestPayload(recordStr2).toHoodieRecord(), + new RawTripTestPayload(recordStr3).toHoodieRecord(), + new RawTripTestPayload(recordStr4).toHoodieRecord())); + } + @ParameterizedTest @MethodSource("indexTypeParams") public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception { setUp(indexType, populateMetaFields, enableMetadataIndex); - String newCommitTime = "001"; + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); final int totalRecords = 4; List records = getInserts(); JavaRDD writeRecords = jsc.parallelize(records, 1); @@ -295,7 +332,7 @@ public void testLookupIndexWithOrWithoutColumnStats() throws Exception { @MethodSource("indexTypeParams") public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception { setUp(indexType, populateMetaFields, enableMetadataIndex); - String newCommitTime = "001"; + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); final int totalRecords = 4; List records = getInserts(); JavaRDD writeRecords = jsc.parallelize(records, 1); @@ -310,7 +347,7 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not // recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent // upsert will not run into conflicts. - metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight")); + metaClient.getFs().delete(new Path(metaClient.getMetaPath(), newCommitTime + ".inflight")); writeClient.upsert(writeRecords, newCommitTime); assertNoWriteErrors(writeStatues.collect()); @@ -360,7 +397,7 @@ public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, bool writeClient.commit(newCommitTime, jsc.parallelize(writeStatuses)); List fileIds = writeStatuses.stream().map(WriteStatus::getFileId).collect(Collectors.toList()); - + metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should tag them @@ -644,6 +681,75 @@ public void testCheckIfValidCommit() throws Exception { assertFalse(HoodieIndexUtils.checkIfValidCommit(timeline, instantTimestampSec)); } + @ParameterizedTest + @ValueSource(booleans = {true}) + public void testRecordIndexTagLocationAndUpdate(boolean populateMetaFields) throws Exception { + setUp(IndexType.RECORD_INDEX, populateMetaFields, false); + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // int initialRecords = 10 + random.nextInt(20); + List records = getInserts(); + int initialRecords = records.size(); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + + // Test tagLocation without any entries in index + JavaRDD javaRDD = tagLocation(hoodieTable.getIndex(), writeRecords, hoodieTable); + assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); + + // Insert totalRecords records + writeClient.startCommitWithTime(newCommitTime); + JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); + Assertions.assertNoWriteErrors(writeStatues.collect()); + + // Now tagLocation for these records, index should not tag them since it was a failed + // commit + javaRDD = tagLocation(hoodieTable.getIndex(), writeRecords, hoodieTable); + assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0); + // Now commit this & update location of records inserted and validate no errors + writeClient.commit(newCommitTime, writeStatues); + + // Create new commit time. + String secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Now tagLocation for these records, index should tag them correctly + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + + // Generate updates for all existing records. + List newRecords = getUpdates(); + // int newInsertsCount = 10; + List newInserts = getInsertsBatch2(); + int newInsertsCount = newInserts.size(); + newRecords.addAll(newInserts); + // Update partitionPath information. + String newPartitionPath = "2022/11/04"; + newRecords = newRecords.stream() + .map(rec -> new HoodieAvroRecord(new HoodieKey(rec.getRecordKey(), newPartitionPath), (HoodieRecordPayload) rec.getData())) + .collect(Collectors.toList()); + JavaRDD newWriteRecords = jsc.parallelize(newRecords, 1); + + javaRDD = tagLocation(hoodieTable.getIndex(), newWriteRecords, hoodieTable); + Map recordKeyToPartitionPathMap = new HashMap(); + List hoodieRecords = newWriteRecords.collect(); + hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath())); + + assertEquals(initialRecords, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size()); + assertEquals(initialRecords + newInsertsCount, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count()); + assertEquals(initialRecords, javaRDD.filter(record -> (record.getCurrentLocation() != null + && record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count()); + assertEquals(newInsertsCount, javaRDD.filter(record -> record.getKey().getPartitionPath().equalsIgnoreCase(newPartitionPath)) + .count(), "PartitionPath mismatch"); + + JavaRDD hoodieKeyJavaRDD = newWriteRecords.map(entry -> entry.getKey()); + JavaPairRDD>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable); + List hoodieRecordKeys = hoodieKeyJavaRDD.map(key -> key.getRecordKey()).collect(); + assertEquals(initialRecords + newInsertsCount, recordLocations.collect().size()); + assertEquals(initialRecords + newInsertsCount, recordLocations.map(record -> record._1).distinct().count()); + recordLocations.foreach(entry -> assertTrue(hoodieRecordKeys.contains(entry._1.getRecordKey()), "Missing HoodieRecordKey")); + assertEquals(newInsertsCount, recordLocations.filter(entry -> newPartitionPath.equalsIgnoreCase(entry._1.getPartitionPath())).count()); + } + private HoodieWriteConfig.Builder getConfigBuilder() { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index 265daaa316a89..27fd2822986f8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -112,11 +112,12 @@ public void init(HoodieTableType tableType, Option writeConfi initWriteConfigAndMetatableWriter(this.writeConfig, enableMetadataTable); } - protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) { + protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) throws IOException { this.writeConfig = writeConfig; if (enableMetadataTable) { metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context); testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + testTable.updateFilesPartitionInTableConfig(); } else { testTable = HoodieTestTable.of(metaClient); } @@ -324,7 +325,7 @@ protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesClea return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withAutoCommit(autoCommit) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(0) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(policy) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java index 7420a272b81af..40d8c77efc878 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java @@ -118,8 +118,7 @@ public void initTimelineService() { context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> new HoodieBackedTestDelayedTableMetadata( - context, config.getMetadataConfig(), basePath, - config.getViewStorageConfig().getSpillableDir(), true))); + context, config.getMetadataConfig(), basePath, true))); timelineService.startService(); timelineServicePort = timelineService.getServerPort(); LOG.info("Started timeline server on port: " + timelineServicePort); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java index 04d0f1ffe549f..ad541fdef57ba 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java @@ -24,6 +24,7 @@ import org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy; import org.apache.hudi.client.clustering.update.strategy.SparkConsistentBucketDuplicateUpdateStrategy; import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; @@ -294,7 +295,7 @@ public void testConcurrentWrite() throws IOException { List writeStatues = writeData(writeTime, 2000, false); // Cannot schedule clustering if there is in-flight writer Assertions.assertFalse(writeClient.scheduleClustering(Option.empty()).isPresent()); - Assertions.assertTrue(writeClient.commitStats(writeTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Assertions.assertTrue(writeClient.commitStats(writeTime, HoodieListData.eager(writeStatues), writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType())); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -330,7 +331,7 @@ private List writeData(String commitTime, int totalRecords, boolean List writeStatues = writeClient.upsert(writeRecords, commitTime).collect(); org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); if (doCommit) { - Assertions.assertTrue(writeClient.commitStats(commitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Assertions.assertTrue(writeClient.commitStats(commitTime, HoodieListData.eager(writeStatues), writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType())); } metaClient = HoodieTableMetaClient.reload(metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index 632fabebb7f97..10cad9644798e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -56,7 +56,7 @@ public void setUp(@TempDir Path tempDir) { } @ParameterizedTest - @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE", "BUCKET"}) + @EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE", "BUCKET", "RECORD_INDEX"}) public void testCreateIndex(IndexType indexType) { HoodieWriteConfig config; HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder(); @@ -99,10 +99,16 @@ public void testCreateIndex(IndexType indexType) { config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(indexConfigBuilder.fromProperties(props).withIndexType(IndexType.BUCKET) - .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build()) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build()) .build(); assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSparkConsistentBucketIndex); break; + case RECORD_INDEX: + config = clientConfigBuilder.withPath(basePath) + .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.RECORD_INDEX).build()) + .build(); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkMetadataTableRecordIndex); + break; default: // no -op. just for checkstyle errors } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java index 178097f2241ad..a0dc0e0124107 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; @@ -181,6 +182,32 @@ public void testSimpleTagLocationAndUpdate(HoodieTableType tableType) throws Exc // Now commit this & update location of records inserted and validate no errors writeClient.commit(newCommitTime, writeStatues); + + // Create new commit time. + String secondCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Now tagLocation for these records, index should tag them correctly + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + + // Generate updates for all existig records. + List newRecords = dataGen.generateUpdatesForAllRecords(secondCommitTime); + // Update partitionPath information. + String newPartitionPath = "2022/11/04"; + newRecords = newRecords.stream() + .map(rec -> new HoodieAvroRecord(new HoodieKey(rec.getRecordKey(), newPartitionPath), (HoodieRecordPayload) rec.getData())) + .collect(Collectors.toList()); + JavaRDD newWriteRecords = jsc().parallelize(newRecords, 1); + // Now tagLocation for these records, hbaseIndex should tag them correctly + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + List records4 = tagLocation(index, newWriteRecords, hoodieTable).collect(); + assertEquals(numRecords, records4.stream().filter(record -> record.isCurrentLocationKnown()).count()); + assertEquals(numRecords, records4.stream().map(record -> record.getKey().getRecordKey()).distinct().count()); + assertEquals(numRecords, records4.stream().filter(record -> (record.getCurrentLocation() != null + && record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count()); + assertEquals(0, records4.stream() + .filter(record -> record.getKey().getPartitionPath().equalsIgnoreCase(newPartitionPath)).count()); + // Now tagLocation for these records, hbaseIndex should tag them correctly metaClient = HoodieTableMetaClient.reload(metaClient); hoodieTable = HoodieSparkTable.create(config, context, metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index 0399bf142426a..ea90bf38c2969 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -134,7 +135,7 @@ public void init(HoodieTableType tableType) throws Exception { hadoopConf.addResource(wrapperFs.getConf()); } - private void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) { + private void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) throws IOException { if (enableMetadataTable) { metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context); testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); @@ -322,6 +323,7 @@ public void testArchiveTableWithReplaceCommits() throws Exception { } } + @Disabled("HUDI-6385") @ParameterizedTest @ValueSource(strings = {"KEEP_LATEST_BY_HOURS", "KEEP_LATEST_COMMITS"}) public void testArchivalWithAutoAdjustmentBasedOnCleanConfigs(String cleaningPolicy) throws Exception { @@ -350,17 +352,17 @@ public void testArchivalWithAutoAdjustmentBasedOnCleanConfigs(String cleaningPol List expectedAllCommits = new ArrayList<>(); // The following commits should be archived - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 90, true, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 80, true, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 70, true, testTable)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 90, true, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 80, true, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 70, true, testTable, config)); // The following commits should not be archived - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 50, true, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 45, true, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 40, true, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 30, false, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 20, false, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 10, true, testTable)); - expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 5, true, testTable)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 50, true, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 45, true, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 40, true, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 30, false, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 20, false, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 10, true, testTable, config)); + expectedAllCommits.add(triggerCommit(p0, p1, commitDateTime, 5, true, testTable, config)); metaClient = HoodieTableMetaClient.reload(metaClient); Pair, List> commitsList = @@ -374,29 +376,60 @@ public void testArchivalWithAutoAdjustmentBasedOnCleanConfigs(String cleaningPol private HoodieInstant triggerCommit( String p0, String p1, ZonedDateTime curDateTime, int minutesForCommit, - boolean isComplete, HoodieTestTable testTable) throws Exception { + boolean isComplete, HoodieTestTable testTable, HoodieWriteConfig config) throws Exception { String file1P0C0 = UUID.randomUUID().toString(); String file1P1C0 = UUID.randomUUID().toString(); String commitTs = HoodieActiveTimeline.formatDate(Date.from(curDateTime.minusMinutes(minutesForCommit).toInstant())); - testTable.addInflightCommit(commitTs).withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }); + return commitWithMdt(commitTs, part1ToFileId, testTable, metadataWriter, true, true, isComplete); + } + private HoodieInstant commitWithMdt(String instantTime, Map> partToFileId, + HoodieTestTable testTable, HoodieTableMetadataWriter metadataWriter, + boolean addBaseFiles, boolean addLogFiles, boolean isComplete) throws Exception { + testTable.addInflightCommit(instantTime); + HoodieCommitMetadata commitMeta; if (isComplete) { - HoodieCommitMetadata commitMetadata = generateCommitMetadata(commitTs, - Collections.unmodifiableMap(new HashMap>() { - { - put(p0, CollectionUtils.createImmutableList(file1P0C0)); - put(p1, CollectionUtils.createImmutableList(file1P1C0)); - } - }) - ); - - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTs), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + Map> partToFileIds = new HashMap<>(); + partToFileId.forEach((key, value) -> { + try { + List files = new ArrayList<>(); + if (addBaseFiles) { + files.addAll(testTable.withBaseFilesInPartition(key, value.toArray(new String[0])).getValue()); + } + if (addLogFiles) { + value.forEach(logFilePrefix -> { + try { + files.addAll(testTable.withLogFile(key, logFilePrefix, 1, 2).getValue()); + } catch (Exception e) { + e.printStackTrace(); + } + }); + } + partToFileIds.put(key, files); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + commitMeta = generateCommitMetadata(instantTime, partToFileIds); + metadataWriter.performTableServices(Option.of(instantTime)); + metadataWriter.update(commitMeta, context.emptyHoodieData(), instantTime); + } else { + commitMeta = generateCommitMetadata(instantTime, new HashMap<>()); } + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), + Option.of(commitMeta.toJsonString().getBytes(StandardCharsets.UTF_8))); + metaClient = HoodieTableMetaClient.reload(metaClient); return new HoodieInstant( - isComplete ? State.COMPLETED : State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTs); + isComplete ? State.COMPLETED : State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime); } protected static HoodieCommitMetadata generateCommitMetadata( @@ -464,7 +497,7 @@ public void testSavepointWithArchival(boolean archiveBeyondSavepoint) throws Exc commitsAfterArchival = commitsList.getValue(); metaClient.reloadActiveTimeline(); - verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002","00000003", "00000004", "00000005", "00000006", "00000007")), + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006", "00000007")), getActiveCommitInstants(Arrays.asList("00000008", "00000009")), commitsAfterArchival); } @@ -650,6 +683,7 @@ public void testLoadArchiveTimelineWithDamagedPlanFile(boolean enableArchiveMerg assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload()); } + @Disabled("HUDI-6386") @ParameterizedTest @ValueSource(booleans = {true, false}) public void testArchivalWithMultiWriters(boolean enableMetadata) throws Exception { @@ -1157,9 +1191,9 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") - .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) - .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) - .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) .build(); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -1217,7 +1251,7 @@ public void testArchiveInflightClean(boolean enableMetadataTable) throws Excepti HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).forTable("test-trip-table") .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) - .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) + .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(2, 3).build()) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 065abd608b59b..9016c83cdaac9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -18,8 +18,6 @@ package org.apache.hudi.table; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; @@ -77,11 +75,13 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.action.clean.CleanPlanner; import org.apache.hudi.testutils.HoodieClientTestBase; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import scala.Tuple3; import java.io.File; import java.io.IOException; @@ -99,7 +99,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.HoodieTestCommitGenerator.getBaseFilename; +import scala.Tuple3; + import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; @@ -121,11 +122,11 @@ public class TestCleaner extends HoodieClientTestBase { /** * Helper method to do first batch of insert for clean by versions/commits tests. * - * @param context Spark engine context - * @param metaClient Hoodie table meta client - * @param client Hoodie Client + * @param context Spark engine context + * @param metaClient Hoodie table meta client + * @param client Hoodie Client * @param recordGenFunction Function to generate records for insertion - * @param insertFn Insertion API for testing + * @param insertFn Insertion API for testing * @throws Exception in case of error */ public static Pair> insertFirstBigBatchForClientCleanerTest( @@ -171,10 +172,10 @@ public static Pair> insertFirstBigBatchForClientCle /** * Helper method to do first batch of insert for clean by versions/commits tests. * - * @param context Spark engine context - * @param client Hoodie Client + * @param context Spark engine context + * @param client Hoodie Client * @param recordGenFunction Function to generate records for insertion - * @param insertFn Insertion API for testing + * @param insertFn Insertion API for testing * @throws Exception in case of error */ public static Pair> insertFirstFailedBigBatchForClientCleanerTest( @@ -352,9 +353,9 @@ public void testFailedInsertAndCleanByCommits() throws Exception { /** * Test Helper for Cleaning failed commits by commits logic from HoodieWriteClient API perspective. * - * @param insertFn Insert API to be tested + * @param insertFn Insert API to be tested * @param isPreppedAPI Flag to indicate if a prepped-version is used. If true, a wrapper function will be used during - * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) + * record generation to also tag the regards (de-dupe is implicit as we use uniq record-gen APIs) * @throws Exception in case of errors */ private void testFailedInsertAndCleanByCommits( @@ -510,12 +511,12 @@ protected List runCleaner( @Test public void testCleanEmptyInstants() throws Exception { HoodieWriteConfig config = - HoodieWriteConfig.newBuilder() - .withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCleanConfig(HoodieCleanConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) - .build(); + HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).build()) + .build(); metaClient = HoodieTableMetaClient.reload(metaClient); int commitCount = 20; @@ -530,9 +531,11 @@ public void testCleanEmptyInstants() throws Exception { int instantClean = startInstant; + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); for (int i = 0; i < commitCount; i++, startInstant++) { String commitTime = makeNewCommitTime(startInstant, "%09d"); - HoodieTestTable.of(metaClient).addCommit(commitTime); + commitWithMdt(commitTime, Collections.emptyMap(), testTable, metadataWriter); } List cleanStats = runCleaner(config); @@ -540,26 +543,26 @@ public void testCleanEmptyInstants() throws Exception { assertEquals(0, cleanStats.size(), "Must not clean any files"); assertEquals(1, timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().countInstants()); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().countInstants()); assertEquals(0, timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflights().countInstants()); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflights().countInstants()); assertEquals(--cleanCount, timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--instantClean, "%09d"))); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--instantClean, "%09d"))); cleanStats = runCleaner(config); timeline = metaClient.reloadActiveTimeline(); assertEquals(0, cleanStats.size(), "Must not clean any files"); assertEquals(1, timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().countInstants()); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().countInstants()); assertEquals(0, timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflights().countInstants()); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflights().countInstants()); assertEquals(--cleanCount, timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--instantClean, "%09d"))); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--instantClean, "%09d"))); } @Test @@ -581,21 +584,13 @@ public void testCleanWithReplaceCommits() throws Exception { // make 1 commit, with 1 file per partition String file1P0C0 = UUID.randomUUID().toString(); String file1P1C0 = UUID.randomUUID().toString(); - testTable.addInflightCommit("00000000000001").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - - HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000001", - Collections.unmodifiableMap(new HashMap>() { - { - put(p0, CollectionUtils.createImmutableList(file1P0C0)); - put(p1, CollectionUtils.createImmutableList(file1P1C0)); - } - }) - ); - metadataWriter.update(commitMetadata, "00000000000001"); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000001"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }); + commitWithMdt("00000000000001", part1ToFileId, testTable, metadataWriter, true, true); metaClient = HoodieTableMetaClient.reload(metaClient); List hoodieCleanStatsOne = runCleanerWithInstantFormat(config, true); @@ -704,7 +699,7 @@ public void testCleanMetadataUpgradeDowngrade() { // map with relative path. Map newExpected = new HashMap<>(); newExpected.put(partition1, new Tuple3<>(Arrays.asList(fileName1, fileName2), Collections.singletonList(fileName1), - Collections.singletonList(fileName2))); + Collections.singletonList(fileName2))); newExpected.put(partition2, new Tuple3<>(deletePathPatterns2, successDeleteFiles2, failedDeleteFiles2)); HoodieCleanMetadata metadata = CleanerUtils.convertCleanMetadata( @@ -772,8 +767,8 @@ public void testCleanPlanUpgradeDowngrade() { HoodieCleanerPlan version1Plan = HoodieCleanerPlan.newBuilder().setEarliestInstantToRetain(HoodieActionInstant.newBuilder() - .setAction(HoodieTimeline.COMMIT_ACTION) - .setTimestamp(instantTime).setState(State.COMPLETED.name()).build()) + .setAction(HoodieTimeline.COMMIT_ACTION) + .setTimestamp(instantTime).setState(State.COMPLETED.name()).build()) .setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) .setFilesToBeDeletedPerPartition(filesToBeCleanedPerPartition) .setVersion(CleanPlanV1MigrationHandler.VERSION) @@ -834,6 +829,7 @@ private static void assertCleanMetadataPathEquals(Map expected, /** * Generate Bootstrap index, bootstrap base file and corresponding metaClient. + * * @return Partition to BootstrapFileMapping Map * @throws IOException */ @@ -1005,21 +1001,13 @@ public void testRerunFailedClean(boolean simulateMetadataFailure) throws Excepti // make 1 commit, with 1 file per partition String file1P0C0 = UUID.randomUUID().toString(); String file1P1C0 = UUID.randomUUID().toString(); - testTable.addInflightCommit("00000000000001").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - - HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000001", - Collections.unmodifiableMap(new HashMap>() { - { - put(p0, CollectionUtils.createImmutableList(file1P0C0)); - put(p1, CollectionUtils.createImmutableList(file1P1C0)); - } - }) - ); - metadataWriter.update(commitMetadata, "00000000000001"); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000001"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }); + commitWithMdt("00000000000001", part1ToFileId, testTable, metadataWriter, true, true); metaClient = HoodieTableMetaClient.reload(metaClient); // make next replacecommit, with 1 clustering operation. logically delete p0. No change to p1 @@ -1054,7 +1042,9 @@ public void testRerunFailedClean(boolean simulateMetadataFailure) throws Excepti assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); } - /** Test if cleaner will fallback to full clean if commit for incremental clean is archived. */ + /** + * Test if cleaner will fallback to full clean if commit for incremental clean is archived. + */ @Test public void testIncrementalFallbackToFullClean() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder() @@ -1138,19 +1128,38 @@ public void testIncrementalFallbackToFullClean() throws Exception { assertTrue(testTable.baseFileExists(p2, "4", file4P2), "Latest FileSlice exists"); } - private void commitWithMdt(String instantTime, Map> partToFileId, - HoodieTestTable testTable, HoodieTableMetadataWriter metadataWriter) throws Exception { - HoodieCommitMetadata commitMeta = generateCommitMetadata(instantTime, partToFileId); + public void commitWithMdt(String instantTime, Map> partToFileId, + HoodieTestTable testTable, HoodieTableMetadataWriter metadataWriter) throws Exception { + commitWithMdt(instantTime, partToFileId, testTable, metadataWriter, true, false); + } + + public void commitWithMdt(String instantTime, Map> partToFileId, + HoodieTestTable testTable, HoodieTableMetadataWriter metadataWriter, boolean addBaseFiles, boolean addLogFiles) throws Exception { testTable.addInflightCommit(instantTime); + Map> partToFileIds = new HashMap<>(); partToFileId.forEach((key, value) -> { try { - testTable.withBaseFilesInPartition(key, value.toArray(new String[0])); + List files = new ArrayList<>(); + if (addBaseFiles) { + files.addAll(testTable.withBaseFilesInPartition(key, value.toArray(new String[0])).getValue()); + } + if (addLogFiles) { + value.forEach(logFilePrefix -> { + try { + files.addAll(testTable.withLogFile(key, logFilePrefix, 1, 2).getValue()); + } catch (Exception e) { + e.printStackTrace(); + } + }); + } + partToFileIds.put(key, files); } catch (Exception e) { throw new RuntimeException(e); } }); + HoodieCommitMetadata commitMeta = generateCommitMetadata(instantTime, partToFileIds); metadataWriter.performTableServices(Option.of(instantTime)); - metadataWriter.update(commitMeta, instantTime); + metadataWriter.update(commitMeta, context.emptyHoodieData(), instantTime); metaClient.getActiveTimeline().saveAsComplete( new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), Option.of(commitMeta.toJsonString().getBytes(StandardCharsets.UTF_8))); @@ -1160,13 +1169,16 @@ private void commitWithMdt(String instantTime, Map> partToF /** * Common test method for validating pending compactions. * - * @param config Hoodie Write Config + * @param config Hoodie Write Config * @param expNumFilesDeleted Number of files deleted */ private void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDeleted, - int expNumFilesUnderCompactionDeleted, boolean retryFailure) throws Exception { + int expNumFilesUnderCompactionDeleted, boolean retryFailure) throws Exception { HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); + + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + final String partition = "2016/03/15"; Map expFileIdToPendingCompaction = new HashMap() { { @@ -1192,54 +1204,73 @@ private void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDel // multiple versions with pending compaction. File Slices (6 - 7) have multiple file-slices but not under // compactions // FileIds 2-5 will be under compaction - HoodieTestTable.of(metaClient) - .addCommit("000") - .withBaseFilesInPartition(partition, "fileId1", "fileId2", "fileId3", "fileId4", "fileId5", "fileId6", "fileId7") - .withLogFile(partition, "fileId1", 1, 2) - .withLogFile(partition, "fileId2", 1, 2) - .withLogFile(partition, "fileId3", 1, 2) - .withLogFile(partition, "fileId4", 1, 2) - .withLogFile(partition, "fileId5", 1, 2) - .withLogFile(partition, "fileId6", 1, 2) - .withLogFile(partition, "fileId7", 1, 2) - .addCommit("001") - .withBaseFilesInPartition(partition, "fileId3", "fileId4", "fileId5", "fileId6", "fileId7") - .withLogFile(partition, "fileId3", 1, 2) - .withLogFile(partition, "fileId4", 1, 2) - .withLogFile(partition, "fileId5", 1, 2) - .withLogFile(partition, "fileId6", 1, 2) - .withLogFile(partition, "fileId7", 1, 2) - .addCommit("003") - .withBaseFilesInPartition(partition, "fileId4", "fileId5", "fileId6", "fileId7") - .withLogFile(partition, "fileId4", 1, 2) - .withLogFile(partition, "fileId5", 1, 2) - .withLogFile(partition, "fileId6", 1, 2) - .withLogFile(partition, "fileId7", 1, 2) - .addRequestedCompaction("004", new FileSlice(partition, "000", "fileId2")) - .withLogFile(partition, "fileId2", 1, 2) - .addCommit("005") - .withBaseFilesInPartition(partition, "fileId5", "fileId6", "fileId7") - .withLogFile(partition, "fileId5", 1, 2) - .withLogFile(partition, "fileId6", 1, 2) - .withLogFile(partition, "fileId7", 1, 2) - .addRequestedCompaction("006", new FileSlice(partition, "001", "fileId3")) - .withLogFile(partition, "fileId3", 1, 2) - .addCommit("007") - .withBaseFilesInPartition(partition, "fileId6", "fileId7") - .withLogFile(partition, "fileId6", 1, 2) - .withLogFile(partition, "fileId7", 1, 2) - .addRequestedCompaction("008", new FileSlice(partition, "003", "fileId4")) - .withLogFile(partition, "fileId4", 1, 2) - .addCommit("009") - .withBaseFilesInPartition(partition, "fileId6", "fileId7") - .withLogFile(partition, "fileId6", 1, 2) - .withLogFile(partition, "fileId7", 1, 2) - .addRequestedCompaction("010", new FileSlice(partition, "005", "fileId5")) - .withLogFile(partition, "fileId5", 1, 2) - .addCommit("011") - .withBaseFilesInPartition(partition, "fileId7") - .withLogFile(partition, "fileId7", 1, 2) - .addCommit("013"); + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + + testTable.withPartitionMetaFiles(partition); + + // add file partition "part_1" + String file1P1 = "fileId1"; + String file2P1 = "fileId2"; + String file3P1 = "fileId3"; + String file4P1 = "fileId4"; + String file5P1 = "fileId5"; + String file6P1 = "fileId6"; + String file7P1 = "fileId7"; + + Map> part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file1P1, file2P1, file3P1, file4P1, file5P1, file6P1, file7P1)); + // all 7 fileIds + commitWithMdt("000", part1ToFileId, testTable, metadataWriter, true, true); + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file3P1, file4P1, file5P1, file6P1, file7P1)); + // fileIds 3 to 7 + commitWithMdt("001", part1ToFileId, testTable, metadataWriter, true, true); + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file4P1, file5P1, file6P1, file7P1)); + // fileIds 4 to 7 + commitWithMdt("003", part1ToFileId, testTable, metadataWriter, true, true); + + // add compaction + testTable.addRequestedCompaction("004", new FileSlice(partition, "000", file2P1)); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file2P1)); + commitWithMdt("005", part1ToFileId, testTable, metadataWriter, false, true); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file5P1, file6P1, file7P1)); + commitWithMdt("0055", part1ToFileId, testTable, metadataWriter, true, true); + + testTable.addRequestedCompaction("006", new FileSlice(partition, "001", file3P1)); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file3P1)); + commitWithMdt("007", part1ToFileId, testTable, metadataWriter, false, true); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file6P1, file7P1)); + commitWithMdt("0075", part1ToFileId, testTable, metadataWriter, true, true); + + testTable.addRequestedCompaction("008", new FileSlice(partition, "003", file4P1)); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file4P1)); + commitWithMdt("009", part1ToFileId, testTable, metadataWriter, false, true); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file6P1, file7P1)); + commitWithMdt("0095", part1ToFileId, testTable, metadataWriter, true, true); + + testTable.addRequestedCompaction("010", new FileSlice(partition, "005", file5P1)); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file5P1)); + commitWithMdt("011", part1ToFileId, testTable, metadataWriter, false, true); + + part1ToFileId = new HashMap<>(); + part1ToFileId.put(partition, Arrays.asList(file7P1)); + commitWithMdt("013", part1ToFileId, testTable, metadataWriter, true, true); + testTable.updateFilesPartitionInTableConfig(); // Clean now metaClient = HoodieTableMetaClient.reload(metaClient); @@ -1262,17 +1293,17 @@ private void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDel // Test for progress (Did we clean some files ?) long numFilesUnderCompactionDeleted = hoodieCleanStats.stream() - .flatMap(cleanStat -> convertPathToFileIdWithCommitTime(newMetaClient, cleanStat.getDeletePathPatterns()) - .map(fileIdWithCommitTime -> { - if (expFileIdToPendingCompaction.containsKey(fileIdWithCommitTime.getKey())) { - assertTrue(HoodieTimeline.compareTimestamps( - fileIdToLatestInstantBeforeCompaction.get(fileIdWithCommitTime.getKey()), - HoodieTimeline.GREATER_THAN, fileIdWithCommitTime.getValue()), - "Deleted instant time must be less than pending compaction"); - return true; - } - return false; - })).filter(x -> x).count(); + .flatMap(cleanStat -> convertPathToFileIdWithCommitTime(newMetaClient, cleanStat.getDeletePathPatterns()) + .map(fileIdWithCommitTime -> { + if (expFileIdToPendingCompaction.containsKey(fileIdWithCommitTime.getKey())) { + assertTrue(HoodieTimeline.compareTimestamps( + fileIdToLatestInstantBeforeCompaction.get(fileIdWithCommitTime.getKey()), + HoodieTimeline.GREATER_THAN, fileIdWithCommitTime.getValue()), + "Deleted instant time must be less than pending compaction"); + return true; + } + return false; + })).filter(x -> x).count(); long numDeleted = hoodieCleanStats.stream().mapToLong(cleanStat -> cleanStat.getDeletePathPatterns().size()).sum(); // Tighter check for regression @@ -1282,7 +1313,7 @@ private void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDel } private Stream> convertPathToFileIdWithCommitTime(final HoodieTableMetaClient metaClient, - List paths) { + List paths) { Predicate roFilePredicate = path -> path.contains(metaClient.getTableConfig().getBaseFileFormat().getFileExtension()); Predicate rtFilePredicate = @@ -1291,10 +1322,8 @@ private Stream> convertPathToFileIdWithCommitTime(final Hoo String fileName = Paths.get(fullPath).getFileName().toString(); return Pair.of(FSUtils.getFileId(fileName), FSUtils.getCommitTime(fileName)); }); - Stream> stream2 = paths.stream().filter(rtFilePredicate).map(path -> { - return Pair.of(FSUtils.getFileIdFromLogPath(new Path(path)), - FSUtils.getBaseCommitTimeFromLogPath(new Path(path))); - }); + Stream> stream2 = paths.stream().filter(rtFilePredicate).map(path -> Pair.of(FSUtils.getFileIdFromLogPath(new Path(path)), + FSUtils.getBaseCommitTimeFromLogPath(new Path(path)))); return Stream.concat(stream1, stream2); } @@ -1305,7 +1334,7 @@ protected static HoodieCommitMetadata generateCommitMetadata( partitionToFilePaths.forEach((partitionPath, fileList) -> fileList.forEach(f -> { HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setPartitionPath(partitionPath); - writeStat.setPath(partitionPath + "/" + getBaseFilename(instantTime, f)); + writeStat.setPath(partitionPath + "/" + f); writeStat.setFileId(f); writeStat.setTotalWriteBytes(1); writeStat.setFileSizeInBytes(1); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java index b3da341df2574..18aadaa887647 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java @@ -95,12 +95,12 @@ public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() HoodieTestTable testTable = HoodieTestTable.of(metaClient) .withPartitionMetaFiles(p1, p2, p3) .addCommit("001") - .withBaseFilesInPartition(p1, "id11") - .withBaseFilesInPartition(p2, "id12") - .withLogFile(p1, "id11", 3) + .withBaseFilesInPartition(p1, "id11").getLeft() + .withBaseFilesInPartition(p2, "id12").getLeft() + .withLogFile(p1, "id11", 3).getLeft() .addCommit("002") - .withBaseFilesInPartition(p1, "id21") - .withBaseFilesInPartition(p2, "id22"); + .withBaseFilesInPartition(p1, "id21").getLeft() + .withBaseFilesInPartition(p2, "id22").getLeft(); HoodieWriteConfig writeConfig = getConfigBuilder().withRollbackUsingMarkers(false).build(); HoodieTable table = this.getHoodieTable(metaClient, writeConfig); @@ -251,12 +251,12 @@ public void testRollbackScale() throws Exception { HoodieTestTable testTable = HoodieTestTable.of(metaClient) .withPartitionMetaFiles(p1, p2, p3) .addCommit("001") - .withBaseFilesInPartition(p1, "id11") - .withBaseFilesInPartition(p2, "id12") - .withLogFile(p1, "id11", 3) + .withBaseFilesInPartition(p1, "id11").getLeft() + .withBaseFilesInPartition(p2, "id12").getLeft() + .withLogFile(p1, "id11", 3).getLeft() .addCommit("002") - .withBaseFilesInPartition(p1, "id21") - .withBaseFilesInPartition(p2, "id22") + .withBaseFilesInPartition(p1, "id21").getLeft() + .withBaseFilesInPartition(p2, "id22").getLeft() .addCommit("003") .withBaseFilesInPartition(p3, fileLengths); @@ -344,12 +344,12 @@ public void testRollbackBackup() throws Exception { HoodieTestTable testTable = HoodieTestTable.of(metaClient) .withPartitionMetaFiles(p1, p2, p3) .addCommit("001") - .withBaseFilesInPartition(p1, "id11") - .withBaseFilesInPartition(p2, "id12") - .withLogFile(p1, "id11", 3) + .withBaseFilesInPartition(p1, "id11").getLeft() + .withBaseFilesInPartition(p2, "id12").getLeft() + .withLogFile(p1, "id11", 3).getLeft() .addCommit("002") - .withBaseFilesInPartition(p1, "id21") - .withBaseFilesInPartition(p2, "id22"); + .withBaseFilesInPartition(p1, "id21").getLeft() + .withBaseFilesInPartition(p2, "id22").getLeft(); HoodieTable table = this.getHoodieTable(metaClient, getConfigBuilder().withRollbackBackupEnabled(true).build()); HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002"); @@ -383,14 +383,14 @@ public void testRollbackForMultiwriter() throws Exception { HoodieTestTable testTable = HoodieTestTable.of(metaClient) .withPartitionMetaFiles(p1, p2, p3) .addCommit("001") - .withBaseFilesInPartition(p1, "id11") - .withBaseFilesInPartition(p2, "id12") - .withLogFile(p1, "id11", 3) + .withBaseFilesInPartition(p1, "id11").getLeft() + .withBaseFilesInPartition(p2, "id12").getLeft() + .withLogFile(p1, "id11", 3).getLeft() .addCommit("002") - .withBaseFilesInPartition(p1, "id21") - .withBaseFilesInPartition(p2, "id22") + .withBaseFilesInPartition(p1, "id21").getLeft() + .withBaseFilesInPartition(p2, "id22").getLeft() .addInflightCommit("003") - .withBaseFilesInPartition(p1, "id31") + .withBaseFilesInPartition(p1, "id31").getLeft() .addCommit("004"); HoodieTable table = this.getHoodieTable(metaClient, getConfigBuilder().build()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java index de47646302e3f..573efcc3724af 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java @@ -42,6 +42,7 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.TestCleaner; + import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -115,32 +116,26 @@ public void testKeepLatestCommits( .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) .retainCommits(2) .withMaxCommitsBeforeCleaning(2) - .build()).build(); + .build()).build(); - HoodieTestTable testTable = HoodieTestTable.of(metaClient); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); String p0 = "2020/01/01"; String p1 = "2020/01/02"; Map> bootstrapMapping = enableBootstrapSourceClean ? generateBootstrapIndexAndSourceData(p0, p1) : null; // make 1 commit, with 1 file per partition String file1P0C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p0).get(0).getFileId() - : UUID.randomUUID().toString(); + : UUID.randomUUID().toString(); String file1P1C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p1).get(0).getFileId() - : UUID.randomUUID().toString(); - testTable.addInflightCommit("00000000000001").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - - HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000001", - Collections.unmodifiableMap(new HashMap>() { - { - put(p0, CollectionUtils.createImmutableList(file1P0C0)); - put(p1, CollectionUtils.createImmutableList(file1P1C0)); - } - }) - ); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000001"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - + : UUID.randomUUID().toString(); + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }); + commitWithMdt("00000000000001", part1ToFileId, testTable, metadataWriter, true, true); metaClient = HoodieTableMetaClient.reload(metaClient); List hoodieCleanStatsOne = @@ -153,16 +148,15 @@ public void testKeepLatestCommits( Map partitionAndFileId002 = testTable.addInflightCommit("00000000000003").getFileIdsWithBaseFilesInPartitions(p0, p1); String file2P0C1 = partitionAndFileId002.get(p0); String file2P1C1 = partitionAndFileId002.get(p1); - testTable.forCommit("00000000000003").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - commitMetadata = generateCommitMetadata("00000000000003", new HashMap>() { + Map> part2ToFileId = Collections.unmodifiableMap(new HashMap>() { { put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1)); put(p1, CollectionUtils.createImmutableList(file1P1C0, file2P1C1)); } }); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000003"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + commitWithMdt("00000000000003", part2ToFileId, testTable, metadataWriter, true, true); + metaClient = HoodieTableMetaClient.reload(metaClient); + List hoodieCleanStatsTwo = runCleaner(config, simulateFailureRetry, simulateMetadataFailure, 4, true); assertEquals(0, hoodieCleanStatsTwo.size(), "Must not scan any partitions and clean any files"); @@ -172,16 +166,14 @@ public void testKeepLatestCommits( assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); // make next commit, with 2 updates to existing files, and 1 insert - String file3P0C2 = testTable.addInflightCommit("00000000000005") - .withBaseFilesInPartition(p0, file1P0C0) - .withBaseFilesInPartition(p0, file2P0C1) - .getFileIdsWithBaseFilesInPartitions(p0).get(p0); - commitMetadata = generateCommitMetadata("00000000000003", - CollectionUtils.createImmutableMap( - p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file3P0C2))); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000005"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + String file3P0C2 = testTable.addInflightCommit("00000000000005").getFileIdsWithBaseFilesInPartitions(p0).get(p0); + Map> part3ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file3P0C2)); + } + }); + commitWithMdt("00000000000005", part3ToFileId, testTable, metadataWriter, true, true); + metaClient = HoodieTableMetaClient.reload(metaClient); List hoodieCleanStatsThree = runCleaner(config, simulateFailureRetry, simulateMetadataFailure, 6, true); @@ -190,25 +182,21 @@ public void testKeepLatestCommits( assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); // make next commit, with 2 updates to existing files, and 1 insert - String file4P0C3 = testTable.addInflightCommit("00000000000007") - .withBaseFilesInPartition(p0, file1P0C0) - .withBaseFilesInPartition(p0, file2P0C1) - .getFileIdsWithBaseFilesInPartitions(p0).get(p0); - commitMetadata = generateCommitMetadata("00000000000004", - CollectionUtils.createImmutableMap( - p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file4P0C3))); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000007"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + String file4P0C3 = testTable.addInflightCommit("00000000000007").getFileIdsWithBaseFilesInPartitions(p0).get(p0); + Map> part4ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file4P0C3)); + } + }); + commitWithMdt("00000000000007", part4ToFileId, testTable, metadataWriter); + metaClient = HoodieTableMetaClient.reload(metaClient); List hoodieCleanStatsFour = runCleaner(config, simulateFailureRetry, simulateMetadataFailure, 8, true); // enableBootstrapSourceClean would delete the bootstrap base file as the same time HoodieCleanStat partitionCleanStat = getCleanStat(hoodieCleanStatsFour, p0); - assertEquals(enableBootstrapSourceClean ? 2 : 1, partitionCleanStat.getSuccessDeleteFiles().size() - + (partitionCleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 - : partitionCleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least one old file"); + assertEquals(3, partitionCleanStat.getSuccessDeleteFiles().size()); assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); assertTrue(testTable.baseFileExists(p0, "00000000000003", file1P0C0)); assertTrue(testTable.baseFileExists(p0, "00000000000005", file1P0C0)); @@ -217,21 +205,21 @@ public void testKeepLatestCommits( assertTrue(testTable.baseFileExists(p0, "00000000000005", file3P0C2)); assertTrue(testTable.baseFileExists(p0, "00000000000007", file4P0C3)); if (enableBootstrapSourceClean) { + assertEquals(1, partitionCleanStat.getSuccessDeleteBootstrapBaseFiles().size()); assertFalse(Files.exists(Paths.get(bootstrapMapping.get( - p0).get(0).getBootstrapFileStatus().getPath().getUri()))); + p0).get(0).getBootstrapFileStatus().getPath().getUri()))); } metaClient = HoodieTableMetaClient.reload(metaClient); - String file5P0C4 = testTable.addInflightCommit("00000000000009") - .withBaseFilesInPartition(p0, file1P0C0) - .withBaseFilesInPartition(p0, file2P0C1) - .getFileIdsWithBaseFilesInPartitions(p0).get(p0); - commitMetadata = generateCommitMetadata("00000000000009", CollectionUtils.createImmutableMap( - p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file5P0C4))); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000009"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + String file5P0C4 = testTable.addInflightCommit("00000000000009").getFileIdsWithBaseFilesInPartitions(p0).get(p0); + Map> part5ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file5P0C4)); + } + }); + commitWithMdt("00000000000009", part5ToFileId, testTable, metadataWriter, true, true); + metaClient = HoodieTableMetaClient.reload(metaClient); List hoodieCleanStatsFive = runCleaner(config, simulateFailureRetry, simulateMetadataFailure, 10, true); @@ -247,7 +235,7 @@ public void testKeepLatestCommits( // No cleaning on partially written file, with no commit. testTable.forCommit("00000000000011").withBaseFilesInPartition(p0, file3P0C2); - commitMetadata = generateCommitMetadata("00000000000011", CollectionUtils.createImmutableMap(p0, + HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000011", CollectionUtils.createImmutableMap(p0, CollectionUtils.createImmutableList(file3P0C2))); metaClient.getActiveTimeline().createNewInstant( new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011")); @@ -460,24 +448,28 @@ public void testKeepLatestFileVersionsMOR() throws Exception { .build()).build(); HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); HoodieTestTable testTable = HoodieTestTable.of(metaClient); String p0 = "2020/01/01"; - // Make 3 files, one base file and 2 log files associated with base file String file1P0 = testTable.addDeltaCommit("000").getFileIdsWithBaseFilesInPartitions(p0).get(p0); - testTable.forDeltaCommit("000") - .withLogFile(p0, file1P0, 1) - .withLogFile(p0, file1P0, 2); + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0)); + } + }); + commitWithMdt("000", part1ToFileId, testTable, metadataWriter, true, true); // Make 2 files, one base file and 1 log files associated with base file testTable.addDeltaCommit("001") - .withBaseFilesInPartition(p0, file1P0) - .withLogFile(p0, file1P0, 3); + .withBaseFilesInPartition(p0, file1P0).getLeft() + .withLogFile(p0, file1P0, 3); + commitWithMdt("001", part1ToFileId, testTable, metadataWriter, true, true); List hoodieCleanStats = runCleaner(config); assertEquals(3, - getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() - .size(), "Must clean three files, one base and 2 log files"); + getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() + .size(), "Must clean three files, one base and 2 log files"); assertFalse(testTable.baseFileExists(p0, "000", file1P0)); assertFalse(testTable.logFilesExist(p0, "000", file1P0, 1, 2)); assertTrue(testTable.baseFileExists(p0, "001", file1P0)); @@ -492,37 +484,42 @@ public void testKeepLatestCommitsMOR() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true) - // Column Stats Index is disabled, since these tests construct tables which are - // not valid (empty commit metadata, invalid parquet files) - .withMetadataIndexColumnStats(false).build()) + // Column Stats Index is disabled, since these tests construct tables which are + // not valid (empty commit metadata, invalid parquet files) + .withMetadataIndexColumnStats(false).build()) .withCleanConfig(HoodieCleanConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) .build(); HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); HoodieTestTable testTable = HoodieTestTable.of(metaClient); String p0 = "2020/01/01"; - // Make 3 files, one base file and 2 log files associated with base file String file1P0 = testTable.addDeltaCommit("000").getFileIdsWithBaseFilesInPartitions(p0).get(p0); - testTable.forDeltaCommit("000") - .withLogFile(p0, file1P0, 1) - .withLogFile(p0, file1P0, 2); + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0)); + } + }); + commitWithMdt("000", part1ToFileId, testTable, metadataWriter, true, true); // Make 2 files, one base file and 1 log files associated with base file testTable.addDeltaCommit("001") - .withBaseFilesInPartition(p0, file1P0) - .withLogFile(p0, file1P0, 3); + .withBaseFilesInPartition(p0, file1P0).getLeft() + .withLogFile(p0, file1P0, 3); + commitWithMdt("001", part1ToFileId, testTable, metadataWriter, true, true); // Make 2 files, one base file and 1 log files associated with base file testTable.addDeltaCommit("002") - .withBaseFilesInPartition(p0, file1P0) - .withLogFile(p0, file1P0, 4); + .withBaseFilesInPartition(p0, file1P0).getLeft() + .withLogFile(p0, file1P0, 4); + commitWithMdt("002", part1ToFileId, testTable, metadataWriter, true, true); List hoodieCleanStats = runCleaner(config); assertEquals(3, - getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() - .size(), "Must clean three files, one base and 2 log files"); + getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() + .size(), "Must clean three files, one base and 2 log files"); assertFalse(testTable.baseFileExists(p0, "000", file1P0)); assertFalse(testTable.logFilesExist(p0, "000", file1P0, 1, 2)); assertTrue(testTable.baseFileExists(p0, "001", file1P0)); @@ -531,39 +528,45 @@ public void testKeepLatestCommitsMOR() throws Exception { assertTrue(testTable.logFileExists(p0, "002", file1P0, 4)); } - /** Test clean delete partition with KEEP_LATEST_COMMITS policy. */ + /** + * Test clean delete partition with KEEP_LATEST_COMMITS policy. + */ @Test public void testKeepLatestCommitWithDeletePartition() throws Exception { testCleanDeletePartition( - HoodieCleanConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .retainCommits(1) - .build()); + HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .retainCommits(1) + .build()); } - /** Test clean delete partition with KEEP_LATEST_BY_HOURS policy. */ + /** + * Test clean delete partition with KEEP_LATEST_BY_HOURS policy. + */ @Test public void testKeepXHoursWithDeletePartition() throws Exception { testCleanDeletePartition( - HoodieCleanConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS) - .cleanerNumHoursRetained(30) - .build()); + HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS) + .cleanerNumHoursRetained(30) + .build()); } - /** Test clean delete partition with KEEP_LATEST_FILE_VERSIONS policy. */ + /** + * Test clean delete partition with KEEP_LATEST_FILE_VERSIONS policy. + */ @Test public void testKeepFileVersionsWithDeletePartition() throws Exception { testCleanDeletePartition( - HoodieCleanConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) - .build()); + HoodieCleanConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) + .build()); } private void testCleanDeletePartition(HoodieCleanConfig cleanConfig) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withCleanConfig(cleanConfig) - .build(); + .withCleanConfig(cleanConfig) + .build(); long now = System.currentTimeMillis(); String commitInstant = HoodieActiveTimeline.formatDate(new Date(now - 49 * 3600 * 1000)); @@ -578,12 +581,17 @@ private void testCleanDeletePartition(HoodieCleanConfig cleanConfig) throws Exce String file1P2 = UUID.randomUUID().toString(); String file2P2 = UUID.randomUUID().toString(); - HoodieTestTable testTable = HoodieTestTable.of(metaClient); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); testTable.withPartitionMetaFiles(p1, p2); + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p1, CollectionUtils.createImmutableList(file1P1, file2P1)); + put(p2, CollectionUtils.createImmutableList(file1P2, file2P2)); + } + }); + commitWithMdt(commitInstant, part1ToFileId, testTable, metadataWriter, true, true); - testTable.forCommit(commitInstant) - .withBaseFilesInPartition(p1, file1P1, file2P1) - .withBaseFilesInPartition(p2, file1P2, file2P2); testTable.addDeletePartitionCommit(deleteInstant1, p1, Arrays.asList(file1P1, file2P1)); testTable.addDeletePartitionCommit(deleteInstant2, p2, Arrays.asList(file1P2, file2P2)); @@ -620,33 +628,27 @@ public void testKeepXHoursWithCleaning( .build()) .build(); - HoodieTestTable testTable = HoodieTestTable.of(metaClient); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); String p0 = "2020/01/01"; String p1 = "2020/01/02"; Map> bootstrapMapping = enableBootstrapSourceClean ? generateBootstrapIndexAndSourceData(p0, p1) : null; String file1P0C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p0).get(0).getFileId() - : UUID.randomUUID().toString(); + : UUID.randomUUID().toString(); String file1P1C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p1).get(0).getFileId() - : UUID.randomUUID().toString(); + : UUID.randomUUID().toString(); Instant instant = Instant.now(); ZonedDateTime commitDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault()); int minutesForFirstCommit = 150; String firstCommitTs = HoodieActiveTimeline.formatDate(Date.from(commitDateTime.minusMinutes(minutesForFirstCommit).toInstant())); - testTable.addInflightCommit(firstCommitTs).withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - - HoodieCommitMetadata commitMetadata = generateCommitMetadata(firstCommitTs, - Collections.unmodifiableMap(new HashMap>() { - { - put(p0, CollectionUtils.createImmutableList(file1P0C0)); - put(p1, CollectionUtils.createImmutableList(file1P1C0)); - } - }) - ); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, firstCommitTs), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }); + commitWithMdt(firstCommitTs, part1ToFileId, testTable, metadataWriter, true, true); metaClient = HoodieTableMetaClient.reload(metaClient); List hoodieCleanStatsOne = @@ -661,18 +663,18 @@ public void testKeepXHoursWithCleaning( Map partitionAndFileId002 = testTable.addInflightCommit(secondCommitTs).getFileIdsWithBaseFilesInPartitions(p0, p1); String file2P0C1 = partitionAndFileId002.get(p0); String file2P1C1 = partitionAndFileId002.get(p1); - testTable.forCommit(secondCommitTs).withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - commitMetadata = generateCommitMetadata(secondCommitTs, new HashMap>() { + Map> part2ToFileId = Collections.unmodifiableMap(new HashMap>() { { put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1)); put(p1, CollectionUtils.createImmutableList(file1P1C0, file2P1C1)); } }); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, secondCommitTs), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - List hoodieCleanStatsTwo = - runCleaner(config, simulateFailureRetry, simulateMetadataFailure); + commitWithMdt(secondCommitTs, part2ToFileId, testTable, metadataWriter, true, true); + metaClient = HoodieTableMetaClient.reload(metaClient); + + List hoodieCleanStatsTwo = runCleaner(config, simulateFailureRetry, simulateMetadataFailure); + metaClient = HoodieTableMetaClient.reload(metaClient); + assertEquals(2, hoodieCleanStatsTwo.size(), "Should clean one file each from both the partitions"); assertTrue(testTable.baseFileExists(p0, secondCommitTs, file2P0C1)); assertTrue(testTable.baseFileExists(p1, secondCommitTs, file2P1C1)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java index d5657c155f1a3..7be8d3293fd27 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -127,7 +128,8 @@ public void testWriteDuringCompaction(String payloadClass) throws IOException { List writeStatuses = writeData(insertTime, 100, false); Assertions.assertEquals(200, readTableTotalRecordsNum()); // commit the write. The records should be visible now even though the compaction does not complete. - client.commitStats(insertTime, writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); + client.commitStats(insertTime, HoodieListData.eager(writeStatuses), writeStatuses.stream().map(WriteStatus::getStat) + .collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); Assertions.assertEquals(300, readTableTotalRecordsNum()); // after the compaction, total records should remain the same config.setValue(AUTO_COMMIT_ENABLE, "true"); @@ -191,7 +193,7 @@ private List writeData(String instant, int numRecords, boolean doCo org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatuses); if (doCommit) { List writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()); - boolean committed = client.commitStats(instant, writeStats, Option.empty(), metaClient.getCommitActionType()); + boolean committed = client.commitStats(instant, HoodieListData.eager(writeStatuses), writeStats, Option.empty(), metaClient.getCommitActionType()); Assertions.assertTrue(committed); } metaClient = HoodieTableMetaClient.reload(metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java index 1f878ee7bc51e..9ab88d3b52a20 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java @@ -133,7 +133,7 @@ public void testCopyOnWriteRollbackWithTestTable() throws Exception { String f0 = testTable.addRequestedCommit("000") .getFileIdsWithBaseFilesInPartitions("partA").get("partA"); String f1 = testTable.addCommit("001") - .withBaseFilesInPartition("partA", f0) + .withBaseFilesInPartition("partA", f0).getLeft() .getFileIdsWithBaseFilesInPartitions("partB").get("partB"); String f2 = "f2"; testTable.forCommit("001") diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index d68aa4edbddaa..3171e8bb1e4fb 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -497,7 +497,7 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom return; } - if (!tableMetadata.getSyncedInstantTime().isPresent() || tableMetadata instanceof FileSystemBackedTableMetadata) { + if (tableMetadata instanceof FileSystemBackedTableMetadata || !tableMetadata.getSyncedInstantTime().isPresent()) { throw new IllegalStateException("Metadata should have synced some commits or tableMetadata should not be an instance " + "of FileSystemBackedTableMetadata"); } @@ -510,7 +510,7 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom List fsPartitionPaths = testTable.getAllPartitionPaths(); List fsPartitions = new ArrayList<>(); fsPartitionPaths.forEach(entry -> fsPartitions.add(entry.getFileName().toString())); - if (fsPartitions.isEmpty()) { + if (fsPartitions.isEmpty() && testTable.isNonPartitioned()) { fsPartitions.add(""); } List metadataPartitions = tableMetadata.getAllPartitionPaths(); @@ -561,8 +561,7 @@ public HoodieBackedTableMetadataWriter metadataWriter(HoodieWriteConfig clientCo } public HoodieTableMetadata metadata(HoodieWriteConfig clientConfig, HoodieEngineContext hoodieEngineContext) { - return HoodieTableMetadata.create(hoodieEngineContext, clientConfig.getMetadataConfig(), clientConfig.getBasePath(), - clientConfig.getSpillableMapBasePath()); + return HoodieTableMetadata.create(hoodieEngineContext, clientConfig.getMetadataConfig(), clientConfig.getBasePath()); } protected void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMetadata tableMetadata, TableFileSystemView tableView, diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc index a47cbf3784f52..e675687b66745 100644 --- a/hudi-common/src/main/avro/HoodieMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -358,6 +358,46 @@ } ], "default" : null + }, + { + "name": "recordIndexMetadata", + "doc": "Metadata Index that contains information about record keys and their location in the dataset", + "type": [ + "null", + { + "type": "record", + "name": "HoodieRecordIndexInfo", + "fields": [ + { + "name": "partition", + "type": "string", + "doc": "Partition which contains the record", + "avro.java.string": "String" + }, + { + "name": "fileIdHighBits", + "type": "long", + "doc": "fileId which contains the record (high 64 bits)" + }, + { + "name": "fileIdLowBits", + "type": "long", + "doc": "fileId which contains the record (low 64 bits)" + }, + { + "name": "fileIndex", + "type": "int", + "doc": "index of the file" + }, + { + "name": "instantTime", + "type": "long", + "doc": "Epoch time in millisecond at which record was added" + } + ] + } + ], + "default" : null } ] } 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 30d6388a1e2ae..3eda336ca35d5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -29,7 +29,6 @@ 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.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; @@ -450,8 +449,7 @@ private static HoodieTableMetadata createMetadataTable( HoodieMetadataConfig metadataConfig, Path basePath ) { - HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePath.toString(), - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue(), true); + HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePath.toString(), true); return newTableMetadata; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 4f1f4afaf74f0..63e0d5dbe1b79 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -19,11 +19,12 @@ package org.apache.hudi.common.config; import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieNotSupportedException; import javax.annotation.concurrent.Immutable; - import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -246,6 +247,56 @@ public final class HoodieMetadataConfig extends HoodieConfig { .withDocumentation("When there is a pending instant in data table, this config limits the allowed number of deltacommits in metadata table to " + "prevent the metadata table's timeline from growing unboundedly as compaction won't be triggered due to the pending data table instant."); + public static final ConfigProperty RECORD_INDEX_ENABLE_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.enable") + .defaultValue(false) + .sinceVersion("0.14.0") + .withDocumentation("Create the HUDI Record Index within the Metadata Table"); + + public static final ConfigProperty RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.min.filegroup.count") + .defaultValue(10) + .sinceVersion("0.14.0") + .withDocumentation("Minimum number of file groups to use for Record Index."); + + public static final ConfigProperty RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.max.filegroup.count") + .defaultValue(1000) + .sinceVersion("0.14.0") + .withDocumentation("Maximum number of file groups to use for Record Index."); + + public static final ConfigProperty RECORD_INDEX_MAX_FILE_GROUP_SIZE_BYTES_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.max.filegroup.size") + .defaultValue(1024 * 1024 * 1024) + .sinceVersion("0.14.0") + .withDocumentation("Maximum size in bytes of a single file group. Large file group takes longer to compact."); + + public static final ConfigProperty RECORD_INDEX_GROWTH_FACTOR_PROP = ConfigProperty + .key(METADATA_PREFIX + ".record.index.growth.factor") + .defaultValue(2.0f) + .sinceVersion("0.14.0") + .withDocumentation("The current number of records are multiplied by this number when estimating the number of " + + "file groups to create automatically. This helps account for growth in the number of records in the dataset."); + + public static final ConfigProperty MAX_READER_MEMORY_PROP = ConfigProperty + .key(METADATA_PREFIX + ".max.reader.memory") + .defaultValue(1024 * 1024 * 1024L) + .sinceVersion("0.14.0") + .withDocumentation("Max memory to use for the reader to read from metadata"); + + public static final ConfigProperty MAX_READER_BUFFER_SIZE_PROP = ConfigProperty + .key(METADATA_PREFIX + ".max.reader.buffer.size") + .defaultValue(10 * 1024 * 1024) + .sinceVersion("0.14.0") + .withDocumentation("Max memory to use for the reader buffer while merging log blocks"); + + public static final ConfigProperty SPILLABLE_MAP_DIR_PROP = ConfigProperty + .key(METADATA_PREFIX + ".spillable.map.path") + .noDefaultValue() + .withInferFunction(cfg -> Option.of(cfg.getStringOrDefault(FileSystemViewStorageConfig.SPILLABLE_DIR))) + .sinceVersion("0.14.0") + .withDocumentation("Path on local storage to use, when keys read from metadata are held in a spillable map."); + private HoodieMetadataConfig() { super(); } @@ -330,6 +381,38 @@ public int getMaxNumDeltacommitsWhenPending() { return getIntOrDefault(METADATA_MAX_NUM_DELTACOMMITS_WHEN_PENDING); } + public boolean enableRecordIndex() { + return enabled() && getBoolean(RECORD_INDEX_ENABLE_PROP); + } + + public int getRecordIndexMinFileGroupCount() { + return getInt(RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP); + } + + public int getRecordIndexMaxFileGroupCount() { + return getInt(RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP); + } + + public float getRecordIndexGrowthFactor() { + return getFloat(RECORD_INDEX_GROWTH_FACTOR_PROP); + } + + public int getRecordIndexMaxFileGroupSizeBytes() { + return getInt(RECORD_INDEX_MAX_FILE_GROUP_SIZE_BYTES_PROP); + } + + public String getSplliableMapDir() { + return getString(SPILLABLE_MAP_DIR_PROP); + } + + public long getMaxReaderMemory() { + return getLong(MAX_READER_MEMORY_PROP); + } + + public int getMaxReaderBufferSize() { + return getInt(MAX_READER_BUFFER_SIZE_PROP); + } + /** * Builder for {@link HoodieMetadataConfig}. */ @@ -470,6 +553,42 @@ public Builder withMaxNumDeltacommitsWhenPending(int maxNumDeltaCommitsWhenPendi return this; } + public Builder withEnableRecordIndex(boolean enabled) { + metadataConfig.setValue(RECORD_INDEX_ENABLE_PROP, String.valueOf(enabled)); + return this; + } + + public Builder withRecordIndexFileGroupCount(int minCount, int maxCount) { + metadataConfig.setValue(RECORD_INDEX_MIN_FILE_GROUP_COUNT_PROP, String.valueOf(minCount)); + metadataConfig.setValue(RECORD_INDEX_MAX_FILE_GROUP_COUNT_PROP, String.valueOf(maxCount)); + return this; + } + + public Builder withRecordIndexGrowthFactor(float factor) { + metadataConfig.setValue(RECORD_INDEX_GROWTH_FACTOR_PROP, String.valueOf(factor)); + return this; + } + + public Builder withRecordIndexMaxFileGroupSizeBytes(long sizeInBytes) { + metadataConfig.setValue(RECORD_INDEX_MAX_FILE_GROUP_SIZE_BYTES_PROP, String.valueOf(sizeInBytes)); + return this; + } + + public Builder withSpillableMapDir(String dir) { + metadataConfig.setValue(SPILLABLE_MAP_DIR_PROP, dir); + return this; + } + + public Builder withMaxReaderMemory(long mem) { + metadataConfig.setValue(MAX_READER_MEMORY_PROP, String.valueOf(mem)); + return this; + } + + public Builder withMaxReaderBufferSize(long mem) { + metadataConfig.setValue(MAX_READER_BUFFER_SIZE_PROP, String.valueOf(mem)); + return this; + } + public HoodieMetadataConfig build() { metadataConfig.setDefaultValue(ENABLE, getDefaultMetadataEnable(engineType)); metadataConfig.setDefaults(HoodieMetadataConfig.class.getName()); 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 fcb83f9cffbea..7dd400271ad75 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 @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -84,8 +83,6 @@ public class FSUtils { Pattern.compile("^\\.(.+)_(.*)\\.(log|archive)\\.(\\d+)(_((\\d+)-(\\d+)-(\\d+))(.cdc)?)?"); public static final Pattern PREFIX_BY_FILE_ID_PATTERN = Pattern.compile("^(.+)-(\\d+)"); private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; - private static final long MIN_CLEAN_TO_KEEP = 10; - private static final long MIN_ROLLBACK_TO_KEEP = 10; private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_"; private static final PathFilter ALLOW_ALL_FILTER = file -> true; @@ -285,8 +282,7 @@ public static List getAllPartitionPaths(HoodieEngineContext engineContex .enable(useFileListingFromMetadata) .withAssumeDatePartitioning(assumeDatePartitioning) .build(); - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr)) { return tableMetadata.getAllPartitionPaths(); } catch (Exception e) { throw new HoodieException("Error fetching partition paths from metadata table", e); @@ -295,8 +291,7 @@ public static List getAllPartitionPaths(HoodieEngineContext engineContex public static List getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String basePathStr) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr)) { return tableMetadata.getAllPartitionPaths(); } catch (Exception e) { throw new HoodieException("Error fetching partition paths from metadata table", e); @@ -307,8 +302,7 @@ public static Map getFilesInPartitions(HoodieEngineContext HoodieMetadataConfig metadataConfig, String basePathStr, String[] partitionPaths) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue(), true)) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, true)) { 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 7514b4af2d418..b5087864ec3cf 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 @@ -710,8 +710,8 @@ public Set getMetadataPartitions() { /** * @returns true if metadata table has been created and is being used for this dataset, else returns false. */ - public boolean isMetadataTableEnabled() { - return isMetadataPartitionEnabled(MetadataPartitionType.FILES); + public boolean isMetadataTableAvailable() { + return isMetadataPartitionAvailable(MetadataPartitionType.FILES); } /** @@ -720,7 +720,7 @@ public boolean isMetadataTableEnabled() { * @param partition The partition to check * @returns true if the specific partition has been initialized, else returns false. */ - public boolean isMetadataPartitionEnabled(MetadataPartitionType partition) { + public boolean isMetadataPartitionAvailable(MetadataPartitionType partition) { return getMetadataPartitions().contains(partition.getPartitionPath()); } 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 87bdd09db10f2..afb590e9cc045 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 @@ -688,7 +688,7 @@ private static HoodieTableMetaClient newMetaClient(Configuration conf, String ba String payloadClassName, String recordMergerStrategy, FileSystemRetryConfig fileSystemRetryConfig, HoodieMetaserverConfig metaserverConfig) { return metaserverConfig.isMetaserverEnabled() ? (HoodieTableMetaClient) ReflectionUtils.loadClass("org.apache.hudi.common.table.HoodieTableMetaserverClient", - new Class[] {Configuration.class, String.class, ConsistencyGuardConfig.class, String.class, FileSystemRetryConfig.class, String.class, String.class, HoodieMetaserverConfig.class}, + new Class[]{Configuration.class, String.class, ConsistencyGuardConfig.class, String.class, FileSystemRetryConfig.class, String.class, String.class, HoodieMetaserverConfig.class}, conf, basePath, consistencyGuardConfig, recordMergerStrategy, fileSystemRetryConfig, metaserverConfig.getDatabaseName(), metaserverConfig.getTableName(), metaserverConfig) : new HoodieTableMetaClient(conf, basePath, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index a948ff2344d2d..5ec53a9132983 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -18,25 +18,26 @@ package org.apache.hudi.common.table.log.block; -import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.inline.InLineFSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.CloseableMappingIterator; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieAvroHFileReader; import org.apache.hudi.io.storage.HoodieHBaseKVComparator; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.compress.Compression; @@ -44,6 +45,8 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -64,7 +67,7 @@ * base file format. */ public class HoodieHFileDataBlock extends HoodieDataBlock { - + private static final Logger LOG = LoggerFactory.getLogger(HoodieHFileDataBlock.class); private static final int DEFAULT_BLOCK_SIZE = 1024 * 1024; private final Option compressionAlgorithm; @@ -135,8 +138,13 @@ protected byte[] serializeRecords(List records) throws IOException } final byte[] recordBytes = serializeRecord(record, writerSchema); - ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), - "Writing multiple records with same key not supported for " + this.getClass().getName()); + if (sortedRecordsMap.containsKey(recordKey)) { + LOG.error("Found duplicate record with recordKey: " + recordKey); + printRecord("Previous record", sortedRecordsMap.get(recordKey), writerSchema); + printRecord("Current record", recordBytes, writerSchema); + throw new HoodieException(String.format("Writing multiple records with same key %s not supported for %s", + recordKey, this.getClass().getName())); + } sortedRecordsMap.put(recordKey, recordBytes); } @@ -211,4 +219,13 @@ private byte[] serializeRecord(HoodieRecord record, Schema schema) throws IOE } return HoodieAvroUtils.recordToBytes(record, schema).get(); } + + /** + * Print the record in json format + */ + private void printRecord(String msg, byte[] bs, Schema schema) throws IOException { + GenericRecord record = HoodieAvroUtils.bytesToAvro(bs, schema); + byte[] json = HoodieAvroUtils.avroToJson(record, true); + LOG.error(String.format("%s: %s", msg, new String(json))); + } } 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 38e1976910edd..345f8e668aef9 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 @@ -164,7 +164,7 @@ private static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieMeta HoodieTableMetaClient metaClient, SerializableSupplier metadataSupplier) { LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePathV2()); HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); - if (metadataConfig.enabled()) { + if (metaClient.getTableConfig().isMetadataTableAvailable()) { ValidationUtils.checkArgument(metadataSupplier != null, "Metadata supplier is null. Cannot instantiate metadata file system view"); return new HoodieMetadataFileSystemView(metaClient, timeline, metadataSupplier.get()); } @@ -189,7 +189,7 @@ public static HoodieTableFileSystemView createInMemoryFileSystemViewWithTimeline HoodieMetadataConfig metadataConfig, HoodieTimeline timeline) { LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath()); - if (metadataConfig.enabled()) { + if (metaClient.getTableConfig().isMetadataTableAvailable()) { return new HoodieMetadataFileSystemView(engineContext, metaClient, timeline, metadataConfig); } if (metaClient.getMetaserverConfig().isMetaserverEnabled()) { @@ -229,7 +229,7 @@ public static FileSystemViewManager createViewManager(final HoodieEngineContext final HoodieCommonConfig commonConfig, final String basePath) { return createViewManager(context, metadataConfig, config, commonConfig, - () -> HoodieTableMetadata.create(context, metadataConfig, basePath, config.getSpillableDir(), true)); + () -> HoodieTableMetadata.create(context, metadataConfig, basePath, true)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java index 8e508a0292c73..8ed597ed920df 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java @@ -25,13 +25,13 @@ import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.compress.Compression; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.orc.CompressionKind; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; @@ -89,9 +89,8 @@ protected HoodieFileWriter newHFileFileWriter( HoodieHFileConfig hfileConfig = new HoodieHFileConfig(conf, Compression.Algorithm.valueOf(config.getString(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM_NAME)), config.getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE), - config.getLong(HoodieStorageConfig.HFILE_MAX_FILE_SIZE), - HoodieAvroHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, - filter, HFILE_COMPARATOR); + config.getLong(HoodieStorageConfig.HFILE_MAX_FILE_SIZE), HoodieAvroHFileReader.KEY_FIELD_NAME, + PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION, filter, HFILE_COMPARATOR); return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java index 1a9da7e3b8f50..dec6a5b29f365 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java @@ -545,6 +545,33 @@ public void close() { } } + @Override + public ClosableIterator getRecordKeyIterator() { + final HFileScanner scanner = reader.getScanner(false, false); + return new ClosableIterator() { + @Override + public boolean hasNext() { + try { + return scanner.next(); + } catch (IOException e) { + throw new HoodieException("Error while scanning for keys", e); + } + } + + @Override + public String next() { + Cell cell = scanner.getCell(); + final byte[] keyBytes = copyKeyFromCell(cell); + return new String(keyBytes); + } + + @Override + public void close() { + scanner.close(); + } + }; + } + private static class RecordIterator implements ClosableIterator { private final HFileScanner scanner; diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java index c143f782d4839..5b66c04045b3d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java @@ -18,6 +18,16 @@ package org.apache.hudi.io.storage; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieDuplicateKeyException; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -30,14 +40,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.io.Writable; -import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.DataInput; import java.io.DataOutput; @@ -54,8 +58,8 @@ */ public class HoodieAvroHFileWriter implements HoodieAvroFileWriter { + private static final Logger LOG = LoggerFactory.getLogger(HoodieAvroHFileWriter.class); private static AtomicLong recordIndex = new AtomicLong(1); - private final Path file; private HoodieHFileConfig hfileConfig; private final HoodieWrapperFileSystem fs; @@ -68,6 +72,7 @@ public class HoodieAvroHFileWriter private HFile.Writer writer; private String minRecordKey; private String maxRecordKey; + private String prevRecordKey; // This is private in CacheConfig so have been copied here. private static String DROP_BEHIND_CACHE_COMPACTION_KEY = "hbase.hfile.drop.behind.compaction"; @@ -106,6 +111,7 @@ public HoodieAvroHFileWriter(String instantTime, Path file, HoodieHFileConfig hf .create(); writer.appendFileInfo(HoodieAvroHFileReader.SCHEMA_KEY.getBytes(), schema.toString().getBytes()); + this.prevRecordKey = ""; } @Override @@ -126,6 +132,10 @@ public boolean canWrite() { @Override public void writeAvro(String recordKey, IndexedRecord record) throws IOException { + if (prevRecordKey.equals(recordKey)) { + throw new HoodieDuplicateKeyException("Duplicate recordKey " + recordKey + " found while writing to HFile." + + "Record payload: " + record); + } byte[] value = null; boolean isRecordSerialized = false; if (keyFieldSchema.isPresent()) { @@ -154,6 +164,7 @@ public void writeAvro(String recordKey, IndexedRecord record) throws IOException } maxRecordKey = recordKey; } + prevRecordKey = recordKey; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java index cccc51432cddf..1420424a58b01 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java @@ -37,13 +37,12 @@ import org.apache.orc.TypeDescription; import java.io.IOException; +import java.util.Iterator; import java.util.Objects; import java.util.Set; /** * {@link HoodieFileReader} implementation for ORC format. - * - * @param Record implementation that permits field access by integer index. */ public class HoodieAvroOrcReader extends HoodieAvroFileReaderBase { @@ -73,13 +72,12 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) { if (!Objects.equals(readerSchema, requestedSchema)) { throw new UnsupportedOperationException("Schema projections are not supported in HFile reader"); } - try { - Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); + try (Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf))) { TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(readerSchema); RecordReader recordReader = reader.rows(new Options(conf).schema(orcSchema)); return new OrcReaderIterator<>(recordReader, readerSchema, orcSchema); @@ -88,13 +86,34 @@ protected ClosableIterator getIndexedRecordIterator(Schema reader } } + @Override + public ClosableIterator getRecordKeyIterator() { + final Iterator iterator = orcUtils.readRowKeys(conf, path).iterator(); + return new ClosableIterator() { + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public String next() { + return iterator.next(); + } + + @Override + public void close() { + } + }; + } + @Override public Schema getSchema() { return orcUtils.readAvroSchema(conf, path); } @Override - public void close() {} + public void close() { + } @Override public long getTotalRecords() { diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index 0b97d052cbb91..ad4d1f16a60ce 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -18,17 +18,19 @@ package org.apache.hudi.io.storage; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.BaseFileUtils; -import org.apache.hudi.common.util.collection.ClosableIterator; -import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetReaderIterator; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -168,4 +170,26 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema readerIterators.add(parquetReaderIterator); return parquetReaderIterator; } + + @Override + public ClosableIterator getRecordKeyIterator() throws IOException { + ClosableIterator recordKeyIterator = getIndexedRecordIterator(HoodieAvroUtils.getRecordKeySchema()); + return new ClosableIterator() { + @Override + public boolean hasNext() { + return recordKeyIterator.hasNext(); + } + + @Override + public String next() { + Object obj = recordKeyIterator.next(); + return ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + } + + @Override + public void close() { + recordKeyIterator.close(); + } + }; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java index 4dc1a73e6ddb3..b22682be46ef9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java @@ -82,6 +82,29 @@ protected void setPartitionPathField(int position, Object fieldValue, T row) { }; } + @Override + public ClosableIterator getRecordKeyIterator() throws IOException { + Schema schema = HoodieAvroUtils.getRecordKeySchema(); + ClosableIterator> skeletonIterator = skeletonFileReader.getRecordIterator(schema, schema); + return new ClosableIterator() { + @Override + public void close() { + skeletonIterator.close(); + } + + @Override + public boolean hasNext() { + return skeletonIterator.hasNext(); + } + + @Override + public String next() { + HoodieRecord skeletonRecord = skeletonIterator.next(); + return skeletonRecord.getRecordKey(schema, HoodieRecord.RECORD_KEY_METADATA_FIELD); + } + }; + } + protected abstract void setPartitionField(int position, Object fieldValue, T row); @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java index 91f88d35dccd9..00fff9a220c64 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java @@ -57,6 +57,8 @@ default ClosableIterator> getRecordIterator() throws IOException return getRecordIterator(getSchema()); } + ClosableIterator getRecordKeyIterator() throws IOException; + Schema getSchema(); void close(); diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java index 5ce377901a4ba..64cc607ef6324 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java @@ -46,8 +46,9 @@ public class HoodieHFileConfig { private final String keyFieldName; public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm compressionAlgorithm, int blockSize, - long maxFileSize, String keyFieldName, boolean prefetchBlocksOnOpen, boolean cacheDataInL1, - boolean dropBehindCacheCompaction, BloomFilter bloomFilter, CellComparator hfileComparator) { + long maxFileSize, String keyFieldName, + boolean prefetchBlocksOnOpen, boolean cacheDataInL1, boolean dropBehindCacheCompaction, + BloomFilter bloomFilter, CellComparator hfileComparator) { this.hadoopConf = hadoopConf; this.compressionAlgorithm = compressionAlgorithm; this.blockSize = blockSize; @@ -103,4 +104,5 @@ public CellComparator getHFileComparator() { public String getKeyFieldName() { return keyFieldName; } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 568b15a7bbdb9..4cfbc9982379a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; @@ -58,11 +59,10 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; -import java.util.TreeSet; import java.util.function.Function; import java.util.stream.Collectors; @@ -73,35 +73,24 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { private static final Logger LOG = LoggerFactory.getLogger(BaseTableMetadata.class); - protected static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; - // NOTE: Buffer-size is deliberately set pretty low, since MT internally is relying - // on HFile (serving as persisted binary key-value mapping) to do caching - protected static final int BUFFER_SIZE = 10 * 1024; // 10Kb - - protected final transient HoodieEngineContext engineContext; + protected transient HoodieEngineContext engineContext; protected final SerializablePath dataBasePath; protected final HoodieTableMetaClient dataMetaClient; protected final Option metrics; protected final HoodieMetadataConfig metadataConfig; - // Directory used for Spillable Map when merging records - protected final String spillableMapDirectory; - protected boolean isMetadataTableEnabled; - protected boolean isBloomFilterIndexEnabled = false; - protected boolean isColumnStatsIndexEnabled = false; + protected boolean isMetadataTableInitialized; - protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, - String dataBasePath, String spillableMapDirectory) { + protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String dataBasePath) { this.engineContext = engineContext; this.dataBasePath = new SerializablePath(new CachingPath(dataBasePath)); this.dataMetaClient = HoodieTableMetaClient.builder() .setConf(engineContext.getHadoopConf().get()) .setBasePath(dataBasePath) .build(); - this.spillableMapDirectory = spillableMapDirectory; this.metadataConfig = metadataConfig; + this.isMetadataTableInitialized = dataMetaClient.getTableConfig().isMetadataTableAvailable(); - this.isMetadataTableEnabled = metadataConfig.enabled(); if (metadataConfig.enableMetrics()) { this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata"))); } else { @@ -109,6 +98,13 @@ protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataCon } } + protected HoodieEngineContext getEngineContext() { + if (engineContext == null) { + engineContext = new HoodieLocalEngineContext(dataMetaClient.getHadoopConf()); + } + return engineContext; + } + /** * Return the list of partitions in the dataset. *

    @@ -119,17 +115,12 @@ protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataCon */ @Override public List getAllPartitionPaths() throws IOException { - if (isMetadataTableEnabled) { - try { - return fetchAllPartitionPaths(); - } catch (Exception e) { - throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); - } + ValidationUtils.checkArgument(isMetadataTableInitialized); + try { + return fetchAllPartitionPaths(); + } catch (Exception e) { + throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); } - - FileSystemBackedTableMetadata fileSystemBackedTableMetadata = - createFileSystemBackedTableMetadata(); - return fileSystemBackedTableMetadata.getAllPartitionPaths(); } /** @@ -143,46 +134,33 @@ public List getAllPartitionPaths() throws IOException { * @param partitionPath The absolute path of the partition to list */ @Override - public FileStatus[] getAllFilesInPartition(Path partitionPath) - throws IOException { - if (isMetadataTableEnabled) { - try { - return fetchAllFilesInPartition(partitionPath); - } catch (Exception e) { - throw new HoodieMetadataException("Failed to retrieve files in partition " + partitionPath + " from metadata", e); - } + public FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException { + ValidationUtils.checkArgument(isMetadataTableInitialized); + try { + return fetchAllFilesInPartition(partitionPath); + } catch (Exception e) { + throw new HoodieMetadataException("Failed to retrieve files in partition " + partitionPath + " from metadata", e); } - - FileSystemBackedTableMetadata fileSystemBackedTableMetadata = - createFileSystemBackedTableMetadata(); - return fileSystemBackedTableMetadata.getAllFilesInPartition(partitionPath); } @Override - public Map getAllFilesInPartitions(Collection partitions) - throws IOException { + public Map getAllFilesInPartitions(Collection partitions) throws IOException { + ValidationUtils.checkArgument(isMetadataTableInitialized); if (partitions.isEmpty()) { return Collections.emptyMap(); } - if (isMetadataTableEnabled) { - try { - List partitionPaths = partitions.stream().map(Path::new).collect(Collectors.toList()); - return fetchAllFilesInPartitionPaths(partitionPaths); - } catch (Exception e) { - throw new HoodieMetadataException("Failed to retrieve files in partition from metadata", e); - } + try { + List partitionPaths = partitions.stream().map(Path::new).collect(Collectors.toList()); + return fetchAllFilesInPartitionPaths(partitionPaths); + } catch (Exception e) { + throw new HoodieMetadataException("Failed to retrieve files in partition from metadata", e); } - - FileSystemBackedTableMetadata fileSystemBackedTableMetadata = - createFileSystemBackedTableMetadata(); - return fileSystemBackedTableMetadata.getAllFilesInPartitions(partitions); } @Override - public Option getBloomFilter(final String partitionName, final String fileName) - throws HoodieMetadataException { - if (!isBloomFilterIndexEnabled) { + public Option getBloomFilter(final String partitionName, final String fileName) throws HoodieMetadataException { + if (!dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.BLOOM_FILTERS)) { LOG.error("Metadata bloom filter index is disabled!"); return Option.empty(); } @@ -201,7 +179,7 @@ public Option getBloomFilter(final String partitionName, final Stri @Override public Map, BloomFilter> getBloomFilters(final List> partitionNameFileNameList) throws HoodieMetadataException { - if (!isBloomFilterIndexEnabled) { + if (!dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.BLOOM_FILTERS)) { LOG.error("Metadata bloom filter index is disabled!"); return Collections.emptyMap(); } @@ -210,44 +188,40 @@ public Map, BloomFilter> getBloomFilters(final List partitionIDFileIDSortedStrings = new TreeSet<>(); + Set partitionIDFileIDStrings = new HashSet<>(); Map> fileToKeyMap = new HashMap<>(); - // TODO simplify (no sorting is required) partitionNameFileNameList.forEach(partitionNameFileNamePair -> { - final String bloomFilterIndexKey = HoodieMetadataPayload.getBloomFilterIndexKey( - new PartitionIndexID(partitionNameFileNamePair.getLeft()), new FileIndexID(partitionNameFileNamePair.getRight())); - partitionIDFileIDSortedStrings.add(bloomFilterIndexKey); - fileToKeyMap.put(bloomFilterIndexKey, partitionNameFileNamePair); - } - ); - - List partitionIDFileIDStrings = new ArrayList<>(partitionIDFileIDSortedStrings); - List>>> hoodieRecordList = - getRecordsByKeys(partitionIDFileIDStrings, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath()); + final String bloomFilterIndexKey = HoodieMetadataPayload.getBloomFilterIndexKey( + new PartitionIndexID(partitionNameFileNamePair.getLeft()), new FileIndexID(partitionNameFileNamePair.getRight())); + partitionIDFileIDStrings.add(bloomFilterIndexKey); + fileToKeyMap.put(bloomFilterIndexKey, partitionNameFileNamePair); + }); + + List partitionIDFileIDStringsList = new ArrayList<>(partitionIDFileIDStrings); + Map> hoodieRecords = + getRecordsByKeys(partitionIDFileIDStringsList, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_BLOOM_FILTERS_METADATA_STR, (timer.endTimer() / partitionIDFileIDStrings.size()))); - Map, BloomFilter> partitionFileToBloomFilterMap = new HashMap<>(); - for (final Pair>> entry : hoodieRecordList) { - if (entry.getRight().isPresent()) { - final Option bloomFilterMetadata = - entry.getRight().get().getData().getBloomFilterMetadata(); - if (bloomFilterMetadata.isPresent()) { - if (!bloomFilterMetadata.get().getIsDeleted()) { - ValidationUtils.checkState(fileToKeyMap.containsKey(entry.getLeft())); - // NOTE: We have to duplicate the [[ByteBuffer]] object here since: - // - Reading out [[ByteBuffer]] mutates its state - // - [[BloomFilterMetadata]] could be re-used, and hence have to stay immutable - final ByteBuffer bloomFilterByteBuffer = - bloomFilterMetadata.get().getBloomFilter().duplicate(); - final String bloomFilterType = bloomFilterMetadata.get().getType(); - final BloomFilter bloomFilter = BloomFilterFactory.fromString( - StandardCharsets.UTF_8.decode(bloomFilterByteBuffer).toString(), bloomFilterType); - partitionFileToBloomFilterMap.put(fileToKeyMap.get(entry.getLeft()), bloomFilter); - } - } else { - LOG.error("Meta index bloom filter missing for: " + fileToKeyMap.get(entry.getLeft())); + Map, BloomFilter> partitionFileToBloomFilterMap = new HashMap<>(hoodieRecords.size()); + for (final Map.Entry> entry : hoodieRecords.entrySet()) { + final Option bloomFilterMetadata = + entry.getValue().getData().getBloomFilterMetadata(); + if (bloomFilterMetadata.isPresent()) { + if (!bloomFilterMetadata.get().getIsDeleted()) { + ValidationUtils.checkState(fileToKeyMap.containsKey(entry.getKey())); + // NOTE: We have to duplicate the [[ByteBuffer]] object here since: + // - Reading out [[ByteBuffer]] mutates its state + // - [[BloomFilterMetadata]] could be re-used, and hence have to stay immutable + final ByteBuffer bloomFilterByteBuffer = + bloomFilterMetadata.get().getBloomFilter().duplicate(); + final String bloomFilterType = bloomFilterMetadata.get().getType(); + final BloomFilter bloomFilter = BloomFilterFactory.fromString( + StandardCharsets.UTF_8.decode(bloomFilterByteBuffer).toString(), bloomFilterType); + partitionFileToBloomFilterMap.put(fileToKeyMap.get(entry.getKey()), bloomFilter); } + } else { + LOG.error("Meta index bloom filter missing for: " + fileToKeyMap.get(entry.getKey())); } } return partitionFileToBloomFilterMap; @@ -256,49 +230,70 @@ public Map, BloomFilter> getBloomFilters(final List, HoodieMetadataColumnStats> getColumnStats(final List> partitionNameFileNameList, final String columnName) throws HoodieMetadataException { - if (!isColumnStatsIndexEnabled) { + if (!dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.COLUMN_STATS)) { LOG.error("Metadata column stats index is disabled!"); return Collections.emptyMap(); } Map> columnStatKeyToFileNameMap = new HashMap<>(); - TreeSet sortedKeys = new TreeSet<>(); + Set columnStatKeyset = new HashSet<>(); final ColumnIndexID columnIndexID = new ColumnIndexID(columnName); for (Pair partitionNameFileNamePair : partitionNameFileNameList) { final String columnStatsIndexKey = HoodieMetadataPayload.getColumnStatsIndexKey( new PartitionIndexID(partitionNameFileNamePair.getLeft()), new FileIndexID(partitionNameFileNamePair.getRight()), columnIndexID); - sortedKeys.add(columnStatsIndexKey); + columnStatKeyset.add(columnStatsIndexKey); columnStatKeyToFileNameMap.put(columnStatsIndexKey, partitionNameFileNamePair); } - List columnStatKeys = new ArrayList<>(sortedKeys); + List columnStatKeylist = new ArrayList<>(columnStatKeyset); HoodieTimer timer = HoodieTimer.start(); - List>>> hoodieRecordList = - getRecordsByKeys(columnStatKeys, MetadataPartitionType.COLUMN_STATS.getPartitionPath()); + Map> hoodieRecords = + getRecordsByKeys(columnStatKeylist, MetadataPartitionType.COLUMN_STATS.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_COLUMN_STATS_METADATA_STR, timer.endTimer())); Map, HoodieMetadataColumnStats> fileToColumnStatMap = new HashMap<>(); - for (final Pair>> entry : hoodieRecordList) { - if (entry.getRight().isPresent()) { - final Option columnStatMetadata = - entry.getRight().get().getData().getColumnStatMetadata(); - if (columnStatMetadata.isPresent()) { - if (!columnStatMetadata.get().getIsDeleted()) { - ValidationUtils.checkState(columnStatKeyToFileNameMap.containsKey(entry.getLeft())); - final Pair partitionFileNamePair = columnStatKeyToFileNameMap.get(entry.getLeft()); - ValidationUtils.checkState(!fileToColumnStatMap.containsKey(partitionFileNamePair)); - fileToColumnStatMap.put(partitionFileNamePair, columnStatMetadata.get()); - } - } else { - LOG.error("Meta index column stats missing for: " + entry.getLeft()); + for (final Map.Entry> entry : hoodieRecords.entrySet()) { + final Option columnStatMetadata = + entry.getValue().getData().getColumnStatMetadata(); + if (columnStatMetadata.isPresent()) { + if (!columnStatMetadata.get().getIsDeleted()) { + ValidationUtils.checkState(columnStatKeyToFileNameMap.containsKey(entry.getKey())); + final Pair partitionFileNamePair = columnStatKeyToFileNameMap.get(entry.getKey()); + ValidationUtils.checkState(!fileToColumnStatMap.containsKey(partitionFileNamePair)); + fileToColumnStatMap.put(partitionFileNamePair, columnStatMetadata.get()); } + } else { + LOG.error("Meta index column stats missing for: " + entry.getKey()); } } return fileToColumnStatMap; } + /** + * Reads record keys from record-level index. + *

    + * If the Metadata Table is not enabled, an exception is thrown to distinguish this from the absence of the key. + * + * @param recordKeys The list of record keys to read + */ + @Override + public Map readRecordIndex(List recordKeys) { + // If record index is not initialized yet, we cannot return an empty result here unlike the code for reading from other + // indexes. This is because results from this function are used for upserts and returning an empty result here would lead + // to existing records being inserted again causing duplicates. + // The caller is required to check for record index existence in MDT before calling this method. + ValidationUtils.checkState(dataMetaClient.getTableConfig().isMetadataPartitionAvailable(MetadataPartitionType.RECORD_INDEX), + "Record index is not initialized in MDT"); + + Map> result = getRecordsByKeys(recordKeys, MetadataPartitionType.RECORD_INDEX.getPartitionPath()); + Map recordKeyToLocation = new HashMap<>(result.size()); + result.forEach((key, record) -> recordKeyToLocation.put(key, record.getData().getRecordGlobalLocation())); + + return recordKeyToLocation; + } + /** * Returns a list of all partitions. */ @@ -366,29 +361,23 @@ Map fetchAllFilesInPartitionPaths(List partitionPath ); HoodieTimer timer = HoodieTimer.start(); - List>>> partitionIdRecordPairs = + Map> partitionIdRecordPairs = getRecordsByKeys(new ArrayList<>(partitionIdToPathMap.keySet()), MetadataPartitionType.FILES.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); FileSystem fs = partitionPaths.get(0).getFileSystem(getHadoopConf()); - Map partitionPathToFilesMap = partitionIdRecordPairs.parallelStream() - .map(pair -> { - String partitionId = pair.getKey(); - Option> recordOpt = pair.getValue(); - + Map partitionPathToFilesMap = partitionIdRecordPairs.entrySet().stream() + .map(e -> { + final String partitionId = e.getKey(); Path partitionPath = partitionIdToPathMap.get(partitionId); - return recordOpt.map(record -> { - HoodieMetadataPayload metadataPayload = record.getData(); - checkForSpuriousDeletes(metadataPayload, partitionId); + HoodieMetadataPayload metadataPayload = e.getValue().getData(); + checkForSpuriousDeletes(metadataPayload, partitionId); - FileStatus[] files = metadataPayload.getFileStatuses(fs, partitionPath); - return Pair.of(partitionPath.toString(), files); - }) - .orElse(null); + FileStatus[] files = metadataPayload.getFileStatuses(fs, partitionPath); + return Pair.of(partitionPath.toString(), files); }) - .filter(Objects::nonNull) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); LOG.info("Listed files in partitions from metadata: partition list =" + Arrays.toString(partitionPaths.toArray())); @@ -411,18 +400,9 @@ private void checkForSpuriousDeletes(HoodieMetadataPayload metadataPayload, Stri } } - private FileSystemBackedTableMetadata createFileSystemBackedTableMetadata() { - return new FileSystemBackedTableMetadata(getEngineContext(), dataMetaClient.getSerializableHadoopConf(), dataBasePath.toString(), - metadataConfig.shouldAssumeDatePartitioning()); - } - protected abstract Option> getRecordByKey(String key, String partitionName); - public abstract List>>> getRecordsByKeys(List key, String partitionName); - - protected HoodieEngineContext getEngineContext() { - return engineContext != null ? engineContext : new HoodieLocalEngineContext(getHadoopConf()); - } + protected abstract Map> getRecordsByKeys(List keys, String partitionName); public HoodieMetadataConfig getMetadataConfig() { return metadataConfig; @@ -436,4 +416,8 @@ protected String getLatestDataInstantTime() { return dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); } + + public boolean isMetadataTableInitialized() { + return isMetadataTableInitialized; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java index 75d2e1479eba5..69c237d6684d1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -106,6 +107,7 @@ private List getPartitionPathWithPathPrefix(String relativePathPrefix) t int listingParallelism = Math.min(DEFAULT_LISTING_PARALLELISM, pathsToList.size()); // List all directories in parallel + engineContext.setJobStatus(this.getClass().getSimpleName(), "Listing all partitions with prefix " + relativePathPrefix); List dirToFileListing = engineContext.flatMap(pathsToList, path -> { FileSystem fileSystem = path.getFileSystem(hadoopConf.get()); return Arrays.stream(fileSystem.listStatus(path)); @@ -118,6 +120,7 @@ private List getPartitionPathWithPathPrefix(String relativePathPrefix) t if (!dirToFileListing.isEmpty()) { // result below holds a list of pair. first entry in the pair optionally holds the deduced list of partitions. // and second entry holds optionally a directory path to be processed further. + engineContext.setJobStatus(this.getClass().getSimpleName(), "Processing listed partitions"); List, Option>> result = engineContext.map(dirToFileListing, fileStatus -> { FileSystem fileSystem = fileStatus.getPath().getFileSystem(hadoopConf.get()); if (fileStatus.isDirectory()) { @@ -152,6 +155,7 @@ public Map getAllFilesInPartitions(Collection part int parallelism = Math.min(DEFAULT_LISTING_PARALLELISM, partitionPaths.size()); + engineContext.setJobStatus(this.getClass().getSimpleName(), "Listing all files in " + partitionPaths.size() + " partitions"); List> partitionToFiles = engineContext.map(new ArrayList<>(partitionPaths), partitionPathStr -> { Path partitionPath = new Path(partitionPathStr); FileSystem fs = partitionPath.getFileSystem(hadoopConf.get()); @@ -202,4 +206,14 @@ public Map, HoodieMetadataColumnStats> getColumnStats(final public HoodieData> getRecordsByKeyPrefixes(List keyPrefixes, String partitionName, boolean shouldLoadInMemory) { throw new HoodieMetadataException("Unsupported operation: getRecordsByKeyPrefixes!"); } + + @Override + public Map readRecordIndex(List recordKeys) { + throw new HoodieMetadataException("Unsupported operation: readRecordIndex!"); + } + + @Override + public int getNumFileGroupsForPartition(MetadataPartitionType partition) { + throw new HoodieMetadataException("Unsupported operation: getNumFileGroupsForPartition"); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 7ca9d6573e897..dea20b2ab6f00 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -36,11 +36,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -62,14 +63,11 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FULL_SCAN_LOG_FILES; -import static org.apache.hudi.common.util.CollectionUtils.isNullOrEmpty; import static org.apache.hudi.common.util.CollectionUtils.toStream; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; @@ -96,14 +94,16 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { private final Transient, Pair, HoodieMetadataLogRecordReader>>> partitionReaders = Transient.lazy(ConcurrentHashMap::new); - public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, - String datasetBasePath, String spillableMapDirectory) { - this(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory, false); + // Latest file slices in the metadata partitions + private final Map> partitionFileSliceMap = new ConcurrentHashMap<>(); + + public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath) { + this(engineContext, metadataConfig, datasetBasePath, false); } public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, - String datasetBasePath, String spillableMapDirectory, boolean reuse) { - super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory); + String datasetBasePath, boolean reuse) { + super(engineContext, metadataConfig, datasetBasePath); this.reuse = reuse; this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath.toString()); @@ -111,7 +111,7 @@ public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetada } private void initIfNeeded() { - if (!isMetadataTableEnabled) { + if (!isMetadataTableInitialized) { if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) { LOG.info("Metadata table is disabled."); } @@ -120,17 +120,15 @@ private void initIfNeeded() { this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(getHadoopConf()).setBasePath(metadataBasePath).build(); this.metadataFileSystemView = getFileSystemView(metadataMetaClient); this.metadataTableConfig = metadataMetaClient.getTableConfig(); - this.isBloomFilterIndexEnabled = metadataConfig.isBloomFilterIndexEnabled(); - this.isColumnStatsIndexEnabled = metadataConfig.isColumnStatsIndexEnabled(); } catch (TableNotFoundException e) { LOG.warn("Metadata table was not found at path " + metadataBasePath); - this.isMetadataTableEnabled = false; + this.isMetadataTableInitialized = false; this.metadataMetaClient = null; this.metadataFileSystemView = null; this.metadataTableConfig = null; } catch (Exception e) { LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e); - this.isMetadataTableEnabled = false; + this.isMetadataTableInitialized = false; this.metadataMetaClient = null; this.metadataFileSystemView = null; this.metadataTableConfig = null; @@ -140,8 +138,8 @@ private void initIfNeeded() { @Override protected Option> getRecordByKey(String key, String partitionName) { - List>>> recordsByKeys = getRecordsByKeys(Collections.singletonList(key), partitionName); - return recordsByKeys.size() == 0 ? Option.empty() : recordsByKeys.get(0).getValue(); + Map> recordsByKeys = getRecordsByKeys(Collections.singletonList(key), partitionName); + return Option.ofNullable(recordsByKeys.get(key)); } @Override @@ -152,7 +150,7 @@ public List getPartitionPathWithPathPrefixes(List relativePathPr // Partition paths stored in metadata table do not have the slash at the end. // If the relativePathPrefix is empty, return all partition paths; // else if the relative path prefix is the same as the path, this is an exact match; - // else, we need to make sure the path is a sub-directory of relativePathPrefix, by + // else, we need to make sure the path is a subdirectory of relativePathPrefix, by // checking if the path starts with relativePathPrefix appended by a slash ("/"). StringUtils.isNullOrEmpty(relativePathPrefix) || p.equals(relativePathPrefix) || p.startsWith(relativePathPrefix + "/"))) @@ -163,16 +161,16 @@ public List getPartitionPathWithPathPrefixes(List relativePathPr public HoodieData> getRecordsByKeyPrefixes(List keyPrefixes, String partitionName, boolean shouldLoadInMemory) { - // Sort the columns so that keys are looked up in order + // Sort the prefixes so that keys are looked up in order List sortedKeyPrefixes = new ArrayList<>(keyPrefixes); Collections.sort(sortedKeyPrefixes); // NOTE: Since we partition records to a particular file-group by full key, we will have // to scan all file-groups for all key-prefixes as each of these might contain some // records matching the key-prefix - List partitionFileSlices = - HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices( - metadataMetaClient, metadataFileSystemView, partitionName); + List partitionFileSlices = partitionFileSliceMap.computeIfAbsent(partitionName, + k -> HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, metadataFileSystemView, partitionName)); + ValidationUtils.checkState(partitionFileSlices.size() > 0, "Number of file slices for partition " + partitionName + " should be > 0"); return (shouldLoadInMemory ? HoodieListData.lazy(partitionFileSlices) : engineContext.parallelize(partitionFileSlices)) @@ -193,75 +191,101 @@ public HoodieData> getRecordsByKeyPrefixes(L return Collections.emptyIterator(); } - boolean fullKeys = false; + boolean fullKeys = false; - Map>> logRecords = - readLogRecords(logRecordScanner, sortedKeyPrefixes, fullKeys, timings); + Map> logRecords = + readLogRecords(logRecordScanner, sortedKeyPrefixes, fullKeys, timings); - List>>> mergedRecords = - readFromBaseAndMergeWithLogRecords(baseFileReader, sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName); + Map> mergedRecords = + readFromBaseAndMergeWithLogRecords(baseFileReader, sortedKeyPrefixes, fullKeys, logRecords, timings, partitionName); - LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", - sortedKeyPrefixes.size(), timings)); + LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", + sortedKeyPrefixes.size(), timings)); - return mergedRecords.stream() - .map(keyRecordPair -> keyRecordPair.getValue().orElse(null)) - .filter(Objects::nonNull) - .iterator(); - } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for " + sortedKeyPrefixes.size() + " key : ", ioe); - } finally { - closeReader(readers); - } - }); + return mergedRecords.values().iterator(); + } catch (IOException ioe) { + throw new HoodieIOException("Error merging records from metadata table for " + sortedKeyPrefixes.size() + " key : ", ioe); + } finally { + closeReader(readers); + } + }); } @Override - public List>>> getRecordsByKeys(List keys, - String partitionName) { - // Sort the columns so that keys are looked up in order - List sortedKeys = new ArrayList<>(keys); - Collections.sort(sortedKeys); - Map, List> partitionFileSliceToKeysMap = getPartitionFileSliceToKeysMapping(partitionName, sortedKeys); - List>>> result = new ArrayList<>(); - AtomicInteger fileSlicesKeysCount = new AtomicInteger(); - partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> { - Pair, HoodieMetadataLogRecordReader> readers = - getOrCreateReaders(partitionName, partitionFileSlicePair.getRight()); - try { - List timings = new ArrayList<>(); - HoodieSeekingFileReader baseFileReader = readers.getKey(); - HoodieMetadataLogRecordReader logRecordScanner = readers.getRight(); - if (baseFileReader == null && logRecordScanner == null) { - return; - } + protected Map> getRecordsByKeys(List keys, String partitionName) { + if (keys.isEmpty()) { + return Collections.emptyMap(); + } - boolean fullKeys = true; - Map>> logRecords = - readLogRecords(logRecordScanner, fileSliceKeys, fullKeys, timings); - - result.addAll(readFromBaseAndMergeWithLogRecords(baseFileReader, fileSliceKeys, fullKeys, logRecords, - timings, partitionName)); - - LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", - fileSliceKeys.size(), timings)); - fileSlicesKeysCount.addAndGet(fileSliceKeys.size()); - } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for " + sortedKeys.size() + " key : ", ioe); - } finally { - if (!reuse) { - closeReader(readers); - } + // sort + Map> result; + + // Load the file slices for the partition. Each file slice is a shard which saves a portion of the keys. + List partitionFileSlices = partitionFileSliceMap.computeIfAbsent(partitionName, + k -> HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, metadataFileSystemView, partitionName)); + final int numFileSlices = partitionFileSlices.size(); + ValidationUtils.checkState(numFileSlices > 0, "Number of file slices for partition " + partitionName + " should be > 0"); + + // Lookup keys from each file slice + if (numFileSlices == 1) { + // Optimization for a single slice for smaller metadata table partitions + result = lookupKeysFromFileSlice(partitionName, keys, partitionFileSlices.get(0)); + } else { + // Parallel lookup for large sized partitions with many file slices + // Partition the keys by the file slice which contains it + ArrayList> partitionedKeys = new ArrayList<>(numFileSlices); + for (int i = 0; i < numFileSlices; ++i) { + partitionedKeys.add(new ArrayList<>()); } - }); + keys.forEach(key -> { + int shardIndex = HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, numFileSlices); + partitionedKeys.get(shardIndex).add(key); + }); + + result = new HashMap<>(keys.size()); + getEngineContext().setJobStatus(this.getClass().getSimpleName(), "Reading keys from metadata table partition " + partitionName); + getEngineContext().map(partitionedKeys, keysList -> { + if (keysList.isEmpty()) { + return Collections.>emptyMap(); + } + int shardIndex = HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(keysList.get(0), numFileSlices); + return lookupKeysFromFileSlice(partitionName, keysList, partitionFileSlices.get(shardIndex)); + }, partitionedKeys.size()).forEach(result::putAll); + } return result; } - private Map>> readLogRecords(HoodieMetadataLogRecordReader logRecordReader, - List keys, - boolean fullKey, - List timings) { + /** + * Lookup list of keys from a single file slice. + * + * @param partitionName Name of the partition + * @param keys The list of keys to lookup + * @param fileSlice The file slice to read + * @return A {@code Map} of key name to {@code HoodieRecord} for the keys which were found in the file slice + */ + private Map> lookupKeysFromFileSlice(String partitionName, List keys, FileSlice fileSlice) { + Pair, HoodieMetadataLogRecordReader> readers = getOrCreateReaders(partitionName, fileSlice); + try { + List timings = new ArrayList<>(); + HoodieSeekingFileReader baseFileReader = readers.getKey(); + HoodieMetadataLogRecordReader logRecordScanner = readers.getRight(); + if (baseFileReader == null && logRecordScanner == null) { + return Collections.emptyMap(); + } + + boolean fullKeys = true; + Map> logRecords = readLogRecords(logRecordScanner, keys, fullKeys, timings); + return readFromBaseAndMergeWithLogRecords(baseFileReader, keys, fullKeys, logRecords, timings, partitionName); + } catch (IOException ioe) { + throw new HoodieIOException("Error merging records from metadata table for " + keys.size() + " key : ", ioe); + } + } + + private Map> readLogRecords(HoodieMetadataLogRecordReader logRecordReader, + List keys, + boolean fullKey, + List timings) { HoodieTimer timer = HoodieTimer.start(); if (logRecordReader == null) { @@ -269,45 +293,25 @@ private Map>> readLogRecords( return Collections.emptyMap(); } - Map>> logRecords = new HashMap<>(keys.size()); - - // First, fetch the keys being looked up - List>>> logRecordsList = - (fullKey ? logRecordReader.getRecordsByKeys(keys) : logRecordReader.getRecordsByKeyPrefixes(keys)) - .stream() - .map(record -> Pair.of(record.getRecordKey(), Option.of(record))) - .collect(Collectors.toList()); - - for (Pair>> entry : logRecordsList) { - logRecords.put(entry.getKey(), entry.getValue()); + try { + return fullKey ? logRecordReader.getRecordsByKeys(keys) : logRecordReader.getRecordsByKeyPrefixes(keys); + } finally { + timings.add(timer.endTimer()); } - - timings.add(timer.endTimer()); - return logRecords; } - private List>>> readFromBaseAndMergeWithLogRecords(HoodieSeekingFileReader reader, - List keys, - boolean fullKeys, - Map>> logRecords, - List timings, - String partitionName) throws IOException { + private Map> readFromBaseAndMergeWithLogRecords(HoodieSeekingFileReader reader, + List keys, + boolean fullKeys, + Map> logRecords, + List timings, + String partitionName) throws IOException { HoodieTimer timer = HoodieTimer.start(); if (reader == null) { // No base file at all timings.add(timer.endTimer()); - if (fullKeys) { - // In case full-keys (not key-prefixes) were provided, it's expected that the list of - // records will contain an (optional) entry for each corresponding key - return keys.stream() - .map(key -> Pair.of(key, logRecords.getOrDefault(key, Option.empty()))) - .collect(Collectors.toList()); - } else { - return logRecords.entrySet().stream() - .map(entry -> Pair.of(entry.getKey(), entry.getValue())) - .collect(Collectors.toList()); - } + return logRecords; } HoodieTimer readTimer = HoodieTimer.start(); @@ -318,31 +322,16 @@ private List>>> readFrom metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); // Iterate over all provided log-records, merging them into existing records - for (Option> logRecordOpt : logRecords.values()) { - if (logRecordOpt.isPresent()) { - HoodieRecord logRecord = logRecordOpt.get(); + logRecords.values().forEach(logRecord -> records.merge( logRecord.getRecordKey(), logRecord, (oldRecord, newRecord) -> new HoodieAvroRecord<>(oldRecord.getKey(), newRecord.getData().preCombine(oldRecord.getData())) - ); - } - } + )); timings.add(timer.endTimer()); - - if (fullKeys) { - // In case full-keys (not key-prefixes) were provided, it's expected that the list of - // records will contain an (optional) entry for each corresponding key - return keys.stream() - .map(key -> Pair.of(key, Option.ofNullable(records.get(key)))) - .collect(Collectors.toList()); - } else { - return records.values().stream() - .map(record -> Pair.of(record.getRecordKey(), Option.of(record))) - .collect(Collectors.toList()); - } + return records; } @SuppressWarnings("unchecked") @@ -375,37 +364,12 @@ private HoodieRecord composeRecord(GenericRecord avroReco false, Option.of(partitionName), Option.empty()); } - /** - * Get the latest file slices for the interested keys in a given partition. - * - * @param partitionName - Partition to get the file slices from - * @param keys - Interested keys - * @return FileSlices for the keys - */ - private Map, List> getPartitionFileSliceToKeysMapping(final String partitionName, final List keys) { - // Metadata is in sync till the latest completed instant on the dataset - List latestFileSlices = - HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices( - metadataMetaClient, metadataFileSystemView, partitionName); - - Map, List> partitionFileSliceToKeysMap = new HashMap<>(); - for (String key : keys) { - if (!isNullOrEmpty(latestFileSlices)) { - final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, - latestFileSlices.size())); - final Pair partitionNameFileSlicePair = Pair.of(partitionName, slice); - partitionFileSliceToKeysMap.computeIfAbsent(partitionNameFileSlicePair, k -> new ArrayList<>()).add(key); - } - } - return partitionFileSliceToKeysMap; - } - /** * Create a file reader and the record scanner for a given partition and file slice * if readers are not already available. * - * @param partitionName - Partition name - * @param slice - The file slice to open readers for + * @param partitionName - Partition name + * @param slice - The file slice to open readers for * @return File reader and the record scanner pair for the requested file slice */ private Pair, HoodieMetadataLogRecordReader> getOrCreateReaders(String partitionName, FileSlice slice) { @@ -433,7 +397,7 @@ private Pair, HoodieMetadataLogRecordReader> openRead final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, - +baseFileOpenMs + logScannerOpenMs)); + baseFileOpenMs + logScannerOpenMs)); return Pair.of(baseFileReader, logRecordScanner); } catch (IOException e) { throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e); @@ -442,7 +406,7 @@ private Pair, HoodieMetadataLogRecordReader> openRead private Pair, Long> getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException { HoodieSeekingFileReader baseFileReader; - Long baseFileOpenMs; + long baseFileOpenMs; // If the base file is present then create a reader Option basefile = slice.getBaseFile(); if (basefile.isPresent()) { @@ -488,9 +452,9 @@ public Pair getLogRecordScanner(List, HoodieMetadataLogRecor } public boolean enabled() { - return isMetadataTableEnabled; + return isMetadataTableInitialized; } public HoodieTableMetaClient getMetadataMetaClient() { @@ -606,5 +571,14 @@ public void reset() { // the cached reader has max instant time restriction, they should be cleared // because the metadata timeline may have changed. closePartitionReaders(); + partitionFileSliceMap.clear(); + } + + @Override + public int getNumFileGroupsForPartition(MetadataPartitionType partition) { + partitionFileSliceMap.computeIfAbsent(partition.getPartitionPath(), + k -> HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, + metadataFileSystemView, partition.getPartitionPath())); + return partitionFileSliceMap.get(partition.getPartitionPath()).size(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java index a5e25d6130c42..da720de59d55a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; @@ -54,8 +53,7 @@ public HoodieMetadataFileSystemView(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, HoodieMetadataConfig metadataConfig) { - this(metaClient, visibleActiveTimeline, HoodieTableMetadata.create(engineContext, metadataConfig, - metaClient.getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue(), true)); + this(metaClient, visibleActiveTimeline, HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(), true)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java index 18c389a1d1b27..6b6f5f95d84df 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java @@ -20,6 +20,7 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; + import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.InstantRange; @@ -27,6 +28,7 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import javax.annotation.concurrent.ThreadSafe; + import java.io.Closeable; import java.io.IOException; import java.util.Collections; @@ -72,23 +74,21 @@ public List> getRecords() { } @SuppressWarnings("unchecked") - public List> getRecordsByKeyPrefixes(List keyPrefixes) { + public Map> getRecordsByKeyPrefixes(List keyPrefixes) { if (keyPrefixes.isEmpty()) { - return Collections.emptyList(); + return Collections.emptyMap(); } // NOTE: Locking is necessary since we're accessing [[HoodieMetadataLogRecordReader]] // materialized state, to make sure there's no concurrent access synchronized (this) { logRecordScanner.scanByKeyPrefixes(keyPrefixes); - Map allRecords = logRecordScanner.getRecords(); - Predicate p = createPrefixMatchingPredicate(keyPrefixes); - return allRecords.entrySet() + return logRecordScanner.getRecords().entrySet() .stream() .filter(r -> r != null && p.test(r.getKey())) .map(r -> (HoodieRecord) r.getValue()) - .collect(Collectors.toList()); + .collect(Collectors.toMap(HoodieRecord::getRecordKey, r -> r)); } } @@ -97,9 +97,9 @@ public List> getRecordsByKeyPrefixes(List> getRecordsByKeys(List keys) { + public Map> getRecordsByKeys(List keys) { if (keys.isEmpty()) { - return Collections.emptyList(); + return Collections.emptyMap(); } // NOTE: Locking is necessary since we're accessing [[HoodieMetadataLogRecordReader]] @@ -110,7 +110,7 @@ public List> getRecordsByKeys(List k return keys.stream() .map(key -> (HoodieRecord) allRecords.get(key)) .filter(Objects::nonNull) - .collect(Collectors.toList()); + .collect(Collectors.toMap(HoodieRecord::getRecordKey, r -> r)); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java index a84d2103f5eb0..31591010bf94e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -143,6 +143,10 @@ protected void incrementMetric(String action, long value) { metricsRegistry.add(action, value); } + protected void setMetric(String action, long value) { + metricsRegistry.set(action, value); + } + public Registry registry() { return metricsRegistry; } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 336d8ad47b067..eb8794a64c7fa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -28,16 +28,20 @@ import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataFileInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.avro.model.HoodieRecordIndexInfo; import org.apache.hudi.avro.model.IntWrapper; import org.apache.hudi.avro.model.LongWrapper; import org.apache.hudi.avro.model.StringWrapper; import org.apache.hudi.avro.model.TimestampMicrosWrapper; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordGlobalLocation; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.FileIndexID; @@ -73,6 +77,7 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -113,28 +118,38 @@ */ public class HoodieMetadataPayload implements HoodieRecordPayload { - // Type of the record. This can be an enum in the schema but Avro1.8 - // has a bug - https://issues.apache.org/jira/browse/AVRO-1810 + /** + * Type of the record. This can be an enum in the schema but Avro1.8 + * has a bug - https://issues.apache.org/jira/browse/AVRO-1810 + */ protected static final int METADATA_TYPE_PARTITION_LIST = 1; protected static final int METADATA_TYPE_FILE_LIST = 2; protected static final int METADATA_TYPE_COLUMN_STATS = 3; protected static final int METADATA_TYPE_BLOOM_FILTER = 4; + private static final int METADATA_TYPE_RECORD_INDEX = 5; - // HoodieMetadata schema field ids + /** + * HoodieMetadata schema field ids + */ public static final String KEY_FIELD_NAME = HoodieAvroHFileReader.KEY_FIELD_NAME; public static final String SCHEMA_FIELD_NAME_TYPE = "type"; public static final String SCHEMA_FIELD_NAME_METADATA = "filesystemMetadata"; public static final String SCHEMA_FIELD_ID_COLUMN_STATS = "ColumnStatsMetadata"; public static final String SCHEMA_FIELD_ID_BLOOM_FILTER = "BloomFilterMetadata"; + public static final String SCHEMA_FIELD_ID_RECORD_INDEX = "recordIndexMetadata"; - // HoodieMetadata bloom filter payload field ids + /** + * HoodieMetadata bloom filter payload field ids + */ private static final String FIELD_IS_DELETED = "isDeleted"; private static final String BLOOM_FILTER_FIELD_TYPE = "type"; private static final String BLOOM_FILTER_FIELD_TIMESTAMP = "timestamp"; private static final String BLOOM_FILTER_FIELD_BLOOM_FILTER = "bloomFilter"; private static final String BLOOM_FILTER_FIELD_IS_DELETED = FIELD_IS_DELETED; - // HoodieMetadata column stats payload field ids + /** + * HoodieMetadata column stats payload field ids + */ public static final String COLUMN_STATS_FIELD_MIN_VALUE = "minValue"; public static final String COLUMN_STATS_FIELD_MAX_VALUE = "maxValue"; public static final String COLUMN_STATS_FIELD_NULL_COUNT = "nullCount"; @@ -147,19 +162,35 @@ public class HoodieMetadataPayload implements HoodieRecordPayload + * In Avro 1.10 generated builders rely on {@code SpecificData.getForSchema} invocation that in turn + * does use reflection to load the code-gen'd class corresponding to the Avro record model. This has + * serious adverse effects in terms of performance when gets executed on the hot-path (both, in terms + * of runtime and efficiency). + *

    + * To work this around instead of using default code-gen'd builder invoking {@code SpecificData.getForSchema}, + * we instead rely on overloaded ctor accepting another instance of the builder: {@code Builder(Builder)}, + * which bypasses such invocation. Following corresponding builder's stubs are statically initialized + * to be used exactly for that purpose. + *

    + * You can find more details in HUDI-3834. + */ private static final Lazy METADATA_COLUMN_STATS_BUILDER_STUB = Lazy.lazily(HoodieMetadataColumnStats::newBuilder); private static final Lazy STRING_WRAPPER_BUILDER_STUB = Lazy.lazily(StringWrapper::newBuilder); private static final Lazy BYTES_WRAPPER_BUILDER_STUB = Lazy.lazily(BytesWrapper::newBuilder); @@ -177,6 +208,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload filesystemMetadata = null; private HoodieMetadataBloomFilter bloomFilterMetadata = null; private HoodieMetadataColumnStats columnStatMetadata = null; + private HoodieRecordIndexInfo recordIndexMetadata; public HoodieMetadataPayload(GenericRecord record, Comparable orderingVal) { this(Option.of(record)); @@ -194,16 +226,16 @@ public HoodieMetadataPayload(Option recordOpt) { key = record.get(KEY_FIELD_NAME).toString(); type = (int) record.get(SCHEMA_FIELD_NAME_TYPE); - Map metadata = getNestedFieldValue(record, SCHEMA_FIELD_NAME_METADATA); - if (metadata != null) { - filesystemMetadata = metadata; - filesystemMetadata.keySet().forEach(k -> { - GenericRecord v = filesystemMetadata.get(k); - filesystemMetadata.put(k, new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); - }); - } - - if (type == METADATA_TYPE_BLOOM_FILTER) { + if (type == METADATA_TYPE_FILE_LIST || type == METADATA_TYPE_PARTITION_LIST) { + Map metadata = getNestedFieldValue(record, SCHEMA_FIELD_NAME_METADATA); + if (metadata != null) { + filesystemMetadata = metadata; + filesystemMetadata.keySet().forEach(k -> { + GenericRecord v = filesystemMetadata.get(k); + filesystemMetadata.put(k, new HoodieMetadataFileInfo((Long) v.get("size"), (Boolean) v.get("isDeleted"))); + }); + } + } else if (type == METADATA_TYPE_BLOOM_FILTER) { GenericRecord bloomFilterRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_BLOOM_FILTER); // NOTE: Only legitimate reason for {@code BloomFilterMetadata} to not be present is when // it's not been read from the storage (ie it's not been a part of projected schema). @@ -219,9 +251,7 @@ public HoodieMetadataPayload(Option recordOpt) { (Boolean) bloomFilterRecord.get(BLOOM_FILTER_FIELD_IS_DELETED) ); } - } - - if (type == METADATA_TYPE_COLUMN_STATS) { + } else if (type == METADATA_TYPE_COLUMN_STATS) { GenericRecord columnStatsRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_COLUMN_STATS); // NOTE: Only legitimate reason for {@code ColumnStatsMetadata} to not be present is when // it's not been read from the storage (ie it's not been a part of projected schema). @@ -243,33 +273,46 @@ public HoodieMetadataPayload(Option recordOpt) { .setTotalSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_SIZE)) .setTotalUncompressedSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_UNCOMPRESSED_SIZE)) .setIsDeleted((Boolean) columnStatsRecord.get(COLUMN_STATS_FIELD_IS_DELETED)) - .build(); + .build(); } + } else if (type == METADATA_TYPE_RECORD_INDEX) { + GenericRecord recordIndexRecord = getNestedFieldValue(record, SCHEMA_FIELD_ID_RECORD_INDEX); + recordIndexMetadata = new HoodieRecordIndexInfo(recordIndexRecord.get(RECORD_INDEX_FIELD_PARTITION).toString(), + Long.parseLong(recordIndexRecord.get(RECORD_INDEX_FIELD_FILEID_HIGH_BITS).toString()), + Long.parseLong(recordIndexRecord.get(RECORD_INDEX_FIELD_FILEID_LOW_BITS).toString()), + Integer.parseInt(recordIndexRecord.get(RECORD_INDEX_FIELD_FILE_INDEX).toString()), + Long.parseLong(recordIndexRecord.get(RECORD_INDEX_FIELD_INSTANT_TIME).toString())); } } } private HoodieMetadataPayload(String key, int type, Map filesystemMetadata) { - this(key, type, filesystemMetadata, null, null); + this(key, type, filesystemMetadata, null, null, null); } private HoodieMetadataPayload(String key, HoodieMetadataBloomFilter metadataBloomFilter) { - this(key, METADATA_TYPE_BLOOM_FILTER, null, metadataBloomFilter, null); + this(key, METADATA_TYPE_BLOOM_FILTER, null, metadataBloomFilter, null, null); } private HoodieMetadataPayload(String key, HoodieMetadataColumnStats columnStats) { - this(key, METADATA_TYPE_COLUMN_STATS, null, null, columnStats); + this(key, METADATA_TYPE_COLUMN_STATS, null, null, columnStats, null); + } + + private HoodieMetadataPayload(String key, HoodieRecordIndexInfo recordIndexMetadata) { + this(key, METADATA_TYPE_RECORD_INDEX, null, null, null, recordIndexMetadata); } protected HoodieMetadataPayload(String key, int type, - Map filesystemMetadata, - HoodieMetadataBloomFilter metadataBloomFilter, - HoodieMetadataColumnStats columnStats) { + Map filesystemMetadata, + HoodieMetadataBloomFilter metadataBloomFilter, + HoodieMetadataColumnStats columnStats, + HoodieRecordIndexInfo recordIndexMetadata) { this.key = key; this.type = type; this.filesystemMetadata = filesystemMetadata; this.bloomFilterMetadata = metadataBloomFilter; this.columnStatMetadata = columnStats; + this.recordIndexMetadata = recordIndexMetadata; } /** @@ -296,23 +339,6 @@ public static HoodieRecord createPartitionListRecord(List return new HoodieAvroRecord<>(key, payload); } - /** - * Create and return a {@code HoodieMetadataPayload} to save list of partitions. - * - * @param partitionsAdded The list of added partitions - * @param partitionsDeleted The list of deleted partitions - */ - public static HoodieRecord createPartitionListRecord(List partitionsAdded, List partitionsDeleted) { - Map fileInfo = new HashMap<>(); - partitionsAdded.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false))); - partitionsDeleted.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, true))); - - HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath()); - HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_PARTITION_LIST, - fileInfo); - return new HoodieAvroRecord<>(key, payload); - } - /** * Create and return a {@code HoodieMetadataPayload} to save list of files within a partition. * @@ -379,6 +405,8 @@ public static HoodieRecord createBloomFilterMetadataRecor public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { checkArgument(previousRecord.type == type, "Cannot combine " + previousRecord.type + " with " + type); + checkArgument(previousRecord.key.equals(key), + "Cannot combine " + previousRecord.key + " with " + key + " as the keys differ"); switch (type) { case METADATA_TYPE_PARTITION_LIST: @@ -390,6 +418,18 @@ public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { return new HoodieMetadataPayload(key, combineBloomFilterMetadata); case METADATA_TYPE_COLUMN_STATS: return new HoodieMetadataPayload(key, combineColumnStatsMetadata(previousRecord)); + case METADATA_TYPE_RECORD_INDEX: + // TODO: does not work with updates + if (previousRecord.recordIndexMetadata.getInstantTime() != recordIndexMetadata.getInstantTime()) { + throw new HoodieMetadataException(String.format("InstantTime for %s should not change from %s to %s", previousRecord.key, + previousRecord, this)); + } + // TODO: This does not work with clustering + if (!previousRecord.getRecordGlobalLocation().equals(getRecordGlobalLocation())) { + throw new HoodieMetadataException(String.format("Location for %s should not change from %s to %s", previousRecord.key, + previousRecord, this)); + } + return this; default: throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type); } @@ -429,7 +469,7 @@ public Option getInsertValue(Schema schemaIgnored, Properties pro } HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata, bloomFilterMetadata, - columnStatMetadata); + columnStatMetadata, recordIndexMetadata); return Option.of(record); } @@ -666,6 +706,76 @@ private static HoodieMetadataColumnStats mergeColumnStatsRecords(HoodieMetadataC .build(); } + /** + * Create and return a {@code HoodieMetadataPayload} to insert or update an entry for the record index. + *

    + * Each entry maps the key of a single record in HUDI to its location. + * + * @param recordKey Key of the record + * @param partition Name of the partition which contains the record + * @param fileId fileId which contains the record + * @param instantTime instantTime when the record was added + */ + public static HoodieRecord createRecordIndexUpdate(String recordKey, String partition, + String fileId, String instantTime) { + HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_INDEX.getPartitionPath()); + // Data file names have a -D suffix to denote the index (D = integer) of the file written + // In older HUID versions the file index was missing + final UUID uuid; + final int fileIndex; + try { + if (fileId.length() == 36) { + uuid = UUID.fromString(fileId); + fileIndex = RECORD_INDEX_MISSING_FILEINDEX_FALLBACK; + } else { + final int index = fileId.lastIndexOf("-"); + uuid = UUID.fromString(fileId.substring(0, index)); + fileIndex = Integer.parseInt(fileId.substring(index + 1)); + } + } catch (Exception e) { + throw new HoodieMetadataException(String.format("Invalid UUID or index: fileID=%s, partition=%s, instantTIme=%s", + fileId, partition, instantTime), e); + } + + try { + long instantTimeMillis = HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime(); + HoodieMetadataPayload payload = new HoodieMetadataPayload(recordKey, + new HoodieRecordIndexInfo( + partition, + uuid.getMostSignificantBits(), + uuid.getLeastSignificantBits(), + fileIndex, + instantTimeMillis)); + return new HoodieAvroRecord<>(key, payload); + } catch (Exception e) { + throw new HoodieMetadataException("Failed to create metadata payload for record index.", e); + } + } + + /** + * Create and return a {@code HoodieMetadataPayload} to delete a record in the Metadata Table's record index. + * + * @param recordKey Key of the record to be deleted + */ + public static HoodieRecord createRecordIndexDelete(String recordKey) { + HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_INDEX.getPartitionPath()); + return new HoodieAvroRecord<>(key, new EmptyHoodieRecordPayload()); + } + + /** + * If this is a record-level index entry, returns the file to which this is mapped. + */ + public HoodieRecordGlobalLocation getRecordGlobalLocation() { + final UUID uuid = new UUID(recordIndexMetadata.getFileIdHighBits(), recordIndexMetadata.getFileIdLowBits()); + final String partition = recordIndexMetadata.getPartition(); + String fileId = uuid.toString(); + if (recordIndexMetadata.getFileIndex() != RECORD_INDEX_MISSING_FILEINDEX_FALLBACK) { + fileId += "-" + recordIndexMetadata.getFileIndex(); + } + final java.util.Date instantDate = new java.util.Date(recordIndexMetadata.getInstantTime()); + return new HoodieRecordGlobalLocation(partition, HoodieActiveTimeline.formatDate(instantDate), fileId); + } + @Override public boolean equals(Object other) { if (other == this) { @@ -680,7 +790,8 @@ public boolean equals(Object other) { && Objects.equals(this.key, otherMetadataPayload.key) && Objects.equals(this.filesystemMetadata, otherMetadataPayload.filesystemMetadata) && Objects.equals(this.bloomFilterMetadata, otherMetadataPayload.bloomFilterMetadata) - && Objects.equals(this.columnStatMetadata, otherMetadataPayload.columnStatMetadata); + && Objects.equals(this.columnStatMetadata, otherMetadataPayload.columnStatMetadata) + && Objects.equals(this.recordIndexMetadata, otherMetadataPayload.recordIndexMetadata); } @Override @@ -693,21 +804,36 @@ public String toString() { final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); sb.append(KEY_FIELD_NAME + "=").append(key).append(", "); sb.append(SCHEMA_FIELD_NAME_TYPE + "=").append(type).append(", "); - sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); - sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); - if (type == METADATA_TYPE_BLOOM_FILTER) { - checkState(getBloomFilterMetadata().isPresent()); - sb.append("BloomFilter: {"); - sb.append("bloom size: ").append(getBloomFilterMetadata().get().getBloomFilter().array().length).append(", "); - sb.append("timestamp: ").append(getBloomFilterMetadata().get().getTimestamp()).append(", "); - sb.append("deleted: ").append(getBloomFilterMetadata().get().getIsDeleted()); - sb.append("}"); - } - if (type == METADATA_TYPE_COLUMN_STATS) { - checkState(getColumnStatMetadata().isPresent()); - sb.append("ColStats: {"); - sb.append(getColumnStatMetadata().get()); - sb.append("}"); + + switch (type) { + case METADATA_TYPE_PARTITION_LIST: + case METADATA_TYPE_FILE_LIST: + sb.append("Files: {"); + sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); + sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); + sb.append("}"); + break; + case METADATA_TYPE_BLOOM_FILTER: + checkState(getBloomFilterMetadata().isPresent()); + sb.append("BloomFilter: {"); + sb.append("bloom size: ").append(getBloomFilterMetadata().get().getBloomFilter().array().length).append(", "); + sb.append("timestamp: ").append(getBloomFilterMetadata().get().getTimestamp()).append(", "); + sb.append("deleted: ").append(getBloomFilterMetadata().get().getIsDeleted()); + sb.append("}"); + break; + case METADATA_TYPE_COLUMN_STATS: + checkState(getColumnStatMetadata().isPresent()); + sb.append("ColStats: {"); + sb.append(getColumnStatMetadata().get()); + sb.append("}"); + break; + case METADATA_TYPE_RECORD_INDEX: + sb.append("RecordIndex: {"); + sb.append("location=").append(getRecordGlobalLocation()); + sb.append("}"); + break; + default: + break; } sb.append('}'); return sb.toString(); 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 e2bc9def0f3e6..ac17eed001b25 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 @@ -25,6 +25,7 @@ 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; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -110,18 +111,20 @@ static boolean isMetadataTable(String basePath) { return basePath.endsWith(HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); } - static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, - String spillableMapPath) { - return create(engineContext, metadataConfig, datasetBasePath, spillableMapPath, false); + static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath) { + return create(engineContext, metadataConfig, datasetBasePath, false); } - static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, - String spillableMapPath, boolean reuse) { + static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, boolean reuse) { if (metadataConfig.enabled()) { - return createHoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, spillableMapPath, reuse); - } else { - return createFSBackedTableMetadata(engineContext, metadataConfig, datasetBasePath); + HoodieBackedTableMetadata metadata = createHoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, reuse); + // If the MDT is not initialized then we fallback to FSBackedTableMetadata + if (metadata.isMetadataTableInitialized()) { + return metadata; + } } + + return createFSBackedTableMetadata(engineContext, metadataConfig, datasetBasePath); } static FileSystemBackedTableMetadata createFSBackedTableMetadata(HoodieEngineContext engineContext, @@ -134,9 +137,8 @@ static FileSystemBackedTableMetadata createFSBackedTableMetadata(HoodieEngineCon static HoodieBackedTableMetadata createHoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, - String spillableMapPath, boolean reuse) { - return new HoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, spillableMapPath, reuse); + return new HoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, reuse); } /** @@ -198,11 +200,17 @@ Map, BloomFilter> getBloomFilters(final List, HoodieMetadataColumnStats> getColumnStats(final List> partitionNameFileNameList, final String columnName) throws HoodieMetadataException; + /** + * Returns the location of record keys which are found in the record index. + * Records that are not found are ignored and wont be part of map object that is returned. + */ + Map readRecordIndex(List recordKeys); + /** * Fetch records by key prefixes. Key prefix passed is expected to match the same prefix as stored in Metadata table partitions. For eg, in case of col stats partition, * actual keys in metadata partition is encoded values of column name, partition name and file name. So, key prefixes passed to this method is expected to be encoded already. * - * @param keyPrefixes list of key prefixes for which interested records are looked up for. + * @param keyPrefixes list of key prefixes for which interested records are looked up for. * @param partitionName partition name in metadata table where the records are looked up for. * @return {@link HoodieData} of {@link HoodieRecord}s with records matching the passed in key prefixes. */ @@ -224,4 +232,9 @@ HoodieData> getRecordsByKeyPrefixes(List convertMetadataToFilesPartitionRecords(HoodieCo records.add(HoodieMetadataPayload.createPartitionListRecord(partitionsAdded)); // Update files listing records for each individual partition + HoodieAccumulator newFileCount = HoodieAtomicLongAccumulator.create(); List> updatedPartitionFilesRecords = commitMetadata.getPartitionToWriteStats().entrySet() .stream() @@ -357,6 +364,7 @@ public static List convertMetadataToFilesPartitionRecords(HoodieCo }, CollectionUtils::combine); + newFileCount.add(updatedFilesToSizesMapping.size()); return HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.of(updatedFilesToSizesMapping), Option.empty()); }) @@ -364,8 +372,8 @@ public static List convertMetadataToFilesPartitionRecords(HoodieCo records.addAll(updatedPartitionFilesRecords); - LOG.info("Updating at " + instantTime + " from Commit/" + commitMetadata.getOperationType() - + ". #partitions_updated=" + records.size()); + LOG.info(String.format("Updating at %s from Commit/%s. #partitions_updated=%d, #files_added=%d", instantTime, commitMetadata.getOperationType(), + records.size(), newFileCount.value())); return records; } @@ -1545,11 +1553,7 @@ public static String deleteMetadataTablePartition(HoodieTableMetaClient dataMeta * @return The fileID */ public static String getFileIDForFileGroup(MetadataPartitionType partitionType, int index) { - if (partitionType == MetadataPartitionType.FILES) { - return String.format("%s%04d-%d", partitionType.getFileIdPrefix(), index, 0); - } else { - return String.format("%s%04d", partitionType.getFileIdPrefix(), index); - } + return String.format("%s%04d-%d", partitionType.getFileIdPrefix(), index, 0); } /** @@ -1620,4 +1624,72 @@ public static String createIndexInitTimestamp(String timestamp, int offset) { public static String createLogCompactionTimestamp(String timestamp) { return timestamp + LOG_COMPACTION_TIMESTAMP_SUFFIX; } + + /** + * Estimates the file group count to use for a MDT partition. + * + * @param partitionType Type of the partition for which the file group count is to be estimated. + * @param recordCount The number of records expected to be written. + * @param averageRecordSize Average size of each record to be writen. + * @param minFileGroupCount Minimum number of file groups to use. + * @param maxFileGroupCount Maximum number of file groups to use. + * @param growthFactor By what factor are the records (recordCount) expected to grow? + * @param maxFileGroupSizeBytes Maximum size of the file group. + * @return The estimated number of file groups. + */ + public static int estimateFileGroupCount(MetadataPartitionType partitionType, long recordCount, int averageRecordSize, int minFileGroupCount, + int maxFileGroupCount, float growthFactor, int maxFileGroupSizeBytes) { + int fileGroupCount; + + // If a fixed number of file groups are desired + if ((minFileGroupCount == maxFileGroupCount) && (minFileGroupCount != 0)) { + fileGroupCount = minFileGroupCount; + } else { + // Number of records to estimate for + final long expectedNumRecords = (long) Math.ceil((float) recordCount * growthFactor); + // Maximum records that should be written to each file group so that it does not go over the size limit required + final long maxRecordsPerFileGroup = maxFileGroupSizeBytes / Math.max(averageRecordSize, 1L); + final long estimatedFileGroupCount = expectedNumRecords / maxRecordsPerFileGroup; + + if (estimatedFileGroupCount >= maxFileGroupCount) { + fileGroupCount = maxFileGroupCount; + } else if (estimatedFileGroupCount <= minFileGroupCount) { + fileGroupCount = minFileGroupCount; + } else { + fileGroupCount = Math.max(1, (int) estimatedFileGroupCount); + } + } + + LOG.info(String.format("Estimated file group count for MDT partition %s is %d " + + "[recordCount=%d, avgRecordSize=%d, minFileGroupCount=%d, maxFileGroupCount=%d, growthFactor=%f, " + + "maxFileGroupSizeBytes=%d]", partitionType, fileGroupCount, recordCount, averageRecordSize, minFileGroupCount, + maxFileGroupCount, growthFactor, maxFileGroupSizeBytes)); + return fileGroupCount; + } + + /** + * Returns true if any enabled metadata partition in the given hoodie table requires WriteStatus to track the written records. + * + * @param config MDT config + * @param metaClient {@code HoodieTableMetaClient} of the data table + * @return true if WriteStatus should track the written records else false. + */ + public static boolean getMetadataPartitionsNeedingWriteStatusTracking(HoodieMetadataConfig config, HoodieTableMetaClient metaClient) { + // Does any enabled partition need to track the written records + if (MetadataPartitionType.getMetadataPartitionsNeedingWriteStatusTracking().stream().anyMatch(p -> metaClient.getTableConfig().isMetadataPartitionAvailable(p))) { + return true; + } + + // Does any inflight partitions need to track the written records + Set metadataPartitionsInflight = metaClient.getTableConfig().getMetadataPartitionsInflight(); + if (MetadataPartitionType.getMetadataPartitionsNeedingWriteStatusTracking().stream().anyMatch(p -> metadataPartitionsInflight.contains(p.getPartitionPath()))) { + return true; + } + + // Does any enabled partition being enabled need to track the written records + if (config.enableRecordIndex()) { + return true; + } + return false; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java index 680f1c6ac81f7..3f6a5adf6f029 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -19,6 +19,7 @@ package org.apache.hudi.metadata; import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -27,7 +28,8 @@ public enum MetadataPartitionType { FILES(HoodieTableMetadataUtil.PARTITION_NAME_FILES, "files-"), COLUMN_STATS(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS, "col-stats-"), - BLOOM_FILTERS(HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS, "bloom-filters-"); + BLOOM_FILTERS(HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS, "bloom-filters-"), + RECORD_INDEX(HoodieTableMetadataUtil.PARTITION_NAME_RECORD_INDEX, "record-index-"); // Partition path in metadata table. private final String partitionPath; @@ -55,6 +57,15 @@ public static List allPaths() { ); } + /** + * Returns the list of metadata table partitions which require WriteStatus to track written records. + *

    + * These partitions need the list of written records so that they can update their metadata. + */ + public static List getMetadataPartitionsNeedingWriteStatusTracking() { + return Collections.singletonList(MetadataPartitionType.RECORD_INDEX); + } + @Override public String toString() { return "Metadata partition {" diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 38f770e0bb5ad..469b5c8bf393a 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -317,17 +317,17 @@ public static void createPartitionMetaFile(String basePath, String partitionPath } } - public static void createBaseFile(String basePath, String partitionPath, String instantTime, String fileId) + public static String createBaseFile(String basePath, String partitionPath, String instantTime, String fileId) throws Exception { - createBaseFile(basePath, partitionPath, instantTime, fileId, 1); + return createBaseFile(basePath, partitionPath, instantTime, fileId, 1); } - public static void createBaseFile(String basePath, String partitionPath, String instantTime, String fileId, long length) + public static String createBaseFile(String basePath, String partitionPath, String instantTime, String fileId, long length) throws Exception { - createBaseFile(basePath, partitionPath, instantTime, fileId, length, Instant.now().toEpochMilli()); + return createBaseFile(basePath, partitionPath, instantTime, fileId, length, Instant.now().toEpochMilli()); } - public static void createBaseFile(String basePath, String partitionPath, String instantTime, String fileId, long length, long lastModificationTimeMilli) + public static String createBaseFile(String basePath, String partitionPath, String instantTime, String fileId, long length, long lastModificationTimeMilli) throws Exception { Path parentPath = Paths.get(basePath, partitionPath); Files.createDirectories(parentPath); @@ -339,6 +339,7 @@ public static void createBaseFile(String basePath, String partitionPath, String raf.setLength(length); } Files.setLastModifiedTime(baseFilePath, FileTime.fromMillis(lastModificationTimeMilli)); + return baseFilePath.toString(); } public static Path getBaseFilePath(String basePath, String partitionPath, String instantTime, String fileId) { @@ -346,12 +347,12 @@ public static Path getBaseFilePath(String basePath, String partitionPath, String return parentPath.resolve(baseFileName(instantTime, fileId)); } - public static void createLogFile(String basePath, String partitionPath, String instantTime, String fileId, int version) + public static String createLogFile(String basePath, String partitionPath, String instantTime, String fileId, int version) throws Exception { - createLogFile(basePath, partitionPath, instantTime, fileId, version, 0); + return createLogFile(basePath, partitionPath, instantTime, fileId, version, 0); } - public static void createLogFile(String basePath, String partitionPath, String instantTime, String fileId, int version, int length) + public static String createLogFile(String basePath, String partitionPath, String instantTime, String fileId, int version, int length) throws Exception { Path parentPath = Paths.get(basePath, partitionPath); Files.createDirectories(parentPath); @@ -362,6 +363,7 @@ public static void createLogFile(String basePath, String partitionPath, String i RandomAccessFile raf = new RandomAccessFile(logFilePath.toFile(), "rw"); raf.setLength(length); raf.close(); + return logFilePath.toString(); } public static String createMarkerFile(String basePath, String partitionPath, String instantTime, String fileId, IOType ioType) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 21e8c8081f779..63c4e15ce2bec 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -60,6 +60,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.metadata.HoodieMetadataPayload; import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hadoop.fs.FileStatus; @@ -86,10 +87,12 @@ import java.util.stream.Stream; import static java.time.temporal.ChronoUnit.SECONDS; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_POPULATE_META_FIELDS; import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; import static org.apache.hudi.common.model.WriteOperationType.CLUSTER; import static org.apache.hudi.common.model.WriteOperationType.COMPACT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; +import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; @@ -103,8 +106,8 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightSavepoint; -import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createLogFileMarker; +import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCleanFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCommit; @@ -122,6 +125,7 @@ import static org.apache.hudi.common.util.CommitUtils.buildMetadata; import static org.apache.hudi.common.util.CommitUtils.getCommitActionType; import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; +import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; /** * Test Hoodie Table for testing only. @@ -140,6 +144,7 @@ public class HoodieTestTable { protected final FileSystem fs; protected HoodieTableMetaClient metaClient; protected String currentInstantTime; + private boolean isNonPartitioned = false; protected HoodieTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient) { ValidationUtils.checkArgument(Objects.equals(basePath, metaClient.getBasePath())); @@ -155,9 +160,28 @@ public static HoodieTestTable of(HoodieTableMetaClient metaClient) { return new HoodieTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient); } - public void updateFilesPartitionInTableConfig() { + public void updateFilesPartitionInTableConfig() throws IOException { metaClient.getTableConfig().setMetadataPartitionState(metaClient, MetadataPartitionType.FILES, true); this.metaClient = HoodieTableMetaClient.reload(metaClient); + + HoodieTableMetaClient.withPropertyBuilder() + .setTableType(HoodieTableType.MERGE_ON_READ) + .setTableName("test_table" + METADATA_TABLE_NAME_SUFFIX) + .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) + .setPayloadClassName(HoodieMetadataPayload.class.getName()) + .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) + .setRecordKeyFields(HoodieMetadataPayload.KEY_FIELD_NAME) + .setPopulateMetaFields(DEFAULT_METADATA_POPULATE_META_FIELDS) + .setKeyGeneratorClassProp("org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator") + .initTable(metaClient.getHadoopConf(), metaClient.getMetaPath() + "/metadata"); + } + + public void setNonPartitioned() { + this.isNonPartitioned = true; + } + + public boolean isNonPartitioned() { + return this.isNonPartitioned; } public static String makeNewCommitTime(int sequence, String instantFormat) { @@ -593,18 +617,20 @@ public Map getFileIdsWithBaseFilesInPartitions(String... partiti return partitionFileIdMap; } - public HoodieTestTable withBaseFilesInPartitions(Map partitionAndFileId) throws Exception { + public Pair> withBaseFilesInPartitions(Map partitionAndFileId) throws Exception { + List files = new ArrayList<>(); for (Map.Entry pair : partitionAndFileId.entrySet()) { - withBaseFilesInPartition(pair.getKey(), pair.getValue()); + files.addAll(withBaseFilesInPartition(pair.getKey(), pair.getValue()).getValue()); } - return this; + return Pair.of(this, files); } - public HoodieTestTable withBaseFilesInPartition(String partition, String... fileIds) throws Exception { + public Pair> withBaseFilesInPartition(String partition, String... fileIds) throws Exception { + List files = new ArrayList<>(); for (String f : fileIds) { - FileCreateUtils.createBaseFile(basePath, partition, currentInstantTime, f); + files.add(FileCreateUtils.createBaseFile(basePath, partition, currentInstantTime, f)); } - return this; + return Pair.of(this, files); } public HoodieTestTable withBaseFilesInPartition(String partition, int... lengths) throws Exception { @@ -628,15 +654,16 @@ public String getFileIdWithLogFile(String partitionPath) throws Exception { return fileId; } - public HoodieTestTable withLogFile(String partitionPath, String fileId) throws Exception { + public Pair> withLogFile(String partitionPath, String fileId) throws Exception { return withLogFile(partitionPath, fileId, 0); } - public HoodieTestTable withLogFile(String partitionPath, String fileId, int... versions) throws Exception { + public Pair> withLogFile(String partitionPath, String fileId, int... versions) throws Exception { + List logFiles = new ArrayList<>(); for (int version : versions) { - FileCreateUtils.createLogFile(basePath, partitionPath, currentInstantTime, fileId, version); + logFiles.add(FileCreateUtils.createLogFile(basePath, partitionPath, currentInstantTime, fileId, version)); } - return this; + return Pair.of(this, logFiles); } public HoodieTestTable withLogFilesInPartition(String partition, List> fileInfos) throws Exception { diff --git a/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.java b/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.java index 9d7c7b456139b..b195083a20131 100644 --- a/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.java +++ b/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.java @@ -35,9 +35,8 @@ public class HoodieBackedTestDelayedTableMetadata extends HoodieBackedTableMetad public HoodieBackedTestDelayedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, - String spillableMapDirectory, boolean reuse) { - super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory, reuse); + super(engineContext, metadataConfig, datasetBasePath, reuse); } @Override diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/ColumnStatsIndices.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/ColumnStatsIndices.java index 428661844eac4..48d5c9d2fa43f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/ColumnStatsIndices.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/stats/ColumnStatsIndices.java @@ -23,7 +23,6 @@ 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.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.hash.ColumnIndexID; @@ -286,8 +285,7 @@ private static List readColumnStatsIndexByColumns( // - Deserializing fetched records into [[RowData]]s HoodieTableMetadata metadataTable = HoodieTableMetadata.create( HoodieFlinkEngineContext.DEFAULT, - metadataConfig, basePath, - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue()); + metadataConfig, basePath); // TODO encoding should be done internally w/in HoodieBackedTableMetadata List encodedTargetColumnNames = Arrays.stream(targetColumns) diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java index 9be924bd1c015..9be28d02c270e 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java @@ -84,7 +84,7 @@ void beforeEach(Map options) throws IOException { // initialize the metadata table path if (conf.getBoolean(FlinkOptions.METADATA_ENABLED)) { FlinkHoodieBackedTableMetadataWriter.create(table.getHadoopConf(), table.getConfig(), - table.getContext(), Option.empty(), Option.empty()); + table.getContext(), Option.empty()); } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java index 260afd5ced03a..cba8d58b2bf81 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieROTablePathFilter.java @@ -51,12 +51,12 @@ public void testHoodiePaths() throws Exception { final String p1 = "2017/01/01"; final String p2 = "2017/01/02"; testTable.addCommit("001") - .withBaseFilesInPartition(p1, "f1", "f2") - .withBaseFilesInPartition(p2, "f3") + .withBaseFilesInPartition(p1, "f1", "f2").getLeft() + .withBaseFilesInPartition(p2, "f3").getLeft() .addCommit("002") - .withBaseFilesInPartition(p1, "f2") + .withBaseFilesInPartition(p1, "f2").getLeft() .addInflightCommit("003") - .withBaseFilesInPartition(p2, "f3") + .withBaseFilesInPartition(p2, "f3").getLeft() .addRequestedCompaction("004"); assertTrue(pathFilter.accept(testTable.forCommit("002").getBaseFilePath(p1, "f2"))); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index bbfe57bb6ec95..82306490ef707 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -83,7 +83,7 @@ public void onDataWriterCommit(String message) { public void commit(List writeStatList) { try { - writeClient.commitStats(instantTime, writeStatList, Option.of(extraMetadata), + writeClient.commitStats(instantTime, writeClient.getEngineContext().emptyHoodieData(), writeStatList, Option.of(extraMetadata), CommitUtils.getCommitActionType(operationType, metaClient.getTableType())); } catch (Exception ioe) { throw new HoodieException(ioe.getMessage(), ioe); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala index cbf30fb2a7f27..dd76aee2f187b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala @@ -28,7 +28,6 @@ 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.table.view.FileSystemViewStorageConfig import org.apache.hudi.common.util.BinaryUtil.toBytes import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.common.util.collection @@ -59,7 +58,7 @@ class ColumnStatsIndexSupport(spark: SparkSession, @transient private lazy val engineCtx = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)) @transient private lazy val metadataTable: HoodieTableMetadata = - HoodieTableMetadata.create(engineCtx, metadataConfig, metaClient.getBasePathV2.toString, FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue) + HoodieTableMetadata.create(engineCtx, metadataConfig, metaClient.getBasePathV2.toString) @transient private lazy val cachedColumnStatsIndexViews: ParHashMap[Seq[String], DataFrame] = ParHashMap() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala index 8d54fa006de3d..e5b5a5298b975 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala @@ -271,12 +271,13 @@ object LogFileIterator { if (HoodieTableMetadata.isMetadataTable(tablePath)) { val metadataConfig = HoodieMetadataConfig.newBuilder() - .fromProperties(tableState.metadataConfig.getProps).enable(true).build() + .fromProperties(tableState.metadataConfig.getProps) + .withSpillableMapDir(hadoopConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + .enable(true).build() val dataTableBasePath = getDataTableBasePathFromMetadataTable(tablePath) val metadataTable = new HoodieBackedTableMetadata( new HoodieLocalEngineContext(hadoopConf), metadataConfig, - dataTableBasePath, - hadoopConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + dataTableBasePath) // We have to force full-scan for the MT log record reader, to make sure // we can iterate over all of the partitions, since by default some of the partitions (Column Stats, diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableFilesProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableFilesProcedure.scala index 1d9ac40087b5a..76b8efe525dd8 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableFilesProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableFilesProcedure.scala @@ -54,8 +54,7 @@ class ShowMetadataTableFilesProcedure() extends BaseProcedure with ProcedureBuil val basePath = getBasePath(table) val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build val config = HoodieMetadataConfig.newBuilder.enable(true).build - val metaReader = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(metaClient.getHadoopConf), - config, basePath, "/tmp") + val metaReader = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(metaClient.getHadoopConf), config, basePath) if (!metaReader.enabled){ throw new HoodieException(s"Metadata Table not enabled/initialized.") } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTablePartitionsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTablePartitionsProcedure.scala index 6142b6df73f14..9a7867d664861 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTablePartitionsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTablePartitionsProcedure.scala @@ -51,8 +51,7 @@ class ShowMetadataTablePartitionsProcedure() extends BaseProcedure with Procedur val basePath = getBasePath(table) val config = HoodieMetadataConfig.newBuilder.enable(true).build - val metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), - config, basePath, "/tmp") + val metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, basePath) if (!metadata.enabled){ throw new HoodieException(s"Metadata Table not enabled/initialized.") } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableStatsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableStatsProcedure.scala index 86fe0575f9cef..096a0ff1e3fa1 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableStatsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableStatsProcedure.scala @@ -50,8 +50,7 @@ class ShowMetadataTableStatsProcedure() extends BaseProcedure with ProcedureBuil val basePath = getBasePath(table) val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build val config = HoodieMetadataConfig.newBuilder.enable(true).build - val metadata = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(metaClient.getHadoopConf), - config, basePath, "/tmp") + val metadata = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(metaClient.getHadoopConf), config, basePath) val stats = metadata.stats val rows = new util.ArrayList[Row] diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateMetadataTableFilesProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateMetadataTableFilesProcedure.scala index 65b9fc49270d6..c756425b5b2b2 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateMetadataTableFilesProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ValidateMetadataTableFilesProcedure.scala @@ -63,7 +63,7 @@ class ValidateMetadataTableFilesProcedure() extends BaseProcedure with Procedure val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build val config = HoodieMetadataConfig.newBuilder.enable(true).build val metadataReader = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(metaClient.getHadoopConf), - config, basePath, "/tmp") + config, basePath) if (!metadataReader.enabled){ throw new HoodieException(s"Metadata Table not enabled/initialized.") @@ -71,7 +71,7 @@ class ValidateMetadataTableFilesProcedure() extends BaseProcedure with Procedure val fsConfig = HoodieMetadataConfig.newBuilder.enable(false).build val fsMetaReader = new HoodieBackedTableMetadata(new HoodieLocalEngineContext(metaClient.getHadoopConf), - fsConfig, basePath, "/tmp") + fsConfig, basePath) val timer = HoodieTimer.start val metadataPartitions = metadataReader.getAllPartitionPaths 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 facd0f6999818..c1db55c58220c 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 @@ -483,8 +483,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS // Test getting partition paths in a subset of directories val metadata = HoodieTableMetadata.create(context, HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build(), - metaClient.getBasePathV2.toString, - metaClient.getBasePathV2.getParent.toString) + metaClient.getBasePathV2.toString) assertEquals( Seq("1/2023/01/01", "1/2023/01/02"), metadata.getPartitionPathWithPathPrefixes(Seq("1")).sorted) 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 a077c2021c79f..16c0b01147984 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 @@ -44,13 +44,11 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -162,11 +160,6 @@ public class HoodieMetadataTableValidator implements Serializable { private final String taskLabels; - public HoodieMetadataTableValidator(HoodieTableMetaClient metaClient) { - this.metaClient = metaClient; - this.taskLabels = StringUtils.EMPTY_STRING; - } - public HoodieMetadataTableValidator(JavaSparkContext jsc, Config cfg) { this.jsc = jsc; this.cfg = cfg; @@ -1061,8 +1054,7 @@ public HoodieMetadataValidationContext( .build(); this.fileSystemView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, metaClient, metadataConfig); - this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(), - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue()); + this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath()); if (metaClient.getCommitsTimeline().filterCompletedInstants().countInstants() > 0) { this.allColumnNameList = getAllColumnNames(); } 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 95b4128f8cf30..0df91b4e169b4 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 @@ -28,7 +28,6 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; @@ -273,8 +272,7 @@ private void logTableStats(String basePath, LocalDate[] dateInterval) throws IOE .enable(isMetadataEnabled(basePath, jsc)) .build(); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePath, - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue()); + HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePath); SerializableConfiguration serializableConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration()); List allPartitions = tableMetadata.getAllPartitionPaths(); @@ -463,4 +461,4 @@ private static void logStats(String header, Histogram histogram) { LOG.info("P95 file size: {}", getFileSizeUnit(snapshot.getValue(0.95))); LOG.info("P99 file size: {}", getFileSizeUnit(snapshot.getValue(0.99))); } -} \ No newline at end of file +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java index 1e4a39dd8b4cc..c46383a1c88e4 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java @@ -187,8 +187,7 @@ public void testIndexerWithWriterFinishingFirst() throws IOException { metaClient.getActiveTimeline().readIndexPlanAsBytes(indexingInstant).get()); String indexUptoInstantTime = indexPlan.getIndexPartitionInfos().get(0).getIndexUptoInstant(); HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata( - context(), metadataConfig, metaClient.getBasePathV2().toString(), - getWriteConfigBuilder(basePath(), tableName).build().getSpillableMapBasePath()); + context(), metadataConfig, metaClient.getBasePathV2().toString()); HoodieTableMetaClient metadataMetaClient = metadata.getMetadataMetaClient(); String mdtCommitTime = indexUptoInstantTime + METADATA_INDEXER_TIME_SUFFIX; assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(mdtCommitTime)); @@ -233,7 +232,6 @@ public void testIndexerWithWriterFinishingFirst() throws IOException { } @Test - @Disabled("HUDI-6332") // Investigate and fix async indexer colstats index initialization public void testIndexerWithWriterFinishingLast() throws IOException { // Test the case where a regular write updating the metadata table is in progress, // i.e., a delta commit in the metadata table is inflight, and the async indexer @@ -253,8 +251,7 @@ public void testIndexerWithWriterFinishingLast() throws IOException { metaClient.getActiveTimeline().revertToInflight(commit); HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata( - context(), metadataConfig, metaClient.getBasePathV2().toString(), - getWriteConfigBuilder(basePath(), tableName).build().getSpillableMapBasePath()); + context(), metadataConfig, metaClient.getBasePathV2().toString()); HoodieTableMetaClient metadataMetaClient = metadata.getMetadataMetaClient(); HoodieInstant mdtCommit = metadataMetaClient.getActiveTimeline() .filter(i -> i.getTimestamp().equals(commitTime)) diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/HoodieOfflineJobTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/HoodieOfflineJobTestBase.java index 0b8735f1d65d0..a13629e6593a8 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/HoodieOfflineJobTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/HoodieOfflineJobTestBase.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -88,7 +89,7 @@ protected List writeData(boolean isUpsert, String instant, int numR org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatuses); if (doCommit) { List writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()); - boolean committed = client.commitStats(instant, writeStats, Option.empty(), metaClient.getCommitActionType()); + boolean committed = client.commitStats(instant, HoodieListData.eager(writeStatuses), writeStats, Option.empty(), metaClient.getCommitActionType()); Assertions.assertTrue(committed); } metaClient = HoodieTableMetaClient.reload(metaClient);