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 new file mode 100644 index 0000000000000..8eac8dd6fcd2a --- /dev/null +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -0,0 +1,226 @@ +/* + * 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.cli.commands; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.cli.HoodieCLI; +import org.apache.hudi.cli.utils.SparkUtil; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; + +import org.apache.spark.api.java.JavaSparkContext; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * CLI commands to operate on the Metadata Table. + */ +@Component +public class MetadataCommand implements CommandMarker { + + private JavaSparkContext jsc; + private static String metadataBaseDirectory; + + /** + * Sets the directory to store/read Metadata Table. + * + * This can be used to store the metadata table away from the dataset directory. + * - Useful for testing as well as for using via the HUDI CLI so that the actual dataset is not written to. + * - Useful for testing Metadata Table performance and operations on existing datasets before enabling. + */ + public static void setMetadataBaseDirectory(String metadataDir) { + ValidationUtils.checkState(metadataBaseDirectory == null, + "metadataBaseDirectory is already set to " + metadataBaseDirectory); + metadataBaseDirectory = metadataDir; + } + + public static String getMetadataTableBasePath(String tableBasePath) { + if (metadataBaseDirectory != null) { + return metadataBaseDirectory; + } + return HoodieTableMetadata.getMetadataTableBasePath(tableBasePath); + } + + @CliCommand(value = "metadata set", help = "Set options for Metadata Table") + public String set(@CliOption(key = {"metadataDir"}, + help = "Directory to read/write metadata table (can be different from dataset)", unspecifiedDefaultValue = "") + final String metadataDir) { + if (!metadataDir.isEmpty()) { + setMetadataBaseDirectory(metadataDir); + } + + return "Ok"; + } + + @CliCommand(value = "metadata create", help = "Create the Metadata Table if it does not exist") + public String create() throws IOException { + 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."); + } + } catch (FileNotFoundException e) { + // Metadata directory does not exist yet + HoodieCLI.fs.mkdirs(metadataPath); + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieWriteConfig writeConfig = getWriteConfig(); + initJavaSparkContext(); + SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); + return String.format("Created Metadata Table in %s (duration=%.2f secs)", metadataPath, timer.endTimer() / 1000.0); + } + + @CliCommand(value = "metadata delete", help = "Remove the Metadata Table") + public String delete() throws Exception { + 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 + } + + return String.format("Removed Metadata Table from %s", metadataPath); + } + + @CliCommand(value = "metadata init", help = "Update the metadata table from commits since the creation") + public String init(@CliOption(key = {"readonly"}, unspecifiedDefaultValue = "false", + help = "Open in read-only mode") final boolean readOnly) throws Exception { + HoodieCLI.getTableMetaClient(); + Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); + try { + HoodieCLI.fs.listStatus(metadataPath); + } catch (FileNotFoundException e) { + // Metadata directory does not exist + throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") does not exist."); + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + if (!readOnly) { + HoodieWriteConfig writeConfig = getWriteConfig(); + initJavaSparkContext(); + SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); + } + + String action = readOnly ? "Opened" : "Initialized"; + return String.format(action + " Metadata Table in %s (duration=%.2fsec)", metadataPath, (timer.endTimer()) / 1000.0); + } + + @CliCommand(value = "metadata stats", help = "Print stats about the metadata") + public String stats() throws IOException { + HoodieCLI.getTableMetaClient(); + HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(HoodieCLI.conf, HoodieCLI.basePath, "/tmp", true, false, false); + Map stats = metadata.stats(); + + StringBuffer out = new StringBuffer("\n"); + out.append(String.format("Base path: %s\n", getMetadataTableBasePath(HoodieCLI.basePath))); + for (Map.Entry entry : stats.entrySet()) { + out.append(String.format("%s: %s\n", entry.getKey(), entry.getValue())); + } + + return out.toString(); + } + + @CliCommand(value = "metadata list-partitions", help = "Print a list of all partitions from the metadata") + public String listPartitions() throws IOException { + HoodieCLI.getTableMetaClient(); + HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(HoodieCLI.conf, HoodieCLI.basePath, "/tmp", true, false, false); + + StringBuffer out = new StringBuffer("\n"); + if (!metadata.enabled()) { + out.append("=== Metadata Table not initilized. Using file listing to get list of partitions. ===\n\n"); + } + + long t1 = System.currentTimeMillis(); + List partitions = metadata.getAllPartitionPaths(); + long t2 = System.currentTimeMillis(); + + int[] count = {0}; + partitions.stream().sorted((p1, p2) -> p2.compareTo(p1)).forEach(p -> { + out.append(p); + if (++count[0] % 15 == 0) { + out.append("\n"); + } else { + out.append(", "); + } + }); + + out.append(String.format("\n\n=== List of partitions retrieved in %.2fsec ===", (t2 - t1) / 1000.0)); + + return out.toString(); + } + + @CliCommand(value = "metadata list-files", help = "Print a list of all files in a partition from the metadata") + public String listFiles( + @CliOption(key = {"partition"}, help = "Name of the partition to list files", mandatory = true) + final String partition) throws IOException { + HoodieCLI.getTableMetaClient(); + HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata(HoodieCLI.conf, HoodieCLI.basePath, "/tmp", true, false, false); + + StringBuffer out = new StringBuffer("\n"); + if (!metaReader.enabled()) { + out.append("=== Metadata Table not initialized. Using file listing to get list of files in partition. ===\n\n"); + } + + long t1 = System.currentTimeMillis(); + FileStatus[] statuses = metaReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition)); + long t2 = System.currentTimeMillis(); + + Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(p -> { + out.append("\t" + p.getPath().getName()); + out.append("\n"); + }); + + out.append(String.format("\n=== Files in partition retrieved in %.2fsec ===", (t2 - t1) / 1000.0)); + + return out.toString(); + } + + private HoodieWriteConfig getWriteConfig() { + return HoodieWriteConfig.newBuilder().withPath(HoodieCLI.basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); + } + + private void initJavaSparkContext() { + if (jsc == null) { + jsc = SparkUtil.initJavaSparkConf("HoodieClI"); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index ba7db3e998713..818751d6b2651 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -71,6 +71,7 @@ protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig cl this.timelineServer = timelineServer; shouldStopTimelineServer = !timelineServer.isPresent(); startEmbeddedServerView(); + initWrapperFSMetrics(); } /** @@ -118,6 +119,10 @@ public HoodieWriteConfig getConfig() { return config; } + protected void initWrapperFSMetrics() { + // no-op. + } + protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) { return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad, config.getConsistencyGuardConfig(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 014998749ca3f..e12b3af040787 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -134,6 +134,7 @@ public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig this.metrics = new HoodieMetrics(config, config.getTableName()); this.rollbackPending = rollbackPending; this.index = createIndex(writeConfig); + syncTableMetadata(); } protected abstract HoodieIndex createIndex(HoodieWriteConfig writeConfig); @@ -220,6 +221,10 @@ void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String } } + protected void syncTableMetadata() { + // no-op + } + /** * Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication. * @@ -407,7 +412,9 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); - autoCleanOnCommit(instantTime); + autoCleanOnCommit(); + + syncTableMetadata(); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } @@ -434,9 +441,8 @@ protected void runAnyPendingClustering(HoodieTable table) { /** * Handle auto clean during commit. * - * @param instantTime */ - protected void autoCleanOnCommit(String instantTime) { + protected void autoCleanOnCommit() { if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, if (config.isAsyncClean()) { @@ -444,8 +450,9 @@ protected void autoCleanOnCommit(String instantTime) { AsyncCleanerService.waitForCompletion(asyncCleanerService); LOG.info("Cleaner has finished"); } else { + // Do not reuse instantTime for clean as metadata table requires all changes to have unique instant timestamps. LOG.info("Auto cleaning is enabled. Running cleaner now"); - clean(instantTime); + clean(); } } } @@ -599,8 +606,14 @@ public HoodieCleanMetadata clean() { * Provides a new commit time for a write operation (insert/update/delete). */ public String startCommit() { + // NOTE : Need to ensure that rollback is done before a new commit is started + if (rollbackPending) { + // Only rollback pending commit/delta-commits. Do not touch compaction commits + rollbackPendingCommits(); + } String instantTime = HoodieActiveTimeline.createNewInstantTime(); - startCommitWithTime(instantTime); + HoodieTableMetaClient metaClient = createMetaClient(true); + startCommit(instantTime, metaClient.getCommitActionType(), metaClient); return instantTime; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java index e8016c957be65..c259acf2df67f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java @@ -19,6 +19,7 @@ package org.apache.hudi.client; import org.apache.hudi.async.HoodieAsyncService; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; @@ -52,11 +53,11 @@ protected Pair startService() { }), executor); } - public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient, - String instantTime) { + public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient) { AsyncCleanerService asyncCleanerService = null; if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) { - LOG.info("Auto cleaning is enabled. Running cleaner async to write operation"); + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + LOG.info("Auto cleaning is enabled. Running cleaner async to write operation at instant time " + instantTime); asyncCleanerService = new AsyncCleanerService(writeClient, instantTime); asyncCleanerService.start(null); } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java index 800c75f824fd4..b6cb6e5add99d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java @@ -62,6 +62,9 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig { public static final String METRICS_REPORTER_CLASS = METRIC_PREFIX + ".reporter.class"; public static final String DEFAULT_METRICS_REPORTER_CLASS = ""; + // Enable metrics collection from executors + public static final String ENABLE_EXECUTOR_METRICS = METRIC_PREFIX + ".executor.enable"; + private HoodieMetricsConfig(Properties props) { super(props); } @@ -126,6 +129,11 @@ public Builder withReporterClass(String className) { return this; } + public Builder withExecutorMetrics(boolean enable) { + props.setProperty(ENABLE_EXECUTOR_METRICS, String.valueOf(enable)); + return this; + } + public HoodieMetricsConfig build() { HoodieMetricsConfig config = new HoodieMetricsConfig(props); setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON)); 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 91223962e4988..2a26abdb94a0c 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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.common.config.DefaultHoodieConfig; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.client.common.EngineType; import org.apache.hudi.common.model.HoodieCleaningPolicy; @@ -81,6 +82,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String BULKINSERT_INPUT_DATA_SCHEMA_DDL = "hoodie.bulkinsert.schema.ddl"; public static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism"; public static final String DELETE_PARALLELISM = "hoodie.delete.shuffle.parallelism"; + public static final String FILE_LISTING_PARALLELISM = "hoodie.file.listing.parallelism"; public static final String DEFAULT_ROLLBACK_PARALLELISM = "100"; public static final String ROLLBACK_PARALLELISM = "hoodie.rollback.parallelism"; public static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes"; @@ -255,6 +257,10 @@ public int getRollbackParallelism() { return Integer.parseInt(props.getProperty(ROLLBACK_PARALLELISM)); } + public int getFileListingParallelism() { + return Math.max(Integer.parseInt(props.getProperty(FILE_LISTING_PARALLELISM)), 1); + } + public boolean shouldRollbackUsingMarkers() { return Boolean.parseBoolean(props.getProperty(ROLLBACK_USING_MARKERS)); } @@ -689,6 +695,10 @@ public boolean isMetricsOn() { return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON)); } + public boolean isExecutorMetricsEnabled() { + return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.ENABLE_EXECUTOR_METRICS, "false")); + } + public MetricsReporterType getMetricsReporterType() { return MetricsReporterType.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE)); } @@ -874,6 +884,41 @@ public Long getMaxMemoryPerPartitionMerge() { return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP)); } + /** + * File listing metadata configs. + */ + public boolean useFileListingMetadata() { + return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.METADATA_ENABLE_PROP)); + } + + public boolean getFileListingMetadataVerify() { + return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.METADATA_VALIDATE_PROP)); + } + + public int getMetadataInsertParallelism() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.METADATA_INSERT_PARALLELISM_PROP)); + } + + public int getMetadataCompactDeltaCommitMax() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.METADATA_COMPACT_NUM_DELTA_COMMITS_PROP)); + } + + public boolean isMetadataAsyncClean() { + return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.METADATA_ASYNC_CLEAN_PROP)); + } + + public int getMetadataMaxCommitsToKeep() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP_PROP)); + } + + public int getMetadataMinCommitsToKeep() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP)); + } + + public int getMetadataCleanerCommitsRetained() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP)); + } + public static class Builder { protected final Properties props = new Properties(); @@ -889,6 +934,7 @@ public static class Builder { private boolean isConsistencyGuardSet = false; private boolean isCallbackConfigSet = false; private boolean isPayloadConfigSet = false; + private boolean isMetadataConfigSet = false; public Builder withEngineType(EngineType engineType) { this.engineType = engineType; @@ -961,6 +1007,11 @@ public Builder withBulkInsertParallelism(int bulkInsertParallelism) { return this; } + public Builder withFileListingParallelism(int parallelism) { + props.setProperty(FILE_LISTING_PARALLELISM, String.valueOf(parallelism)); + return this; + } + public Builder withUserDefinedBulkInsertPartitionerClass(String className) { props.setProperty(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS, className); return this; @@ -1056,6 +1107,12 @@ public Builder withPayloadConfig(HoodiePayloadConfig payloadConfig) { return this; } + public Builder withMetadataConfig(HoodieMetadataConfig metadataConfig) { + props.putAll(metadataConfig.getProps()); + isMetadataConfigSet = true; + return this; + } + public Builder withAutoCommit(boolean autoCommit) { props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit)); return this; @@ -1141,6 +1198,8 @@ protected void setDefaults() { DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(DELETE_PARALLELISM), DELETE_PARALLELISM, DEFAULT_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(FILE_LISTING_PARALLELISM), FILE_LISTING_PARALLELISM, + DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM, DEFAULT_ROLLBACK_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(KEYGENERATOR_CLASS_PROP), @@ -1204,6 +1263,8 @@ protected void setDefaults() { HoodieWriteCommitCallbackConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isPayloadConfigSet, HoodiePayloadConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isMetadataConfigSet, + HoodieMetadataConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !props.containsKey(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION), EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION); 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 new file mode 100644 index 0000000000000..823e70c83b6ef --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -0,0 +1,459 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +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.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +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.Pair; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieMetricsConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieMetadataException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; +import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; + +/** + * Writer implementation backed by an internal hudi table. Partition and file listing are saved within an internal MOR table + * called Metadata Table. This table is created by listing files and partitions (first time) + * and kept in sync using the instants on the main dataset. + */ +public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMetadataWriter { + + private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadataWriter.class); + + protected HoodieWriteConfig metadataWriteConfig; + protected HoodieWriteConfig datasetWriteConfig; + protected String tableName; + + protected HoodieBackedTableMetadata metadata; + protected HoodieTableMetaClient metaClient; + protected Option metrics; + protected boolean enabled; + protected SerializableConfiguration hadoopConf; + protected final transient HoodieEngineContext engineContext; + + protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { + this.datasetWriteConfig = writeConfig; + this.engineContext = engineContext; + this.hadoopConf = new SerializableConfiguration(hadoopConf); + + if (writeConfig.useFileListingMetadata()) { + this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX; + this.metadataWriteConfig = createMetadataWriteConfig(writeConfig); + enabled = true; + + // Inline compaction and auto clean is required as we dont expose this table outside + ValidationUtils.checkArgument(!this.metadataWriteConfig.isAutoClean(), "Cleaning is controlled internally for Metadata table."); + ValidationUtils.checkArgument(!this.metadataWriteConfig.isInlineCompaction(), "Compaction is controlled internally for metadata table."); + // Metadata Table cannot have metadata listing turned on. (infinite loop, much?) + ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(), "Auto commit is required for Metadata Table"); + ValidationUtils.checkArgument(!this.metadataWriteConfig.useFileListingMetadata(), "File listing cannot be used for Metadata Table"); + + initRegistry(); + HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf, datasetWriteConfig.getBasePath()); + initialize(engineContext, datasetMetaClient); + if (enabled) { + // (re) init the metadata for reading. + initTableMetadata(); + + // This is always called even in case the table was created for the first time. This is because + // initFromFilesystem() does file listing and hence may take a long time during which some new updates + // may have occurred on the table. Hence, calling this always ensures that the metadata is brought in sync + // with the active timeline. + HoodieTimer timer = new HoodieTimer().startTimer(); + syncFromInstants(datasetMetaClient); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.SYNC_STR, timer.endTimer())); + } + } else { + enabled = false; + this.metrics = Option.empty(); + } + } + + protected abstract void initRegistry(); + + /** + * Create a {@code HoodieWriteConfig} to use for the Metadata Table. + * + * @param writeConfig {@code HoodieWriteConfig} of the main dataset writer + */ + private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfig) { + int parallelism = writeConfig.getMetadataInsertParallelism(); + + // Create the write config for the metadata table by borrowing options from the main write config. + HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() + .withConsistencyCheckEnabled(writeConfig.getConsistencyGuardConfig().isConsistencyCheckEnabled()) + .withInitialConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getInitialConsistencyCheckIntervalMs()) + .withMaxConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getMaxConsistencyCheckIntervalMs()) + .withMaxConsistencyChecks(writeConfig.getConsistencyGuardConfig().getMaxConsistencyChecks()) + .build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .withAutoCommit(true) + .withAvroSchemaValidate(true) + .withEmbeddedTimelineServerEnabled(false) + .withAssumeDatePartitioning(false) + .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) + .withSchema(HoodieMetadataRecord.getClassSchema().toString()) + .forTable(tableName) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withAsyncClean(writeConfig.isMetadataAsyncClean()) + // we will trigger cleaning manually, to control the instant times + .withAutoClean(false) + .withCleanerParallelism(parallelism) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) + .archiveCommitsWith(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMetadataMaxCommitsToKeep()) + // we will trigger compaction manually, to control the instant times + .withInlineCompaction(false) + .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()).build()) + .withParallelism(parallelism, parallelism) + .withDeleteParallelism(parallelism) + .withRollbackParallelism(parallelism) + .withFinalizeWriteParallelism(parallelism); + + if (writeConfig.isMetricsOn()) { + HoodieMetricsConfig.Builder metricsConfig = HoodieMetricsConfig.newBuilder() + .withReporterType(writeConfig.getMetricsReporterType().toString()) + .withExecutorMetrics(writeConfig.isExecutorMetricsEnabled()) + .on(true); + switch (writeConfig.getMetricsReporterType()) { + case GRAPHITE: + metricsConfig.onGraphitePort(writeConfig.getGraphiteServerPort()) + .toGraphiteHost(writeConfig.getGraphiteServerHost()) + .usePrefix(writeConfig.getGraphiteMetricPrefix()); + break; + case JMX: + metricsConfig.onJmxPort(writeConfig.getJmxPort()) + .toJmxHost(writeConfig.getJmxHost()); + break; + case DATADOG: + // TODO: + break; + case CONSOLE: + case INMEMORY: + break; + default: + throw new HoodieMetadataException("Unsupported Metrics Reporter type " + writeConfig.getMetricsReporterType()); + } + + builder.withMetricsConfig(metricsConfig.build()); + } + + return builder.build(); + } + + public HoodieWriteConfig getWriteConfig() { + return metadataWriteConfig; + } + + public HoodieBackedTableMetadata metadata() { + return metadata; + } + + /** + * Initialize the metadata table if it does not exist. Update the metadata to bring it in sync with the file system. + * + * This can happen in two ways: + * 1. If the metadata table did not exist, then file and partition listing is used + * 2. If the metadata table exists, the instants from active timeline are read in order and changes applied + * + * The above logic has been chosen because it is faster to perform #1 at scale rather than read all the Instants + * which are large in size (AVRO or JSON encoded and not compressed) and incur considerable IO for de-serialization + * and decoding. + */ + protected abstract void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient); + + private void initTableMetadata() { + this.metadata = new HoodieBackedTableMetadata(hadoopConf.get(), datasetWriteConfig.getBasePath(), datasetWriteConfig.getSpillableMapBasePath(), + datasetWriteConfig.useFileListingMetadata(), datasetWriteConfig.getFileListingMetadataVerify(), false, + datasetWriteConfig.shouldAssumeDatePartitioning()); + this.metaClient = metadata.getMetaClient(); + } + + protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + HoodieTimer timer = new HoodieTimer().startTimer(); + boolean exists = datasetMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); + if (!exists) { + // Initialize for the first time by listing partitions and files directly from the file system + bootstrapFromFilesystem(engineContext, datasetMetaClient); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); + } + } + + /** + * Initialize the Metadata Table by listing files and partitions from the file system. + * + * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + */ + private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + ValidationUtils.checkState(enabled, "Metadata table cannot be initialized as it is not enabled"); + + // If there is no commit on the dataset yet, use the SOLO_COMMIT_TIMESTAMP as the instant time for initial commit + // Otherwise, we use the timestamp of the instant which does not have any non-completed instants before it. + Option latestInstant = Option.empty(); + boolean foundNonComplete = false; + for (HoodieInstant instant : datasetMetaClient.getActiveTimeline().getInstants().collect(Collectors.toList())) { + if (!instant.isCompleted()) { + foundNonComplete = true; + } else if (!foundNonComplete) { + latestInstant = Option.of(instant); + } + } + + String createInstantTime = latestInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); + + HoodieTableMetaClient.initTableType(hadoopConf.get(), metadataWriteConfig.getBasePath(), + HoodieTableType.MERGE_ON_READ, tableName, "archived", HoodieMetadataPayload.class.getName(), + HoodieFileFormat.HFILE.toString()); + initTableMetadata(); + + // List all partitions in the basePath of the containing dataset + LOG.info("Initializing metadata table by using file listings in " + datasetWriteConfig.getBasePath()); + Map> partitionToFileStatus = getPartitionsToFilesMapping(datasetMetaClient); + + // Create a HoodieCommitMetadata with writeStats for all discovered files + int[] stats = {0}; + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + + partitionToFileStatus.forEach((partition, statuses) -> { + // Filter the statuses to only include files which were created before or on createInstantTime + statuses.stream().filter(status -> { + String filename = status.getPath().getName(); + return !HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(filename), HoodieTimeline.GREATER_THAN, + createInstantTime); + }).forEach(status -> { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName()); + writeStat.setPartitionPath(partition); + writeStat.setTotalWriteBytes(status.getLen()); + commitMetadata.addWriteStat(partition, writeStat); + stats[0] += 1; + }); + + // If the partition has no files then create a writeStat with no file path + if (commitMetadata.getWriteStats(partition) == null) { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPartitionPath(partition); + commitMetadata.addWriteStat(partition, writeStat); + } + }); + + LOG.info("Committing " + partitionToFileStatus.size() + " partitions and " + stats[0] + " files to metadata"); + update(commitMetadata, createInstantTime); + } + + /** + * Function to find hoodie partitions and list files in them in parallel. + * + * @param datasetMetaClient + * @return Map of partition names to a list of FileStatus for all the files in the partition + */ + private Map> getPartitionsToFilesMapping(HoodieTableMetaClient datasetMetaClient) { + List pathsToList = new LinkedList<>(); + pathsToList.add(new Path(datasetWriteConfig.getBasePath())); + + Map> partitionToFileStatus = new HashMap<>(); + final int fileListingParallelism = metadataWriteConfig.getFileListingParallelism(); + SerializableConfiguration conf = new SerializableConfiguration(datasetMetaClient.getHadoopConf()); + + while (!pathsToList.isEmpty()) { + int listingParallelism = Math.min(fileListingParallelism, pathsToList.size()); + // List all directories in parallel + List> dirToFileListing = engineContext.map(pathsToList, path -> { + FileSystem fs = path.getFileSystem(conf.get()); + return Pair.of(path, fs.listStatus(path)); + }, listingParallelism); + pathsToList.clear(); + + // If the listing reveals a directory, add it to queue. If the listing reveals a hoodie partition, add it to + // the results. + dirToFileListing.forEach(p -> { + List filesInDir = Arrays.stream(p.getRight()).parallel() + .filter(fs -> !fs.getPath().getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) + .collect(Collectors.toList()); + + if (p.getRight().length > filesInDir.size()) { + // Is a partition. Add all data files to result. + String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), p.getLeft()); + partitionToFileStatus.put(partitionName, filesInDir); + } else { + // Add sub-dirs to the queue + pathsToList.addAll(Arrays.stream(p.getRight()) + .filter(fs -> fs.isDirectory() && !fs.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) + .map(fs -> fs.getPath()) + .collect(Collectors.toList())); + } + }); + } + + return partitionToFileStatus; + } + + /** + * Sync the Metadata Table from the instants created on the dataset. + * + * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + */ + private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { + ValidationUtils.checkState(enabled, "Metadata table cannot be synced as it is not enabled"); + + try { + List instantsToSync = metadata.findInstantsToSync(datasetMetaClient); + if (instantsToSync.isEmpty()) { + return; + } + + LOG.info("Syncing " + instantsToSync.size() + " instants to metadata table: " + instantsToSync); + + // Read each instant in order and sync it to metadata table + final HoodieActiveTimeline timeline = datasetMetaClient.getActiveTimeline(); + for (HoodieInstant instant : instantsToSync) { + LOG.info("Syncing instant " + instant + " to metadata table"); + + Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, instant, metadata.getSyncedInstantTime()); + if (records.isPresent()) { + commit(records.get(), MetadataPartitionType.FILES.partitionPath(), instant.getTimestamp()); + } + } + // re-init the table metadata, for any future writes. + initTableMetadata(); + } catch (IOException ioe) { + throw new HoodieIOException("Unable to sync instants from data to metadata table.", ioe); + } + } + + /** + * Update from {@code HoodieCommitMetadata}. + * + * @param commitMetadata {@code HoodieCommitMetadata} + * @param instantTime Timestamp at which the commit was performed + */ + @Override + public void update(HoodieCommitMetadata commitMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieCleanerPlan}. + * + * @param cleanerPlan {@code HoodieCleanerPlan} + * @param instantTime Timestamp at which the clean plan was generated + */ + @Override + public void update(HoodieCleanerPlan cleanerPlan, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanerPlan, instantTime); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieCleanMetadata}. + * + * @param cleanMetadata {@code HoodieCleanMetadata} + * @param instantTime Timestamp at which the clean was completed + */ + @Override + public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieRestoreMetadata}. + * + * @param restoreMetadata {@code HoodieRestoreMetadata} + * @param instantTime Timestamp at which the restore was performed + */ + @Override + public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(restoreMetadata, instantTime, metadata.getSyncedInstantTime()); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieRollbackMetadata}. + * + * @param rollbackMetadata {@code HoodieRollbackMetadata} + * @param instantTime Timestamp at which the rollback was performed + */ + @Override + public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(rollbackMetadata, instantTime, metadata.getSyncedInstantTime()); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit. + * + */ + protected abstract void commit(List records, String partitionName, String instantTime); +} 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 new file mode 100644 index 0000000000000..fa1f46402c88e --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; + +import java.io.Serializable; + +/** + * Interface that supports updating metadata for a given table, as actions complete. + */ +public interface HoodieTableMetadataWriter extends Serializable { + + void update(HoodieCommitMetadata commitMetadata, String instantTime); + + void update(HoodieCleanerPlan cleanerPlan, String instantTime); + + void update(HoodieCleanMetadata cleanMetadata, String instantTime); + + void update(HoodieRestoreMetadata restoreMetadata, String instantTime); + + void update(HoodieRollbackMetadata rollbackMetadata, String instantTime); +} 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 6b7a7d29cc436..d56e6e7dd412f 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 @@ -49,6 +49,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; @@ -61,6 +62,8 @@ import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.metadata.HoodieMetadataFileSystemView; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.log4j.LogManager; @@ -94,6 +97,7 @@ public abstract class HoodieTable implem private SerializableConfiguration hadoopConfiguration; private transient FileSystemViewManager viewManager; + private HoodieTableMetadata metadata; protected final TaskContextSupplier taskContextSupplier; @@ -242,28 +246,41 @@ public Configuration getHadoopConf() { * Get the view of the file system for this table. */ public TableFileSystemView getFileSystemView() { - return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline()); + if (config.useFileListingMetadata()) { + return getFileSystemViewInternal(getCompletedCommitsTimeline()); + } else { + return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline()); + } } /** * Get the base file only view of the file system for this table. */ public BaseFileOnlyView getBaseFileOnlyView() { - return getViewManager().getFileSystemView(metaClient); + return getFileSystemViewInternal(metaClient.getActiveTimeline().filterCompletedAndCompactionInstants()); } /** * Get the full view of the file system for this table. */ public SliceView getSliceView() { - return getViewManager().getFileSystemView(metaClient); + return getFileSystemViewInternal(metaClient.getActiveTimeline().filterCompletedAndCompactionInstants()); } /** * Get complete view of the file system for this table with ability to force sync. */ public SyncableFileSystemView getHoodieView() { - return getViewManager().getFileSystemView(metaClient); + return getFileSystemViewInternal(metaClient.getActiveTimeline().filterCompletedAndCompactionInstants()); + } + + private SyncableFileSystemView getFileSystemViewInternal(HoodieTimeline timeline) { + if (config.useFileListingMetadata()) { + FileSystemViewStorageConfig viewConfig = config.getViewStorageConfig(); + return new HoodieMetadataFileSystemView(metaClient, this.metadata(), timeline, viewConfig.isIncrementalTimelineSyncEnabled()); + } else { + return getViewManager().getFileSystemView(metaClient); + } } /** @@ -640,4 +657,12 @@ public String getBaseFileExtension() { public boolean requireSortedRecords() { return getBaseFileFormat() == HoodieFileFormat.HFILE; } + + public HoodieTableMetadata metadata() { + if (metadata == null) { + metadata = HoodieTableMetadata.create(hadoopConfiguration.get(), config.getBasePath(), config.getSpillableMapBasePath(), + config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.isMetricsOn(), config.shouldAssumeDatePartitioning()); + } + return metadata; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index e24ae73e38389..80724c85e5b3e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -195,6 +195,20 @@ private Stream getInstantsToArchive() { .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), HoodieInstant.getComparableAction(i.getAction())))); + // If metadata table is enabled, do not archive instants which are more recent that the latest synced + // instant on the metadata table. This is required for metadata table sync. + if (config.useFileListingMetadata()) { + Option lastSyncedInstantTime = table.metadata().getSyncedInstantTime(); + if (lastSyncedInstantTime.isPresent()) { + LOG.info("Limiting archiving of instants to last synced instant on metadata table at " + lastSyncedInstantTime.get()); + instants = instants.filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.LESSER_THAN, + lastSyncedInstantTime.get())); + } else { + LOG.info("Not archiving as there is no instants yet on the metadata table"); + instants = Stream.empty(); + } + } + return instants.flatMap(hoodieInstant -> groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), HoodieInstant.getComparableAction(hoodieInstant.getAction()))).stream()); 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 4f9b2a24ccd66..31d433d0fe386 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 @@ -20,7 +20,6 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CleanFileInfo; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; @@ -180,14 +179,13 @@ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata } /** - * Scan and list all paritions for cleaning. + * Scan and list all partitions for cleaning. * @return all partitions paths for the dataset. * @throws IOException */ private List getPartitionPathsForFullCleaning() throws IOException { // Go to brute force mode of scanning all partitions - return FSUtils.getAllPartitionPaths(hoodieTable.getMetaClient().getFs(), hoodieTable.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + return hoodieTable.metadata().getAllPartitionPaths(); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java index 404cc0258e242..5a42cdcb7b912 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -66,8 +66,10 @@ public Option generateClusteringPlan() { try { HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); LOG.info("Scheduling clustering for " + metaClient.getBasePath()); + HoodieWriteConfig config = getWriteConfig(); List partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), - getWriteConfig().shouldAssumeDatePartitioning()); + config.useFileListingMetadata(), config.getFileListingMetadataVerify(), + config.shouldAssumeDatePartitioning()); // filter the partition paths if needed to reduce list status partitionPaths = filterPartitionPaths(partitionPaths); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java index 657057f91e2e9..d3c2557941538 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java @@ -18,6 +18,7 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; @@ -37,6 +38,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; +import java.util.Objects; /** * Performs rollback using marker files generated during the write.. @@ -119,10 +121,17 @@ protected HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant } } + Map filesToNumBlocksRollback = Collections.emptyMap(); + if (config.useFileListingMetadata()) { + // When metadata is enabled, the information of files appended to is required + filesToNumBlocksRollback = Collections.singletonMap( + table.getMetaClient().getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()), + 1L); + } + return HoodieRollbackStat.newBuilder() .withPartitionPath(partitionPath) - // we don't use this field per se. Avoiding the extra file status call. - .withRollbackBlockAppendResults(Collections.emptyMap()) + .withRollbackBlockAppendResults(filesToNumBlocksRollback) .build(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index 897b448fb5340..bb59a66048352 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -88,12 +88,13 @@ static HoodieRollbackStat mergeRollbackStat(HoodieRollbackStat stat1, HoodieRoll * Generate all rollback requests that needs rolling back this action without actually performing rollback for COW table type. * @param fs instance of {@link FileSystem} to use. * @param basePath base path of interest. - * @param shouldAssumeDatePartitioning {@code true} if date partitioning should be assumed. {@code false} otherwise. + * @param config instance of {@link HoodieWriteConfig} to use. * @return {@link List} of {@link ListingBasedRollbackRequest}s thus collected. */ - public static List generateRollbackRequestsByListingCOW(FileSystem fs, String basePath, boolean shouldAssumeDatePartitioning) { + public static List generateRollbackRequestsByListingCOW(FileSystem fs, String basePath, HoodieWriteConfig config) { try { - return FSUtils.getAllPartitionPaths(fs, basePath, shouldAssumeDatePartitioning).stream() + return FSUtils.getAllPartitionPaths(fs, basePath, config.useFileListingMetadata(), + config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()).stream() .map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction) .collect(Collectors.toList()); } catch (IOException e) { @@ -113,7 +114,7 @@ public static List generateRollbackRequestsUsingFil String commit = instantToRollback.getTimestamp(); HoodieWriteConfig config = table.getConfig(); List partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1); context.setJobStatus(RollbackUtils.class.getSimpleName(), "Generate all rollback requests"); return context.flatMap(partitions, partitionPath -> { 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 90a96b9145414..c1d2c4a9cf290 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 @@ -36,6 +36,7 @@ import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -89,15 +90,18 @@ public HoodieSavepointMetadata execute() { "Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained); context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime); - Map> latestFilesMap = context.mapToPair(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), - table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()), partitionPath -> { - // Scan all partitions files with this commit time - LOG.info("Collecting latest files in partition path " + partitionPath); - TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); - List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) - .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); - return new ImmutablePair<>(partitionPath, latestFiles); - }, null); + List partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), + table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(), + config.shouldAssumeDatePartitioning() + ); + Map> latestFilesMap = context.mapToPair(partitions, partitionPath -> { + // Scan all partitions files with this commit time + LOG.info("Collecting latest files in partition path " + partitionPath); + TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); + List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) + .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); + return new ImmutablePair<>(partitionPath, latestFiles); + }, null); HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint table.getActiveTimeline().createNewInstant( 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 3e8952e0c4439..e7fda7d803925 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 @@ -107,7 +107,7 @@ public List upsert(List> records, String instantTim getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.UPSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -126,7 +126,7 @@ public List insert(List> records, String instantTim getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.insert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java index 28b713b24b9ab..6221dd5d49106 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java @@ -64,8 +64,8 @@ protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() { @Override protected List executeRollbackUsingFileListing(HoodieInstant instantToRollback) { - List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW( + table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), config); return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 1fa3ad0a32b7c..17453bb0d3662 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -94,7 +94,7 @@ private static void recreateMarkerFiles(final String commitInstantTime, List rollbackRequests; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - table.getConfig().shouldAssumeDatePartitioning()); + table.getConfig()); } else { rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } 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 71a85deff9c30..b6ffbb99a3a61 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 @@ -98,7 +98,7 @@ public List upsert(List> records, getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.UPSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -113,7 +113,7 @@ public List upsertPreppedRecords(List> preppedRecor getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.UPSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -124,7 +124,7 @@ public List insert(List> records, String instantTim getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.insert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -139,7 +139,7 @@ public List insertPreppedRecords(List> preppedRecor getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } 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 f7e7690bb163f..9a22f7894d798 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 @@ -23,6 +23,8 @@ import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -41,6 +43,8 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.metrics.DistributedRegistry; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -51,6 +55,7 @@ import org.apache.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -136,7 +141,7 @@ public JavaRDD upsert(JavaRDD> records, String inst getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.UPSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -150,7 +155,7 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); table.validateUpsertSchema(); setOperationType(WriteOperationType.UPSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -161,7 +166,7 @@ public JavaRDD insert(JavaRDD> records, String inst getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.insert(context,instantTime, records); return postWrite(result, instantTime, table); } @@ -172,7 +177,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -188,7 +193,7 @@ public HoodieWriteResult insertOverwrite(JavaRDD> records, final HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.INSERT_OVERWRITE); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records); return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); } @@ -205,7 +210,7 @@ public HoodieWriteResult insertOverwriteTable(JavaRDD> records, HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.INSERT_OVERWRITE_TABLE); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records); return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); } @@ -221,7 +226,7 @@ public JavaRDD bulkInsert(JavaRDD> records, String getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.BULK_INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner); return postWrite(result, instantTime, table); } @@ -232,7 +237,7 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime); table.validateInsertSchema(); setOperationType(WriteOperationType.BULK_INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner); return postWrite(result, instantTime, table); } @@ -394,4 +399,34 @@ private HoodieTable>, JavaRDD, JavaRDD> loadInvolvedFiles(List partitio HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); try { List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), - config.shouldAssumeDatePartitioning()); + config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable); } catch (IOException e) { throw new HoodieIOException("Failed to load all partitions", e); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java index bdb4991cf76e3..092c62becd2f8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java @@ -104,7 +104,8 @@ protected List> getAllBaseFilesInTable(final Hoodie final HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); try { - List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); + List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), + config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); // Obtain the latest data files from all the partitions. return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable); } catch (IOException e) { 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 new file mode 100644 index 0000000000000..262ad0e4d9730 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.metrics.DistributedRegistry; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.conf.Configuration; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { + + private static final Logger LOG = LogManager.getLogger(SparkHoodieBackedTableMetadataWriter.class); + + public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) { + return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context); + } + + SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { + super(hadoopConf, writeConfig, engineContext); + } + + @Override + protected void initRegistry() { + if (metadataWriteConfig.isMetricsOn()) { + Registry registry; + if (metadataWriteConfig.isExecutorMetricsEnabled()) { + registry = Registry.getRegistry("HoodieMetadata", DistributedRegistry.class.getName()); + } else { + registry = Registry.getRegistry("HoodieMetadata"); + } + this.metrics = Option.of(new HoodieMetadataMetrics(registry)); + } else { + this.metrics = Option.empty(); + } + } + + @Override + protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) { + try { + metrics.map(HoodieMetadataMetrics::registry).ifPresent(registry -> { + if (registry instanceof DistributedRegistry) { + HoodieSparkEngineContext sparkEngineContext = (HoodieSparkEngineContext) engineContext; + ((DistributedRegistry) registry).register(sparkEngineContext.getJavaSparkContext()); + } + }); + + if (enabled) { + bootstrapIfNeeded(engineContext, datasetMetaClient); + } + } catch (IOException e) { + LOG.error("Failed to initialize metadata table. Disabling the writer.", e); + enabled = false; + } + } + + @Override + protected void commit(List records, String partitionName, String instantTime) { + ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); + metadata.closeReaders(); + + JavaRDD recordRDD = prepRecords(records, partitionName); + + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { + writeClient.startCommitWithTime(instantTime); + List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect(); + statuses.forEach(writeStatus -> { + if (writeStatus.hasErrors()) { + throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); + } + }); + // trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future + // delta commits synced over will not have an instant time lesser than the last completed instant on the + // metadata table. + if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) { + writeClient.compact(instantTime + "001"); + } + writeClient.clean(instantTime + "002"); + } + + // Update total size of the metadata and count of base/log files + metrics.ifPresent(m -> { + try { + Map stats = m.getStats(false, metaClient, metadata); + m.updateMetrics(Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)), + Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)), + Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)), + Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES))); + } catch (HoodieIOException e) { + LOG.error("Could not publish metadata size metrics", e); + } + }); + } + + /** + * Tag each record with the location. + * + * Since we only read the latest base file in a partition, we tag the records with the instant time of the latest + * base file. + */ + private JavaRDD prepRecords(List records, String partitionName) { + HoodieTable table = HoodieSparkTable.create(metadataWriteConfig, engineContext); + TableFileSystemView.SliceView fsView = table.getSliceView(); + List baseFiles = fsView.getLatestFileSlices(partitionName) + .map(FileSlice::getBaseFile) + .filter(Option::isPresent) + .map(Option::get) + .collect(Collectors.toList()); + + // All the metadata fits within a single base file + if (partitionName.equals(MetadataPartitionType.FILES.partitionPath())) { + if (baseFiles.size() > 1) { + throw new HoodieMetadataException("Multiple base files found in metadata partition"); + } + } + + JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext(); + String fileId; + String instantTime; + if (!baseFiles.isEmpty()) { + fileId = baseFiles.get(0).getFileId(); + instantTime = baseFiles.get(0).getCommitTime(); + } else { + // If there is a log file then we can assume that it has the data + List logFiles = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()) + .map(FileSlice::getLatestLogFile) + .filter(Option::isPresent) + .map(Option::get) + .collect(Collectors.toList()); + if (logFiles.isEmpty()) { + // No base and log files. All are new inserts + return jsc.parallelize(records, 1); + } + + fileId = logFiles.get(0).getFileId(); + instantTime = logFiles.get(0).getBaseCommitTime(); + } + + return jsc.parallelize(records, 1).map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/DistributedRegistry.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/DistributedRegistry.java new file mode 100644 index 0000000000000..22b3afd10e058 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/DistributedRegistry.java @@ -0,0 +1,107 @@ +/* + * 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.metrics; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hudi.common.metrics.Registry; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.util.AccumulatorV2; + +/** + * Lightweight Metrics Registry to track Hudi events. + */ +public class DistributedRegistry extends AccumulatorV2, Map> + implements Registry, Serializable { + private String name; + ConcurrentHashMap counters = new ConcurrentHashMap<>(); + + public DistributedRegistry(String name) { + this.name = name; + } + + public void register(JavaSparkContext jsc) { + if (!isRegistered()) { + jsc.sc().register(this); + } + } + + @Override + public void clear() { + counters.clear(); + } + + @Override + public void increment(String name) { + counters.merge(name, 1L, (oldValue, newValue) -> oldValue + newValue); + } + + @Override + public void add(String name, long value) { + counters.merge(name, value, (oldValue, newValue) -> oldValue + newValue); + } + + /** + * Get all Counter type metrics. + */ + @Override + public Map getAllCounts(boolean prefixWithRegistryName) { + HashMap countersMap = new HashMap<>(); + counters.forEach((k, v) -> { + String key = prefixWithRegistryName ? name + "." + k : k; + countersMap.put(key, v); + }); + return countersMap; + } + + @Override + public void add(Map arg) { + arg.forEach((key, value) -> add(key, value)); + } + + @Override + public AccumulatorV2, Map> copy() { + DistributedRegistry registry = new DistributedRegistry(name); + counters.forEach((key, value) -> registry.add(key, value)); + return registry; + } + + @Override + public boolean isZero() { + return counters.isEmpty(); + } + + @Override + public void merge(AccumulatorV2, Map> acc) { + acc.value().forEach((key, value) -> add(key, value)); + } + + @Override + public void reset() { + counters.clear(); + } + + @Override + public Map value() { + return counters; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java index c0145158f3b71..6cc474f6670f2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -51,7 +51,8 @@ protected Map> getPartitionToReplacedFileIds(JavaRDD> partitionToExistingFileIds = new HashMap<>(); try { List partitionPaths = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), - table.getMetaClient().getBasePath(), false); + table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(), + false); JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); if (partitionPaths != null && partitionPaths.size() > 0) { context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index 96d52a1605c62..21ffd46cd24d0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -196,7 +196,7 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); List partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), - config.shouldAssumeDatePartitioning()); + config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); // filter the partition paths if needed to reduce list status partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java index 965d8055943a5..b770bbfe55330 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java @@ -66,8 +66,8 @@ protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() { @Override protected List executeRollbackUsingFileListing(HoodieInstant instantToRollback) { - List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), + table.getMetaClient().getBasePath(), config); return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 7e3faf32b6528..1a1cb3f55cbab 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -93,7 +93,7 @@ private static void recreateMarkerFiles(final String commitInstantTime, List rollbackRequests; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - table.getConfig().shouldAssumeDatePartitioning()); + table.getConfig()); } else { rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } 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 d04a2df1b4eb4..eab9bb191d3c0 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 @@ -99,8 +99,10 @@ public void testSavepointAndRollback() throws Exception { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); + HoodieWriteConfig config = getConfig(); List partitionPaths = - FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); + FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), config.useFileListingMetadata(), + config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view1 = table.getBaseFileOnlyView(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java index 03328ddddc142..e59a95009f212 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java @@ -37,6 +37,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -70,6 +71,11 @@ public void setUp() throws Exception { client = new CompactionAdminClient(context, basePath); } + @AfterEach + public void cleanUp() throws Exception { + cleanupResources(); + } + @Test public void testUnscheduleCompactionPlan() throws Exception { int numEntriesPerInstant = 10; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java new file mode 100644 index 0000000000000..34c0a3570740a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -0,0 +1,904 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieMetricsConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestHarness; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieBackedMetadata extends HoodieClientTestHarness { + private static final Logger LOG = LogManager.getLogger(TestHoodieBackedMetadata.class); + + @TempDir + public java.nio.file.Path tempFolder; + + private String metadataTableBasePath; + + private HoodieTableType tableType; + + public void init(HoodieTableType tableType) throws IOException { + this.tableType = tableType; + initPath(); + initSparkContexts("TestHoodieMetadata"); + initFileSystem(); + fs.mkdirs(new Path(basePath)); + initMetaClient(tableType); + initTestDataGenerator(); + metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + + } + + @AfterEach + public void clean() throws IOException { + cleanupResources(); + } + + /** + * Metadata Table should not be created unless it is enabled in config. + */ + @Test + public void testDefaultNoMetadataTable() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Metadata table should not exist until created for the first time + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist"); + assertThrows(TableNotFoundException.class, () -> new HoodieTableMetaClient(hadoopConf, metadataTableBasePath)); + + // Metadata table is not created if disabled by config + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + client.startCommitWithTime("001"); + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not be created"); + assertThrows(TableNotFoundException.class, () -> new HoodieTableMetaClient(hadoopConf, metadataTableBasePath)); + } + + // Metadata table created when enabled by config + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + client.startCommitWithTime("001"); + assertTrue(fs.exists(new Path(metadataTableBasePath))); + validateMetadata(client); + } + } + + /** + * Only valid partition directories are added to the metadata. + */ + @Test + public void testOnlyValidPartitionsAdded() throws Exception { + // This test requires local file system + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Create an empty directory which is not a partition directory (lacks partition metadata) + final String nonPartitionDirectory = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0] + "-nonpartition"; + Files.createDirectories(Paths.get(basePath, nonPartitionDirectory)); + + // Create some commits + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + testTable.withPartitionMetaFiles("p1", "p2") + .addCommit("001").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10) + .addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10) + .addInflightCommit("003").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + client.startCommitWithTime("005"); + + List partitions = metadataWriter(client).metadata().getAllPartitionPaths(); + assertFalse(partitions.contains(nonPartitionDirectory), + "Must not contain the non-partition " + nonPartitionDirectory); + assertTrue(partitions.contains("p1"), "Must contain partition p1"); + assertTrue(partitions.contains("p2"), "Must contain partition p2"); + + FileStatus[] statuses = metadata(client).getAllFilesInPartition(new Path(basePath, "p1")); + assertTrue(statuses.length == 2); + statuses = metadata(client).getAllFilesInPartition(new Path(basePath, "p2")); + assertTrue(statuses.length == 5); + } + } + + /** + * Test various table operations sync to Metadata Table correctly. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testTableOperations(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + + // Write 1 (Bulk insert) + String newCommitTime = "001"; + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 2 (inserts) + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + validateMetadata(client); + + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 3 (updates) + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 4 (updates and inserts) + newCommitTime = "004"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "005"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Write 5 (updates and inserts) + newCommitTime = "006"; + client.startCommitWithTime(newCommitTime); + 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) { + newCommitTime = "007"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Deletes + newCommitTime = "008"; + records = dataGen.generateDeletes(newCommitTime, 10); + JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); + client.startCommitWithTime(newCommitTime); + client.delete(deleteKeys, newCommitTime); + validateMetadata(client); + + // Clean + newCommitTime = "009"; + client.clean(newCommitTime); + validateMetadata(client); + + // Restore + client.restoreToInstant("006"); + validateMetadata(client); + } + } + + /** + * Test rollback of various table operations sync to Metadata Table correctly. + */ + //@ParameterizedTest + //@EnumSource(HoodieTableType.class) + //public void testRollbackOperations(HoodieTableType tableType) throws Exception { + @Test + public void testRollbackOperations() throws Exception { + //FIXME(metadata): This is broken for MOR, until HUDI-1434 is fixed + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Write 1 (Bulk insert) + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 2 (inserts) + Rollback of inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Write 3 (updates) + Rollback of updates + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 20); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Rollback of updates and inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Rollback of Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Rollback of Deletes + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateDeletes(newCommitTime, 10); + JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); + client.startCommitWithTime(newCommitTime); + writeStatuses = client.delete(deleteKeys, newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Rollback of Clean + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.clean(newCommitTime); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + } + + // Rollback of partial commits + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(false).build())) { + // Write updates and inserts + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + } + + // Marker based rollback of partial commits + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(true).build())) { + // Write updates and inserts + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + } + } + + /** + * Test when syncing rollback to metadata if the commit being rolled back has not been synced that essentially a no-op + * occurs to metadata. + * @throws Exception + */ + @Test + public void testRollbackUnsyncedCommit() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Initialize table with metadata + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // Commit with metadata disabled + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + client.rollback(newCommitTime); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true))) { + validateMetadata(client); + } + } + + /** + * Test sync of table operations. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testSync(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + String newCommitTime; + List records; + List writeStatuses; + + // Initial commits without metadata table enabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + } + + // Enable metadata table so it initialized by listing from file system + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 5); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + validateMetadata(client); + assertTrue(metadata(client).isInSync()); + } + + // Various table operations without metadata table enabled + String restoreToInstant; + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // updates + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 5); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + assertFalse(metadata(client).isInSync()); + + // updates and inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + assertFalse(metadata(client).isInSync()); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + assertFalse(metadata(client).isInSync()); + } + + // Savepoint + restoreToInstant = newCommitTime; + if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { + client.savepoint("hoodie", "metadata test"); + assertFalse(metadata(client).isInSync()); + } + + // Deletes + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateDeletes(newCommitTime, 5); + JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); + client.startCommitWithTime(newCommitTime); + client.delete(deleteKeys, newCommitTime); + assertFalse(metadata(client).isInSync()); + + // Clean + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.clean(newCommitTime); + assertFalse(metadata(client).isInSync()); + + // updates + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + assertFalse(metadata(client).isInSync()); + } + + // Enable metadata table and ensure it is synced + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details + client.restoreToInstant(restoreToInstant); + assertFalse(metadata(client).isInSync()); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + + validateMetadata(client); + assertTrue(metadata(client).isInSync()); + } + } + + /** + * Instants on Metadata Table should be archived as per config. + * Metadata Table should be automatically compacted as per config. + */ + @ParameterizedTest + @ValueSource(booleans = {false}) + public void testCleaningArchivingAndCompaction(boolean asyncClean) throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + final int maxDeltaCommitsBeforeCompaction = 4; + HoodieWriteConfig config = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) + .archiveCommitsWith(6, 8).retainCommits(1) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3) + .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(asyncClean).build()) + .build(); + + List records; + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, config)) { + for (int i = 1; i < 10; ++i) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + if (i == 1) { + records = dataGen.generateInserts(newCommitTime, 5); + } else { + records = dataGen.generateUpdates(newCommitTime, 2); + } + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + } + + HoodieTableMetaClient metadataMetaClient = new HoodieTableMetaClient(hadoopConf, metadataTableBasePath); + HoodieActiveTimeline metadataTimeline = metadataMetaClient.getActiveTimeline(); + // check that there are 2 compactions. + assertEquals(2, metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants()); + // check that cleaning has, once after each compaction. There will be more instances on the timeline, since it's less aggressively archived + assertEquals(4, metadataTimeline.getCleanerTimeline().filterCompletedInstants().countInstants()); + // ensure archiving has happened + List instants = metadataTimeline.getCommitsAndCompactionTimeline() + .getInstants().collect(Collectors.toList()); + Collections.reverse(instants); + long numDeltaCommits = instants.stream().filter(instant -> instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)).count(); + assertEquals(5, numDeltaCommits); + } + + /** + * Test various error scenarios. + */ + @Test + public void testErrorCases() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table + // should be rolled back to last valid commit. + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 5); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // 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, + commitInstantFileName), false)); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + // Start the next commit which will rollback the previous one and also should update the metadata table by + // updating it with HoodieRollbackMetadata. + String newCommitTime = client.startCommit(); + + // Dangling commit but metadata should be valid at this time + validateMetadata(client); + + // Next insert + List records = dataGen.generateInserts(newCommitTime, 5); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Post rollback commit and metadata should be valid + validateMetadata(client); + } + } + + /** + * Test non-partitioned datasets. + */ + @Test + public void testNonPartitioned() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + HoodieTestDataGenerator nonPartitionedGenerator = new HoodieTestDataGenerator(new String[] {""}); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Write 1 (Bulk insert) + String newCommitTime = "001"; + List records = nonPartitionedGenerator.generateInserts(newCommitTime, 10); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + validateMetadata(client); + + List metadataPartitions = metadata(client).getAllPartitionPaths(); + assertTrue(metadataPartitions.contains(""), "Must contain empty partition"); + } + } + + /** + * Test various metrics published by metadata table. + */ + @Test + public void testMetadataMetrics() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, true).build())) { + // Write + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + Registry metricsRegistry = Registry.getRegistry("HoodieMetadata"); + assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count")); + assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration")); + assertEquals(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count"), 1L); + assertTrue(metricsRegistry.getAllCounts().containsKey("basefile.size")); + assertTrue(metricsRegistry.getAllCounts().containsKey("logfile.size")); + assertTrue(metricsRegistry.getAllCounts().containsKey("basefile.count")); + assertTrue(metricsRegistry.getAllCounts().containsKey("logfile.count")); + } + } + + /** + * Test when reading from metadata table which is out of sync with dataset that results are still consistent. + */ + @Test + public void testMetadataOutOfSync() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true)); + + // Enable metadata so table is initialized + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Perform Bulk Insert + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 20); + client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + } + + // Perform commit operations with metadata disabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // Perform Insert + String newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 20); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + + // Perform Upsert + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 20); + client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "004"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + } + } + + assertFalse(metadata(unsyncedClient).isInSync()); + validateMetadata(unsyncedClient); + + // Perform clean operation with metadata disabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // One more commit needed to trigger clean so upsert and compact + String newCommitTime = "005"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 20); + client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "006"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + } + + // Clean + newCommitTime = "007"; + client.clean(newCommitTime); + } + + assertFalse(metadata(unsyncedClient).isInSync()); + validateMetadata(unsyncedClient); + + // Perform restore with metadata disabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + client.restoreToInstant("004"); + } + + assertFalse(metadata(unsyncedClient).isInSync()); + validateMetadata(unsyncedClient); + } + + + /** + * Validate the metadata tables contents to ensure it matches what is on the file system. + * + * @throws IOException + */ + private void validateMetadata(SparkRDDWriteClient client) throws IOException { + HoodieWriteConfig config = client.getConfig(); + + HoodieBackedTableMetadata tableMetadata = metadata(client); + assertNotNull(tableMetadata, "MetadataReader should have been initialized"); + if (!config.useFileListingMetadata()) { + return; + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Partitions should match + List fsPartitions = FSUtils.getAllFoldersWithPartitionMetaFile(fs, basePath); + List metadataPartitions = tableMetadata.getAllPartitionPaths(); + + Collections.sort(fsPartitions); + Collections.sort(metadataPartitions); + + assertEquals(fsPartitions.size(), metadataPartitions.size(), "Partitions should match"); + 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(); + fsPartitions.forEach(partition -> { + try { + Path partitionPath; + if (partition.equals("")) { + // Should be the non-partitioned case + partitionPath = new Path(basePath); + } else { + partitionPath = new Path(basePath, partition); + } + FileStatus[] fsStatuses = FSUtils.getAllDataFilesInPartition(fs, partitionPath); + FileStatus[] metaStatuses = tableMetadata.getAllFilesInPartition(partitionPath); + List fsFileNames = Arrays.stream(fsStatuses) + .map(s -> s.getPath().getName()).collect(Collectors.toList()); + List metadataFilenames = Arrays.stream(metaStatuses) + .map(s -> s.getPath().getName()).collect(Collectors.toList()); + Collections.sort(fsFileNames); + Collections.sort(metadataFilenames); + + // File sizes should be valid + Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getLen() > 0)); + + if ((fsFileNames.size() != metadataFilenames.size()) || (!fsFileNames.equals(metadataFilenames))) { + LOG.info("*** File system listing = " + Arrays.toString(fsFileNames.toArray())); + LOG.info("*** Metadata listing = " + Arrays.toString(metadataFilenames.toArray())); + } + assertEquals(fsFileNames.size(), metadataFilenames.size(), "Files within partition " + partition + " should match"); + assertTrue(fsFileNames.equals(metadataFilenames), "Files within partition " + partition + " should match"); + + // FileSystemView should expose the same data + List fileGroups = tableView.getAllFileGroups(partition).collect(Collectors.toList()); + + fileGroups.forEach(g -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(g)); + fileGroups.forEach(g -> g.getAllBaseFiles().forEach(b -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(b))); + fileGroups.forEach(g -> g.getAllFileSlices().forEach(s -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(s))); + + long numFiles = fileGroups.stream() + .mapToLong(g -> g.getAllBaseFiles().count() + g.getAllFileSlices().mapToLong(s -> s.getLogFiles().count()).sum()) + .sum(); + assertEquals(metadataFilenames.size(), numFiles); + } catch (IOException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + assertTrue(false, "Exception should not be raised: " + e); + } + }); + + HoodieBackedTableMetadataWriter metadataWriter = metadataWriter(client); + assertNotNull(metadataWriter, "MetadataWriter should have been initialized"); + + // Validate write config for metadata table + HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig(); + assertFalse(metadataWriteConfig.useFileListingMetadata(), "No metadata table for metadata table"); + assertFalse(metadataWriteConfig.getFileListingMetadataVerify(), "No verify for metadata table"); + + // Metadata table should be in sync with the dataset + assertTrue(metadata(client).isInSync()); + HoodieTableMetaClient metadataMetaClient = new HoodieTableMetaClient(hadoopConf, metadataTableBasePath); + + // Metadata table is MOR + assertEquals(metadataMetaClient.getTableType(), HoodieTableType.MERGE_ON_READ, "Metadata Table should be MOR"); + + // Metadata table is HFile format + assertEquals(metadataMetaClient.getTableConfig().getBaseFileFormat(), HoodieFileFormat.HFILE, + "Metadata Table base file format should be HFile"); + + // 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(fs, HoodieTableMetadata.getMetadataTableBasePath(basePath), + false, false, false); + assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); + + // Metadata table should automatically compact and clean + // versions are +1 as autoclean / compaction happens end of commits + int numFileVersions = metadataWriteConfig.getCleanerFileVersionsRetained() + 1; + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline()); + metadataTablePartitions.forEach(partition -> { + List latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList()); + assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count() <= 1, "Should have a single latest base file"); + assertTrue(latestSlices.size() <= 1, "Should have a single latest file slice"); + assertTrue(latestSlices.size() <= numFileVersions, "Should limit file slice to " + + numFileVersions + " but was " + latestSlices.size()); + }); + + LOG.info("Validation time=" + timer.endTimer()); + } + + private HoodieBackedTableMetadataWriter metadataWriter(SparkRDDWriteClient client) { + return (HoodieBackedTableMetadataWriter) SparkHoodieBackedTableMetadataWriter + .create(hadoopConf, client.getConfig(), new HoodieSparkEngineContext(jsc)); + } + + private HoodieBackedTableMetadata metadata(SparkRDDWriteClient client) { + HoodieWriteConfig clientConfig = client.getConfig(); + return (HoodieBackedTableMetadata) HoodieTableMetadata.create(hadoopConf, clientConfig.getBasePath(), clientConfig.getSpillableMapBasePath(), + clientConfig.useFileListingMetadata(), clientConfig.getFileListingMetadataVerify(), false, clientConfig.shouldAssumeDatePartitioning()); + } + + // TODO: this can be moved to TestHarness after merge from master + private void assertNoWriteErrors(List statuses) { + // Verify there are no errors + for (WriteStatus status : statuses) { + assertFalse(status.hasErrors(), "Errors found in write of " + status.getFileId()); + } + } + + private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) { + return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build(); + } + + private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) + .withAutoCommit(autoCommit).withAssumeDatePartitioning(false) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) + .withEmbeddedTimelineServerEnabled(false).forTable("test-trip-table") + .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() + .withEnableBackupForRemoteFileSystemView(false).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(useFileListingMetadata).build()) + .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) + .withExecutorMetrics(true).usePrefix("unit-test").build()); + } + + @Override + protected HoodieTableType getTableType() { + return tableType; + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index b8e02b905b8cb..6a292f5f1f53f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -46,6 +46,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -91,6 +92,11 @@ public void setUp() throws Exception { initDFSMetaClient(); } + @AfterEach + public void cleanUp() throws Exception { + cleanupResources(); + } + @Test public void testLeftOverUpdatedPropFileCleanup() throws IOException { testUpgradeInternal(true, true, HoodieTableType.MERGE_ON_READ); 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 9fa1c47e814a7..e6523af462a78 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 @@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -204,6 +205,10 @@ protected void cleanupFileSystem() throws IOException { * @throws IOException */ protected void initMetaClient() throws IOException { + initMetaClient(getTableType()); + } + + protected void initMetaClient(HoodieTableType tableType) throws IOException { if (basePath == null) { throw new IllegalStateException("The base path has not been initialized."); } @@ -212,7 +217,7 @@ protected void initMetaClient() throws IOException { throw new IllegalStateException("The Spark context has not been initialized."); } - metaClient = HoodieTestUtils.init(context.getHadoopConf().get(), basePath, getTableType()); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); } /** diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 4fbc63da7c644..f1e8e90621b5d 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -89,6 +89,7 @@ ${basedir}/src/main/avro/HoodieClusteringStrategy.avsc ${basedir}/src/main/avro/HoodieClusteringPlan.avsc ${basedir}/src/main/avro/HoodieRequestedReplaceMetadata.avsc + ${basedir}/src/main/avro/HoodieMetadata.avsc diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc new file mode 100644 index 0000000000000..bf85587a3a7ac --- /dev/null +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -0,0 +1,56 @@ +/* + * 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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieMetadataRecord", + "doc": "A record saved within the Metadata Table", + "fields": [ + { + "name": "key", + "type": "string" + }, + { + "name": "type", + "doc": "Type of the metadata record", + "type": "int" + }, + { "name": "filesystemMetadata", + "doc": "Contains information about partitions and files within the dataset", + "type": ["null", { + "type": "map", + "values": { + "type": "record", + "name": "HoodieMetadataFileInfo", + "fields": [ + { + "name": "size", + "type": "long", + "doc": "Size of the file" + }, + { + "name": "isDeleted", + "type": "boolean", + "doc": "True if this file has been deleted" + } + ] + } + }] + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc b/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc index a6bd4c20ef278..069881ea113c2 100644 --- a/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc @@ -30,7 +30,15 @@ "fields": [ {"name": "partitionPath", "type": "string"}, {"name": "successDeleteFiles", "type": {"type": "array", "items": "string"}}, - {"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}} + {"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}}, + {"name": "appendFiles", "type": { + "type": "map", + "doc": "Files to which append blocks were written", + "values": { + "type": "long", + "doc": "Size of this file in bytes" + } + }} ] } } 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 new file mode 100644 index 0000000000000..02e67e19f6373 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.config; + +import javax.annotation.concurrent.Immutable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Configurations used by the HUDI Metadata Table. + */ +@Immutable +public final class HoodieMetadataConfig extends DefaultHoodieConfig { + + public static final String METADATA_PREFIX = "hoodie.metadata"; + + // Enable the internal Metadata Table which saves file listings + public static final String METADATA_ENABLE_PROP = METADATA_PREFIX + ".enable"; + public static final boolean DEFAULT_METADATA_ENABLE = false; + + // Validate contents of Metadata Table on each access against the actual filesystem + public static final String METADATA_VALIDATE_PROP = METADATA_PREFIX + ".validate"; + public static final boolean DEFAULT_METADATA_VALIDATE = false; + + // Parallelism for inserts + public static final String METADATA_INSERT_PARALLELISM_PROP = METADATA_PREFIX + ".insert.parallelism"; + public static final int DEFAULT_METADATA_INSERT_PARALLELISM = 1; + + // Async clean + public static final String METADATA_ASYNC_CLEAN_PROP = METADATA_PREFIX + ".clean.async"; + public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = false; + + // Maximum delta commits before compaction occurs + public static final String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = METADATA_PREFIX + ".compact.max.delta.commits"; + public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = 24; + + // Archival settings + public static final String MIN_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.min.commits"; + public static final int DEFAULT_MIN_COMMITS_TO_KEEP = 20; + public static final String MAX_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.max.commits"; + public static final int DEFAULT_MAX_COMMITS_TO_KEEP = 30; + + // Cleaner commits retained + public static final String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained"; + public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3; + + // We can set the default to true for readers, as it will internally default to listing from filesystem if metadata + // table is not found + public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = true; + + private HoodieMetadataConfig(Properties props) { + super(props); + } + + public static HoodieMetadataConfig.Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + + private final Properties props = new Properties(); + + public Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.props.load(reader); + return this; + } + } + + public Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public Builder enable(boolean enable) { + props.setProperty(METADATA_ENABLE_PROP, String.valueOf(enable)); + return this; + } + + public Builder validate(boolean validate) { + props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate)); + return this; + } + + public Builder withInsertParallelism(int parallelism) { + props.setProperty(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism)); + return this; + } + + public Builder withAsyncClean(boolean asyncClean) { + props.setProperty(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean)); + return this; + } + + public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) { + props.setProperty(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction)); + return this; + } + + public Builder archiveCommitsWith(int minToKeep, int maxToKeep) { + props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep)); + props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep)); + return this; + } + + public Builder retainCommits(int commitsRetained) { + props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained)); + return this; + } + + public HoodieMetadataConfig build() { + HoodieMetadataConfig config = new HoodieMetadataConfig(props); + setDefaultOnCondition(props, !props.containsKey(METADATA_ENABLE_PROP), METADATA_ENABLE_PROP, + String.valueOf(DEFAULT_METADATA_ENABLE)); + setDefaultOnCondition(props, !props.containsKey(METADATA_VALIDATE_PROP), METADATA_VALIDATE_PROP, + String.valueOf(DEFAULT_METADATA_VALIDATE)); + setDefaultOnCondition(props, !props.containsKey(METADATA_INSERT_PARALLELISM_PROP), METADATA_INSERT_PARALLELISM_PROP, + String.valueOf(DEFAULT_METADATA_INSERT_PARALLELISM)); + setDefaultOnCondition(props, !props.containsKey(METADATA_ASYNC_CLEAN_PROP), METADATA_ASYNC_CLEAN_PROP, + String.valueOf(DEFAULT_METADATA_ASYNC_CLEAN)); + setDefaultOnCondition(props, !props.containsKey(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP), + METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS)); + setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP, + String.valueOf(DEFAULT_CLEANER_COMMITS_RETAINED)); + setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP, + String.valueOf(DEFAULT_MAX_COMMITS_TO_KEEP)); + setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP, + String.valueOf(DEFAULT_MIN_COMMITS_TO_KEEP)); + + return config; + } + } +} 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 0ce557348b795..2d638b47e8703 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 @@ -29,6 +29,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.InvalidHoodiePathException; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -46,6 +47,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; import java.util.List; import java.util.Map.Entry; @@ -193,8 +196,17 @@ public static String getRelativePartitionPath(Path basePath, Path fullPartitionP /** * Obtain all the partition paths, that are present in this table, denoted by presence of * {@link HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}. + * + * If the basePathStr is a subdirectory of .hoodie folder then we assume that the partitions of an internal + * table (a hoodie table within the .hoodie directory) are to be obtained. + * + * @param fs FileSystem instance + * @param basePathStr base directory */ public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr) throws IOException { + // If the basePathStr is a folder within the .hoodie directory then we are listing partitions within an + // internal table. + final boolean isMetadataTable = HoodieTableMetadata.isMetadataTable(basePathStr); final Path basePath = new Path(basePathStr); final List partitions = new ArrayList<>(); processFiles(fs, basePathStr, (locatedFileStatus) -> { @@ -203,7 +215,7 @@ public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, Str partitions.add(getRelativePartitionPath(basePath, filePath.getParent())); } return true; - }, true); + }, !isMetadataTable); return partitions; } @@ -240,12 +252,14 @@ public static void processFiles(FileSystem fs, String basePathStr, Function getAllPartitionPaths(FileSystem fs, String basePathStr, boolean assumeDatePartitioning) - throws IOException { + public static List getAllPartitionPaths(FileSystem fs, String basePathStr, boolean useFileListingFromMetadata, boolean verifyListings, + boolean assumeDatePartitioning) throws IOException { if (assumeDatePartitioning) { return getAllPartitionFoldersThreeLevelsDown(fs, basePathStr); } else { - return getAllFoldersWithPartitionMetaFile(fs, basePathStr); + HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(fs.getConf(), basePathStr, "/tmp/", useFileListingFromMetadata, + verifyListings, false, false); + return tableMetadata.getAllPartitionPaths(); } } @@ -384,6 +398,20 @@ public static boolean isLogFile(Path logPath) { return matcher.find() && logPath.getName().contains(".log"); } + /** + * Get the names of all the base and log files in the given partition path. + */ + public static FileStatus[] getAllDataFilesInPartition(FileSystem fs, Path partitionPath) throws IOException { + final Set validFileExtensions = Arrays.stream(HoodieFileFormat.values()) + .map(HoodieFileFormat::getFileExtension).collect(Collectors.toCollection(HashSet::new)); + final String logFileExtension = HoodieFileFormat.HOODIE_LOG.getFileExtension(); + + return Arrays.stream(fs.listStatus(partitionPath, path -> { + String extension = FSUtils.getFileExtension(path.getName()); + return validFileExtensions.contains(extension) || path.getName().contains(logFileExtension); + })).filter(FileStatus::isFile).toArray(FileStatus[]::new); + } + /** * Get the latest log file written from the list of log files passed in. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java index c3f6189e8a97e..1faaad5337927 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java @@ -19,6 +19,8 @@ package org.apache.hudi.common.fs; import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -65,15 +67,56 @@ public class HoodieWrapperFileSystem extends FileSystem { public static final String HOODIE_SCHEME_PREFIX = "hoodie-"; - private enum MetricName { - create, rename, delete, listStatus, mkdirs, getFileStatus, globStatus, listFiles + protected enum MetricName { + create, rename, delete, listStatus, mkdirs, getFileStatus, globStatus, listFiles, read, write } + private static Registry METRICS_REGISTRY_DATA; + private static Registry METRICS_REGISTRY_META; + + public static void setMetricsRegistry(Registry registry, Registry registryMeta) { + METRICS_REGISTRY_DATA = registry; + METRICS_REGISTRY_META = registryMeta; + } + + private ConcurrentMap openStreams = new ConcurrentHashMap<>(); private FileSystem fileSystem; private URI uri; private ConsistencyGuard consistencyGuard = new NoOpConsistencyGuard(); - private Registry metricsRegistry = Registry.getRegistry(this.getClass().getSimpleName()); + + @FunctionalInterface + public interface CheckedFunction { + R get() throws IOException; + } + + private static Registry getMetricRegistryForPath(Path p) { + return ((p != null) && (p.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))) + ? METRICS_REGISTRY_META : METRICS_REGISTRY_DATA; + } + + protected static R executeFuncWithTimeMetrics(String metricName, Path p, CheckedFunction func) throws IOException { + HoodieTimer timer = new HoodieTimer().startTimer(); + R res = func.get(); + + Registry registry = getMetricRegistryForPath(p); + if (registry != null) { + registry.increment(metricName); + registry.add(metricName + ".totalDuration", timer.endTimer()); + } + + return res; + } + + protected static R executeFuncWithTimeAndByteMetrics(String metricName, Path p, long byteCount, + CheckedFunction func) throws IOException { + Registry registry = getMetricRegistryForPath(p); + if (registry != null) { + registry.add(metricName + ".totalBytes", byteCount); + } + + return executeFuncWithTimeMetrics(metricName, p, func); + } public HoodieWrapperFileSystem() {} @@ -140,16 +183,17 @@ public URI getUri() { @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return fileSystem.open(convertToDefaultPath(f), bufferSize); + return wrapInputStream(f, fileSystem.open(convertToDefaultPath(f), bufferSize)); } @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - final Path translatedPath = convertToDefaultPath(f); - return wrapOutputStream(f, - fileSystem.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, progress)); + short replication, long blockSize, Progressable progress) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + final Path translatedPath = convertToDefaultPath(f); + return wrapOutputStream(f, + fileSystem.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, progress)); + }); } private FSDataOutputStream wrapOutputStream(final Path path, FSDataOutputStream fsDataOutputStream) @@ -164,79 +208,97 @@ private FSDataOutputStream wrapOutputStream(final Path path, FSDataOutputStream return os; } + private FSDataInputStream wrapInputStream(final Path path, FSDataInputStream fsDataInputStream) throws IOException { + if (fsDataInputStream instanceof TimedFSDataInputStream) { + return fsDataInputStream; + } + return new TimedFSDataInputStream(path, fsDataInputStream); + } + @Override public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite)); + }); } @Override public FSDataOutputStream create(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f))); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f))); + }); } @Override public FSDataOutputStream create(Path f, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress)); + }); } @Override public FSDataOutputStream create(Path f, short replication) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication)); + }); } @Override public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, - Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, - fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress)); + Progressable progress) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, + fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress)); + }); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, - fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress)); + short replication, long blockSize, Progressable progress) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, + fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress)); + }); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, - short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, - blockSize, progress, checksumOpt)); + short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, + blockSize, progress, checksumOpt)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, - fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, + fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize)); + }); } @Override @@ -246,50 +308,53 @@ public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) @Override public boolean rename(Path src, Path dst) throws IOException { - this.metricsRegistry.increment(MetricName.rename.name()); - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(src)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + src + " to appear", e); - } - - boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst)); - - if (success) { + return executeFuncWithTimeMetrics(MetricName.rename.name(), src, () -> { try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst)); + consistencyGuard.waitTillFileAppears(convertToDefaultPath(src)); } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + dst + " to appear", e); + throw new HoodieException("Timed out waiting for " + src + " to appear", e); } - try { - consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + src + " to disappear", e); + boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst)); + + if (success) { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for " + dst + " to appear", e); + } + + try { + consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for " + src + " to disappear", e); + } } - } - return success; + return success; + }); } @Override public boolean delete(Path f, boolean recursive) throws IOException { - this.metricsRegistry.increment(MetricName.delete.name()); - boolean success = fileSystem.delete(convertToDefaultPath(f), recursive); - - if (success) { - try { - consistencyGuard.waitTillFileDisappears(f); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + f + " to disappear", e); + return executeFuncWithTimeMetrics(MetricName.delete.name(), f, () -> { + boolean success = fileSystem.delete(convertToDefaultPath(f), recursive); + + if (success) { + try { + consistencyGuard.waitTillFileDisappears(f); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for " + f + " to disappear", e); + } } - } - return success; + return success; + }); } @Override public FileStatus[] listStatus(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertToDefaultPath(f)); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), f, () -> { + return fileSystem.listStatus(convertToDefaultPath(f)); + }); } @Override @@ -304,27 +369,29 @@ public void setWorkingDirectory(Path newDir) { @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException { - this.metricsRegistry.increment(MetricName.mkdirs.name()); - boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission); - if (success) { - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + return executeFuncWithTimeMetrics(MetricName.mkdirs.name(), f, () -> { + boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission); + if (success) { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + } } - } - return success; + return success; + }); } @Override public FileStatus getFileStatus(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.getFileStatus.name()); - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); - } catch (TimeoutException e) { - // pass - } - return fileSystem.getFileStatus(convertToDefaultPath(f)); + return executeFuncWithTimeMetrics(MetricName.getFileStatus.name(), f, () -> { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); + } catch (TimeoutException e) { + // pass + } + return fileSystem.getFileStatus(convertToDefaultPath(f)); + }); } @Override @@ -389,12 +456,12 @@ public Path resolvePath(Path p) throws IOException { @Override public FSDataInputStream open(Path f) throws IOException { - return fileSystem.open(convertToDefaultPath(f)); + return wrapInputStream(f, fileSystem.open(convertToDefaultPath(f))); } @Override public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize, short replication, - long blockSize, Progressable progress) throws IOException { + long blockSize, Progressable progress) throws IOException { Path p = convertToDefaultPath(f); return wrapOutputStream(p, fileSystem.createNonRecursive(p, overwrite, bufferSize, replication, blockSize, progress)); @@ -402,7 +469,7 @@ public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int buff @Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { + short replication, long blockSize, Progressable progress) throws IOException { Path p = convertToDefaultPath(f); return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, overwrite, bufferSize, replication, blockSize, progress)); @@ -410,7 +477,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, bo @Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet flags, - int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { Path p = convertToDefaultPath(f); return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, flags, bufferSize, replication, blockSize, progress)); @@ -462,8 +529,9 @@ public boolean setReplication(Path src, short replication) throws IOException { @Override public boolean delete(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.delete.name()); - return delete(f, true); + return executeFuncWithTimeMetrics(MetricName.delete.name(), f, () -> { + return delete(f, true); + }); } @Override @@ -508,32 +576,37 @@ public RemoteIterator listCorruptFileBlocks(Path path) throws IOException @Override public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertToDefaultPath(f), filter); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), f, () -> { + return fileSystem.listStatus(convertToDefaultPath(f), filter); + }); } @Override public FileStatus[] listStatus(Path[] files) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertDefaults(files)); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), files.length > 0 ? files[0] : null, () -> { + return fileSystem.listStatus(convertDefaults(files)); + }); } @Override public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertDefaults(files), filter); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), files.length > 0 ? files[0] : null, () -> { + return fileSystem.listStatus(convertDefaults(files), filter); + }); } @Override public FileStatus[] globStatus(Path pathPattern) throws IOException { - this.metricsRegistry.increment(MetricName.globStatus.name()); - return fileSystem.globStatus(convertToDefaultPath(pathPattern)); + return executeFuncWithTimeMetrics(MetricName.globStatus.name(), pathPattern, () -> { + return fileSystem.globStatus(convertToDefaultPath(pathPattern)); + }); } @Override public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { - this.metricsRegistry.increment(MetricName.globStatus.name()); - return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter); + return executeFuncWithTimeMetrics(MetricName.globStatus.name(), pathPattern, () -> { + return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter); + }); } @Override @@ -543,8 +616,9 @@ public RemoteIterator listLocatedStatus(Path f) throws IOExce @Override public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { - this.metricsRegistry.increment(MetricName.listFiles.name()); - return fileSystem.listFiles(convertToDefaultPath(f), recursive); + return executeFuncWithTimeMetrics(MetricName.listFiles.name(), f, () -> { + return fileSystem.listFiles(convertToDefaultPath(f), recursive); + }); } @Override @@ -554,16 +628,17 @@ public Path getHomeDirectory() { @Override public boolean mkdirs(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.mkdirs.name()); - boolean success = fileSystem.mkdirs(convertToDefaultPath(f)); - if (success) { - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + return executeFuncWithTimeMetrics(MetricName.mkdirs.name(), f, () -> { + boolean success = fileSystem.mkdirs(convertToDefaultPath(f)); + if (success) { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + } } - } - return success; + return success; + }); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java index 0b70bedc0988c..6869be80d5742 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java @@ -43,7 +43,7 @@ public class SizeAwareFSDataOutputStream extends FSDataOutputStream { private final ConsistencyGuard consistencyGuard; public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out, ConsistencyGuard consistencyGuard, - Runnable closeCallback) throws IOException { + Runnable closeCallback) throws IOException { super(out, null); this.path = path; this.closeCallback = closeCallback; @@ -52,14 +52,22 @@ public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out, Consistenc @Override public synchronized void write(byte[] b, int off, int len) throws IOException { - bytesWritten.addAndGet(len); - super.write(b, off, len); + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.write.name(), path, + len, () -> { + bytesWritten.addAndGet(len); + super.write(b, off, len); + return null; + }); } @Override public void write(byte[] b) throws IOException { - bytesWritten.addAndGet(b.length); - super.write(b); + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.write.name(), path, + b.length, () -> { + bytesWritten.addAndGet(b.length); + super.write(b); + return null; + }); } @Override @@ -76,5 +84,4 @@ public void close() throws IOException { public long getBytesWritten() { return bytesWritten.get(); } - } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/TimedFSDataInputStream.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/TimedFSDataInputStream.java new file mode 100644 index 0000000000000..eca8ec368b869 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/TimedFSDataInputStream.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.fs; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.ReadOption; +import org.apache.hadoop.io.ByteBufferPool; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.EnumSet; + +/** + * Wrapper over FSDataInputStream that also times the operations. + */ +public class TimedFSDataInputStream extends FSDataInputStream { + + // Path + private final Path path; + + public TimedFSDataInputStream(Path path, FSDataInputStream in) { + super(in); + this.path = path; + } + + @Override + public int read(ByteBuffer buf) throws IOException { + return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, 0, () -> super.read(buf)); + } + + @Override + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, length, () -> super.read(position, buffer, offset, length)); + } + + @Override + public ByteBuffer read(ByteBufferPool bufferPool, int maxLength, EnumSet opts) + throws IOException, UnsupportedOperationException { + return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, maxLength, () -> super.read(bufferPool, maxLength, opts)); + } + + @Override + public void readFully(long position, byte[] buffer) throws IOException { + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, buffer.length, () -> { + super.readFully(position, buffer); + return null; + }); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, length, () -> { + super.readFully(position, buffer, offset, length); + return null; + }); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/metrics/LocalRegistry.java b/hudi-common/src/main/java/org/apache/hudi/common/metrics/LocalRegistry.java new file mode 100644 index 0000000000000..4fdf9f60dddea --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/metrics/LocalRegistry.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.metrics; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Registry that tracks metrics local to a single jvm process. + */ +public class LocalRegistry implements Registry { + ConcurrentHashMap counters = new ConcurrentHashMap<>(); + private final String name; + + public LocalRegistry(String name) { + this.name = name; + } + + @Override + public void clear() { + counters.clear(); + } + + @Override + public void increment(String name) { + getCounter(name).increment(); + } + + @Override + public void add(String name, long value) { + getCounter(name).add(value); + } + + /** + * Get all Counter type metrics. + */ + @Override + public Map getAllCounts(boolean prefixWithRegistryName) { + HashMap countersMap = new HashMap<>(); + counters.forEach((k, v) -> { + String key = prefixWithRegistryName ? name + "." + k : k; + countersMap.put(key, v.getValue()); + }); + return countersMap; + } + + private synchronized Counter getCounter(String name) { + if (!counters.containsKey(name)) { + counters.put(name, new Counter()); + } + return counters.get(name); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java index 169e8bc9003ca..4ac1e61788972 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java @@ -18,87 +18,99 @@ package org.apache.hudi.common.metrics; +import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import org.apache.hudi.common.util.ReflectionUtils; + /** - * Lightweight Metrics Registry to track Hudi events. + * Interface which defines a lightweight Metrics Registry to track Hudi events. */ -public class Registry { - ConcurrentHashMap counters = new ConcurrentHashMap<>(); - final String name; +public interface Registry extends Serializable { - private static ConcurrentHashMap registryMap = new ConcurrentHashMap<>(); + ConcurrentHashMap REGISTRY_MAP = new ConcurrentHashMap<>(); - private Registry(String name) { - this.name = name; + /** + * Get (or create) the registry for a provided name. + * + * This function creates a {@code LocalRegistry}. + * + * @param registryName Name of the registry + */ + static Registry getRegistry(String registryName) { + return getRegistry(registryName, LocalRegistry.class.getName()); } /** - * Get (or create) the registry for a provided name. + * Get (or create) the registry for a provided name and given class. + * + * @param registryName Name of the registry. + * @param clazz The fully qualified name of the registry class to create. */ - public static synchronized Registry getRegistry(String registryName) { - if (!registryMap.containsKey(registryName)) { - registryMap.put(registryName, new Registry(registryName)); + static Registry getRegistry(String registryName, String clazz) { + synchronized (Registry.class) { + if (!REGISTRY_MAP.containsKey(registryName)) { + Registry registry = (Registry)ReflectionUtils.loadClass(clazz, registryName); + REGISTRY_MAP.put(registryName, registry); + } + return REGISTRY_MAP.get(registryName); } - return registryMap.get(registryName); } /** * Get all registered metrics. - * @param flush clean all metrics as part of this operation. + * + * @param flush clear all metrics after this operation. * @param prefixWithRegistryName prefix each metric name with the registry name. * @return */ - public static synchronized Map getAllMetrics(boolean flush, boolean prefixWithRegistryName) { - HashMap allMetrics = new HashMap<>(); - registryMap.forEach((registryName, registry) -> { - allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName)); - if (flush) { - registry.clear(); - } - }); - return allMetrics; - } - - public void clear() { - counters.clear(); + static Map getAllMetrics(boolean flush, boolean prefixWithRegistryName) { + synchronized (Registry.class) { + HashMap allMetrics = new HashMap<>(); + REGISTRY_MAP.forEach((registryName, registry) -> { + allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName)); + if (flush) { + registry.clear(); + } + }); + return allMetrics; + } } - public void increment(String name) { - getCounter(name).increment(); - } + /** + * Clear all metrics. + */ + void clear(); - public void add(String name, long value) { - getCounter(name).add(value); - } + /** + * Increment the metric. + * + * @param name Name of the metric to increment. + */ + void increment(String name); - private synchronized Counter getCounter(String name) { - if (!counters.containsKey(name)) { - counters.put(name, new Counter()); - } - return counters.get(name); - } + /** + * Add value to the metric. + * + * @param name Name of the metric. + * @param value The value to add to the metrics. + */ + void add(String name, long value); /** * Get all Counter type metrics. */ - public Map getAllCounts() { + default Map getAllCounts() { return getAllCounts(false); } /** * Get all Counter type metrics. + * + * @param prefixWithRegistryName If true, the names of all metrics are prefixed with name of this registry. */ - public Map getAllCounts(boolean prefixWithRegistryName) { - HashMap countersMap = new HashMap<>(); - counters.forEach((k, v) -> { - String key = prefixWithRegistryName ? name + "." + k : k; - countersMap.put(key, v.getValue()); - }); - return countersMap; - } - -} \ No newline at end of file + Map getAllCounts(boolean prefixWithRegistryName); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java index 4ae709eda1de7..6fb0a059094e2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java @@ -80,7 +80,7 @@ public abstract class AbstractHoodieLogRecordScanner { // Merge strategy to use when combining records from log private final String payloadClassFQN; // Log File Paths - private final List logFilePaths; + protected final List logFilePaths; // Read Lazily flag private final boolean readBlocksLazily; // Reverse reader - Not implemented yet (NA -> Why do we need ?) @@ -148,7 +148,8 @@ public void scan() { switch (r.getBlockType()) { case HFILE_DATA_BLOCK: case AVRO_DATA_BLOCK: - LOG.info("Reading a data block from file " + logFile.getPath()); + LOG.info("Reading a data block from file " + logFile.getPath() + " at instant " + + r.getLogBlockHeader().get(INSTANT_TIME)); if (isNewInstantBlock(r) && !readBlocksLazily) { // If this is an avro data block belonging to a different commit/instant, // then merge the last blocks and records into the main result @@ -202,8 +203,7 @@ public void scan() { LOG.info("Rolling back the last corrupted log block read in " + logFile.getPath()); currentInstantLogBlocks.pop(); numBlocksRolledBack++; - } else if (lastBlock.getBlockType() != CORRUPT_BLOCK - && targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { + } else if (targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { // rollback last data block or delete block LOG.info("Rolling back the last log block read in " + logFile.getPath()); currentInstantLogBlocks.pop(); @@ -278,12 +278,14 @@ private void processDataBlock(HoodieDataBlock dataBlock) throws Exception { List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); for (IndexedRecord rec : recs) { - HoodieRecord hoodieRecord = - SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); - processNextRecord(hoodieRecord); + processNextRecord(createHoodieRecord(rec)); } } + protected HoodieRecord createHoodieRecord(IndexedRecord rec) { + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); + } + /** * Process next record. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index 14d523ad9825e..5bd43ac4b7cf4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.log; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.TimedFSDataInputStream; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; @@ -73,8 +74,8 @@ public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSc boolean readBlockLazily, boolean reverseReader) throws IOException { FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize); if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) { - this.inputStream = new FSDataInputStream( - new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)); + this.inputStream = new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream( + new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize))); } else { // fsDataInputStream.getWrappedStream() maybe a BufferedFSInputStream // need to wrap in another BufferedFSInputStream the make bufferSize work? diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 18f2167b7e346..e99859e318129 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -57,39 +57,56 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner private static final Logger LOG = LogManager.getLogger(HoodieMergedLogRecordScanner.class); // Final map of compacted/merged records - private final ExternalSpillableMap> records; + protected final ExternalSpillableMap> records; // count of merged records in log private long numMergedRecordsInLog; + private long maxMemorySizeInBytes; // Stores the total time taken to perform reading and merging of log blocks - private final long totalTimeTakenToReadAndMergeBlocks; + private long totalTimeTakenToReadAndMergeBlocks; // A timer for calculating elapsed time in millis public final HoodieTimer timer = new HoodieTimer(); @SuppressWarnings("unchecked") public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, - String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, boolean reverseReader, - int bufferSize, String spillableMapBasePath) { + String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, + boolean reverseReader, int bufferSize, String spillableMapBasePath) { + this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, + reverseReader, bufferSize, spillableMapBasePath, true); + } + + @SuppressWarnings("unchecked") + public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, + String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, + boolean reverseReader, int bufferSize, String spillableMapBasePath, boolean autoScan) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize); try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema)); - // Do the scan and merge - timer.startTimer(); - scan(); - this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); - this.numMergedRecordsInLog = records.size(); - LOG.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); - LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); - LOG.info( - "Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); - LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); - LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } catch (IOException e) { - throw new HoodieIOException("IOException when reading log file ", e); + throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e); } + + if (autoScan) { + performScan(); + } + } + + protected void performScan() { + // Do the scan and merge + timer.startTimer(); + scan(); + this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); + this.numMergedRecordsInLog = records.size(); + LOG.info("Number of log files scanned => " + logFilePaths.size()); + LOG.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); + LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); + LOG.info( + "Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); + LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); + LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } @Override 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 61d9b7f233bee..6d2682a4ffa09 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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieHFileReader; import org.apache.log4j.LogManager; @@ -118,6 +119,8 @@ protected byte[] serializeRecords() throws IOException { recordKey = record.get(keyField.pos()).toString(); } byte[] recordBytes = HoodieAvroUtils.indexedRecordToBytes(record); + ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), + "Writing multiple records with same key not supported for " + this.getClass().getName()); sortedRecordsMap.put(recordKey, recordBytes); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 918d568b9faaf..fcb4fd9176dda 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -62,12 +62,16 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( - COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, - INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, - INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, - INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, - REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION)); - + COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, + DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, + SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, + CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, + INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, + INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, + ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION + )); + private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class); protected HoodieTableMetaClient metaClient; private static AtomicReference lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java index 640d4894feb69..32e60c30bd868 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java @@ -27,6 +27,8 @@ import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificDatumWriter; import org.apache.avro.specific.SpecificRecordBase; +import org.apache.hadoop.fs.FileStatus; + import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; @@ -68,8 +70,10 @@ public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbac Map partitionMetadataBuilder = new HashMap<>(); int totalDeleted = 0; for (HoodieRollbackStat stat : rollbackStats) { + Map appendFiles = stat.getCommandBlocksCount().keySet().stream() + .collect(Collectors.toMap(f -> f.getPath().toString(), FileStatus::getLen)); HoodieRollbackPartitionMetadata metadata = new HoodieRollbackPartitionMetadata(stat.getPartitionPath(), - stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles()); + stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles(), appendFiles); partitionMetadataBuilder.put(stat.getPartitionPath(), metadata); totalDeleted += stat.getSuccessDeleteFiles().size(); } @@ -146,6 +150,10 @@ public static HoodieRollbackMetadata deserializeHoodieRollbackMetadata(byte[] by return deserializeAvroMetadata(bytes, HoodieRollbackMetadata.class); } + public static HoodieRestoreMetadata deserializeHoodieRestoreMetadata(byte[] bytes) throws IOException { + return deserializeAvroMetadata(bytes, HoodieRestoreMetadata.class); + } + public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieSavepointMetadata.class); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 1dd6b006b7a30..65e9231652ef6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -276,7 +276,7 @@ private void ensurePartitionLoadedCorrectly(String partition) { Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr); FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); long beginLsTs = System.currentTimeMillis(); - FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); + FileStatus[] statuses = listPartition(partitionPath); long endLsTs = System.currentTimeMillis(); LOG.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken =" + (endLsTs - beginLsTs)); @@ -297,6 +297,16 @@ private void ensurePartitionLoadedCorrectly(String partition) { }); } + /** + * Return all the files from the partition. + * + * @param partitionPath The absolute path of the partition + * @throws IOException + */ + protected FileStatus[] listPartition(Path partitionPath) throws IOException { + return metaClient.getFs().listStatus(partitionPath); + } + /** * Helper to convert file-status to base-files. * 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 d31018123c3a5..c5a31fa3adb0e 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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Functions.Function2; +import org.apache.hudi.metadata.HoodieMetadataFileSystemView; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -158,6 +159,22 @@ private static HoodieTableFileSystemView createInMemoryFileSystemView(Serializab return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled()); } + public static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieTableMetaClient metaClient, + boolean useFileListingFromMetadata, + boolean verifyListings) { + LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath()); + if (useFileListingFromMetadata) { + return new HoodieMetadataFileSystemView(metaClient, + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), + true, + verifyListings); + } + + return new HoodieTableFileSystemView(metaClient, + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); + } + + /** * Create a remote file System view for a table. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieMetadataException.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieMetadataException.java new file mode 100644 index 0000000000000..132a9f804edf2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieMetadataException.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +/** + *

+ * Exception thrown for table metadata related failures. + *

+ */ +public class HoodieMetadataException extends HoodieException { + public HoodieMetadataException(String msg, Exception t) { + super(msg, t); + } + + public HoodieMetadataException(String msg) { + super(msg); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java b/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java index 7666e90a74f90..2febe51f7710c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java +++ b/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java @@ -18,9 +18,11 @@ package org.apache.hudi.exception; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import java.io.FileNotFoundException; import java.io.IOException; /** @@ -39,16 +41,14 @@ private static String getErrorMessage(String basePath) { } public static void checkTableValidity(FileSystem fs, Path basePathDir, Path metaPathDir) { - // Check if the base path is found + // Check if the base and meta paths are found try { - if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) { - throw new TableNotFoundException(basePathDir.toString()); - } - // Check if the meta path is found - if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) { + // Since metaPath is within the basePath, it is enough to check the metaPath exists + FileStatus status = fs.getFileStatus(metaPathDir); + if (!status.isDirectory()) { throw new TableNotFoundException(metaPathDir.toString()); } - } catch (IllegalArgumentException e) { + } catch (FileNotFoundException | IllegalArgumentException e) { // if the base path is file:///, then we have a IllegalArgumentException throw new TableNotFoundException(metaPathDir.toString()); } catch (IOException e) { 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 new file mode 100644 index 0000000000000..33371da0b3358 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -0,0 +1,293 @@ + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public abstract class BaseTableMetadata implements HoodieTableMetadata { + + private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class); + + static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; + static final int BUFFER_SIZE = 10 * 1024 * 1024; + + protected final SerializableConfiguration hadoopConf; + protected final String datasetBasePath; + protected boolean enabled; + protected final Option metrics; + + private final boolean validateLookups; + private final boolean assumeDatePartitioning; + + // Directory used for Spillable Map when merging records + protected final String spillableMapDirectory; + private transient HoodieMetadataMergedInstantRecordScanner timelineRecordScanner; + + protected BaseTableMetadata(Configuration hadoopConf, String datasetBasePath, String spillableMapDirectory, + boolean enabled, boolean validateLookups, boolean enableMetrics, + boolean assumeDatePartitioning) { + this.hadoopConf = new SerializableConfiguration(hadoopConf); + this.datasetBasePath = datasetBasePath; + this.spillableMapDirectory = spillableMapDirectory; + + this.enabled = enabled; + this.validateLookups = validateLookups; + this.assumeDatePartitioning = assumeDatePartitioning; + + if (enableMetrics) { + this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata"))); + } else { + this.metrics = Option.empty(); + } + } + + /** + * Return the list of partitions in the dataset. + * + * If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of + * partitions is retrieved directly from the underlying {@code FileSystem}. + * + * On any errors retrieving the listing from the metadata, defaults to using the file system listings. + * + */ + @Override + public List getAllPartitionPaths() throws IOException { + if (enabled) { + try { + return fetchAllPartitionPaths(); + } catch (Exception e) { + LOG.error("Failed to retrieve list of partition from metadata", e); + } + } + return new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning).getAllPartitionPaths(); + } + + /** + * Return the list of files in a partition. + * + * If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of + * partitions is retrieved directly from the underlying {@code FileSystem}. + * + * On any errors retrieving the listing from the metadata, defaults to using the file system listings. + * + * @param partitionPath The absolute path of the partition to list + */ + @Override + public FileStatus[] getAllFilesInPartition(Path partitionPath) + throws IOException { + if (enabled) { + try { + return fetchAllFilesInPartition(partitionPath); + } catch (Exception e) { + LOG.error("Failed to retrieve files in partition " + partitionPath + " from metadata", e); + } + } + + return FSUtils.getFs(partitionPath.toString(), hadoopConf.get()).listStatus(partitionPath); + } + + /** + * Returns a list of all partitions. + */ + protected List fetchAllPartitionPaths() throws IOException { + HoodieTimer timer = new HoodieTimer().startTimer(); + Option> hoodieRecord = getMergedRecordByKey(RECORDKEY_PARTITION_LIST); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer())); + + List partitions = Collections.emptyList(); + if (hoodieRecord.isPresent()) { + if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { + throw new HoodieMetadataException("Metadata partition list record is inconsistent: " + + hoodieRecord.get().getData()); + } + + partitions = hoodieRecord.get().getData().getFilenames(); + // Partition-less tables have a single empty partition + if (partitions.contains(NON_PARTITIONED_NAME)) { + partitions.remove(NON_PARTITIONED_NAME); + partitions.add(""); + } + } + + if (validateLookups) { + // Validate the Metadata Table data by listing the partitions from the file system + timer.startTimer(); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning); + List actualPartitions = fileSystemBackedTableMetadata.getAllPartitionPaths(); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_PARTITIONS_STR, timer.endTimer())); + + Collections.sort(actualPartitions); + Collections.sort(partitions); + if (!actualPartitions.equals(partitions)) { + LOG.error("Validation of metadata partition list failed. Lists do not match."); + LOG.error("Partitions from metadata: " + Arrays.toString(partitions.toArray())); + LOG.error("Partitions from file system: " + Arrays.toString(actualPartitions.toArray())); + + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0)); + } + + // Return the direct listing as it should be correct + partitions = actualPartitions; + } + + LOG.info("Listed partitions from metadata: #partitions=" + partitions.size()); + return partitions; + } + + /** + * Return all the files from the partition. + * + * @param partitionPath The absolute path of the partition + */ + FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { + String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), partitionPath); + if (partitionName.isEmpty()) { + partitionName = NON_PARTITIONED_NAME; + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + Option> hoodieRecord = getMergedRecordByKey(partitionName); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); + + FileStatus[] statuses = {}; + if (hoodieRecord.isPresent()) { + if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { + throw new HoodieMetadataException("Metadata record for partition " + partitionName + " is inconsistent: " + + hoodieRecord.get().getData()); + } + statuses = hoodieRecord.get().getData().getFileStatuses(partitionPath); + } + + if (validateLookups) { + // Validate the Metadata Table data by listing the partitions from the file system + timer.startTimer(); + + // Ignore partition metadata file + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath); + FileStatus[] directStatuses = metaClient.getFs().listStatus(partitionPath, + p -> !p.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer())); + + List directFilenames = Arrays.stream(directStatuses) + .map(s -> s.getPath().getName()).sorted() + .collect(Collectors.toList()); + + List metadataFilenames = Arrays.stream(statuses) + .map(s -> s.getPath().getName()).sorted() + .collect(Collectors.toList()); + + if (!metadataFilenames.equals(directFilenames)) { + LOG.error("Validation of metadata file listing for partition " + partitionName + " failed."); + LOG.error("File list from metadata: " + Arrays.toString(metadataFilenames.toArray())); + LOG.error("File list from direct listing: " + Arrays.toString(directFilenames.toArray())); + + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0)); + } + + // Return the direct listing as it should be correct + statuses = directStatuses; + } + + LOG.info("Listed file in partition from metadata: partition=" + partitionName + ", #files=" + statuses.length); + return statuses; + } + + /** + * Retrieve the merged {@code HoodieRecord} mapped to the given key. + * + * @param key The key of the record + */ + private Option> getMergedRecordByKey(String key) throws IOException { + + Option> mergedRecord; + openTimelineScanner(); + + Option> metadataHoodieRecord = getRecordByKeyFromMetadata(key); + // Retrieve record from unsynced timeline instants + Option> timelineHoodieRecord = timelineRecordScanner.getRecordByKey(key); + if (timelineHoodieRecord.isPresent()) { + if (metadataHoodieRecord.isPresent()) { + HoodieRecordPayload mergedPayload = timelineHoodieRecord.get().getData().preCombine(metadataHoodieRecord.get().getData()); + mergedRecord = Option.of(new HoodieRecord(metadataHoodieRecord.get().getKey(), mergedPayload)); + } else { + mergedRecord = timelineHoodieRecord; + } + } else { + mergedRecord = metadataHoodieRecord; + } + return mergedRecord; + } + + protected abstract Option> getRecordByKeyFromMetadata(String key) throws IOException; + + private void openTimelineScanner() throws IOException { + if (timelineRecordScanner != null) { + // Already opened + return; + } + + HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath); + List unSyncedInstants = findInstantsToSync(datasetMetaClient); + Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + timelineRecordScanner = + new HoodieMetadataMergedInstantRecordScanner(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), schema, MAX_MEMORY_SIZE_IN_BYTES, spillableMapDirectory, null); + } + + protected List findInstantsToSync() { + HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath); + return findInstantsToSync(datasetMetaClient); + } + + protected abstract List findInstantsToSync(HoodieTableMetaClient datasetMetaClient); + + public boolean isInSync() { + return enabled && findInstantsToSync().isEmpty(); + } + + protected void closeReaders() { + timelineRecordScanner = null; + } +} 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 new file mode 100644 index 0000000000000..73ce8e487edd5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.Option; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.List; + +public class FileSystemBackedTableMetadata implements HoodieTableMetadata { + + private final SerializableConfiguration hadoopConf; + private final String datasetBasePath; + private final boolean assumeDatePartitioning; + + public FileSystemBackedTableMetadata(SerializableConfiguration conf, String datasetBasePath, boolean assumeDatePartitioning) { + this.hadoopConf = conf; + this.datasetBasePath = datasetBasePath; + this.assumeDatePartitioning = assumeDatePartitioning; + } + + @Override + public FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException { + FileSystem fs = partitionPath.getFileSystem(hadoopConf.get()); + return FSUtils.getAllDataFilesInPartition(fs, partitionPath); + } + + @Override + public List getAllPartitionPaths() throws IOException { + FileSystem fs = new Path(datasetBasePath).getFileSystem(hadoopConf.get()); + if (assumeDatePartitioning) { + return FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, datasetBasePath); + } else { + return FSUtils.getAllFoldersWithPartitionMetaFile(fs, datasetBasePath); + } + } + + @Override + public Option getSyncedInstantTime() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isInSync() { + throw new UnsupportedOperationException(); + } +} 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 new file mode 100644 index 0000000000000..65c3244f9a771 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +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.ValidationUtils; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Table metadata provided by an internal DFS backed Hudi metadata table. + * + * If the metadata table does not exist, RPC calls are used to retrieve file listings from the file system. + * No updates are applied to the table and it is not synced. + */ +public class HoodieBackedTableMetadata extends BaseTableMetadata { + + private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadata.class); + + private final String metadataBasePath; + private HoodieTableMetaClient metaClient; + + // Readers for the base and log file which store the metadata + private transient HoodieFileReader baseFileReader; + private transient HoodieMetadataMergedLogRecordScanner logRecordScanner; + + public HoodieBackedTableMetadata(Configuration conf, String datasetBasePath, String spillableMapDirectory, + boolean enabled, boolean validateLookups, boolean assumeDatePartitioning) { + this(conf, datasetBasePath, spillableMapDirectory, enabled, validateLookups, false, assumeDatePartitioning); + } + + public HoodieBackedTableMetadata(Configuration conf, String datasetBasePath, String spillableMapDirectory, + boolean enabled, boolean validateLookups, boolean enableMetrics, + boolean assumeDatePartitioning) { + super(conf, datasetBasePath, spillableMapDirectory, enabled, validateLookups, enableMetrics, assumeDatePartitioning); + this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath); + if (enabled) { + try { + this.metaClient = new HoodieTableMetaClient(hadoopConf.get(), metadataBasePath); + } catch (TableNotFoundException e) { + LOG.warn("Metadata table was not found at path " + metadataBasePath); + this.enabled = false; + } catch (Exception e) { + LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e); + this.enabled = false; + } + } else { + LOG.info("Metadata table is disabled."); + } + } + + @Override + protected Option> getRecordByKeyFromMetadata(String key) throws IOException { + openBaseAndLogFiles(); + + // Retrieve record from base file + HoodieRecord hoodieRecord = null; + if (baseFileReader != null) { + HoodieTimer timer = new HoodieTimer().startTimer(); + Option baseRecord = baseFileReader.getRecordByKey(key); + if (baseRecord.isPresent()) { + hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + metaClient.getTableConfig().getPayloadClass()); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, timer.endTimer())); + } + } + + // Retrieve record from log file + Option> logHoodieRecord = logRecordScanner.getRecordByKey(key); + if (logHoodieRecord.isPresent()) { + if (hoodieRecord != null) { + // Merge the payloads + HoodieRecordPayload mergedPayload = logHoodieRecord.get().getData().preCombine(hoodieRecord.getData()); + hoodieRecord = new HoodieRecord(hoodieRecord.getKey(), mergedPayload); + } else { + hoodieRecord = logHoodieRecord.get(); + } + } + + return Option.ofNullable(hoodieRecord); + } + + /** + * Open readers to the base and log files. + */ + protected synchronized void openBaseAndLogFiles() throws IOException { + if (logRecordScanner != null) { + // Already opened + return; + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + + // Metadata is in sync till the latest completed instant on the dataset + HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath); + String latestInstantTime = datasetMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() + .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + + // Find the latest file slice + HoodieTimeline timeline = metaClient.reloadActiveTimeline(); + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); + List latestSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + ValidationUtils.checkArgument(latestSlices.size() == 1); + + // If the base file is present then create a reader + Option basefile = latestSlices.get(0).getBaseFile(); + if (basefile.isPresent()) { + String basefilePath = basefile.get().getPath(); + baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); + LOG.info("Opened metadata base file from " + basefilePath + " at instant " + basefile.get().getCommitTime()); + } + + // Open the log record scanner using the log files from the latest file slice + List logFilePaths = latestSlices.get(0).getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) + .map(o -> o.getPath().toString()) + .collect(Collectors.toList()); + + Option lastInstant = timeline.filterCompletedInstants().lastInstant(); + String latestMetaInstantTimestamp = lastInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + + // Load the schema + Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + + logRecordScanner = + new HoodieMetadataMergedLogRecordScanner(metaClient.getFs(), metadataBasePath, + logFilePaths, schema, latestMetaInstantTimestamp, MAX_MEMORY_SIZE_IN_BYTES, BUFFER_SIZE, + spillableMapDirectory, null); + + LOG.info("Opened metadata log files from " + logFilePaths + " at instant " + latestInstantTime + + "(dataset instant=" + latestInstantTime + ", metadata instant=" + latestMetaInstantTimestamp + ")"); + + metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, timer.endTimer())); + } + + public void closeReaders() { + if (baseFileReader != null) { + baseFileReader.close(); + baseFileReader = null; + } + logRecordScanner = null; + } + + /** + * Return an ordered list of instants which have not been synced to the Metadata Table. + * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + */ + protected List findInstantsToSync(HoodieTableMetaClient datasetMetaClient) { + HoodieActiveTimeline metaTimeline = metaClient.reloadActiveTimeline(); + + // All instants on the data timeline, which are greater than the last instant on metadata timeline + // are candidates for sync. + Option latestMetadataInstant = metaTimeline.filterCompletedInstants().lastInstant(); + ValidationUtils.checkArgument(latestMetadataInstant.isPresent(), + "At least one completed instant should exist on the metadata table, before syncing."); + String latestMetadataInstantTime = latestMetadataInstant.get().getTimestamp(); + HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE); + Option earliestIncompleteInstant = candidateTimeline.filterInflightsAndRequested().firstInstant(); + + if (earliestIncompleteInstant.isPresent()) { + return candidateTimeline.filterCompletedInstants() + .findInstantsBefore(earliestIncompleteInstant.get().getTimestamp()) + .getInstants().collect(Collectors.toList()); + } else { + return candidateTimeline.filterCompletedInstants() + .getInstants().collect(Collectors.toList()); + } + } + + /** + * Return the timestamp of the latest compaction instant. + */ + @Override + public Option getSyncedInstantTime() { + if (!enabled) { + return Option.empty(); + } + + HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); + return timeline.getDeltaCommitTimeline().filterCompletedInstants() + .lastInstant().map(HoodieInstant::getTimestamp); + } + + public boolean enabled() { + return enabled; + } + + public SerializableConfiguration getHadoopConf() { + return hadoopConf; + } + + public String getDatasetBasePath() { + return datasetBasePath; + } + + public HoodieTableMetaClient getMetaClient() { + return metaClient; + } + + public Map stats() { + return metrics.map(m -> m.getStats(true, metaClient, this)).orElse(new HashMap<>()); + } +} 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 new file mode 100644 index 0000000000000..9f9e4054b03b9 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import java.io.IOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +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; + +/** + * {@code HoodieTableFileSystemView} implementation that retrieved partition listings from the Metadata Table. + */ +public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView { + + private final HoodieTableMetadata tableMetadata; + + public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient, HoodieTableMetadata tableMetadata, + HoodieTimeline visibleActiveTimeline, boolean enableIncrementalTimelineSync) { + super(metaClient, visibleActiveTimeline, enableIncrementalTimelineSync); + this.tableMetadata = tableMetadata; + } + + public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline, + boolean useFileListingFromMetadata, + boolean verifyListings) { + super(metaClient, visibleActiveTimeline); + this.tableMetadata = HoodieTableMetadata.create(metaClient.getHadoopConf(), metaClient.getBasePath(), + FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR, useFileListingFromMetadata, verifyListings, + false, false); + } + + /** + * Return all the files in the partition by reading from the Metadata Table. + * + * @param partitionPath The absolute path of the partition + * @throws IOException + */ + @Override + protected FileStatus[] listPartition(Path partitionPath) throws IOException { + return tableMetadata.getAllFilesInPartition(partitionPath); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedInstantRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedInstantRecordScanner.java new file mode 100644 index 0000000000000..1dcd322b2d11b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedInstantRecordScanner.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.avro.Schema; +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.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.DefaultSizeEstimator; +import org.apache.hudi.common.util.HoodieRecordSizeEstimator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Provides functionality to convert timeline instants to table metadata records and then merge by key. Specify + * a filter to limit keys that are merged and stored in memory. + */ +public class HoodieMetadataMergedInstantRecordScanner { + + private static final Logger LOG = LogManager.getLogger(HoodieMetadataMergedInstantRecordScanner.class); + + HoodieTableMetaClient metaClient; + private List instants; + private Option lastSyncTs; + private Set mergeKeyFilter; + protected final ExternalSpillableMap> records; + + public HoodieMetadataMergedInstantRecordScanner(HoodieTableMetaClient metaClient, List instants, + Option lastSyncTs, Schema readerSchema, Long maxMemorySizeInBytes, + String spillableMapBasePath, Set mergeKeyFilter) throws IOException { + this.metaClient = metaClient; + this.instants = instants; + this.lastSyncTs = lastSyncTs; + this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet(); + this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(readerSchema)); + + scan(); + } + + /** + * Converts instants in scanner to metadata table records and processes each record. + * + * @param + * @throws IOException + */ + private void scan() { + for (HoodieInstant instant : instants) { + try { + Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(metaClient, instant, lastSyncTs); + if (records.isPresent()) { + records.get().forEach(record -> processNextRecord(record)); + } + } catch (Exception e) { + LOG.error(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e); + throw new HoodieException(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e); + } + } + } + + /** + * Process metadata table record by merging with existing record if it is a part of the key filter. + * + * @param hoodieRecord + */ + private void processNextRecord(HoodieRecord hoodieRecord) { + String key = hoodieRecord.getRecordKey(); + if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(key)) { + if (records.containsKey(key)) { + // Merge and store the merged record + HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(records.get(key).getData()); + records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); + } else { + // Put the record as is + records.put(key, hoodieRecord); + } + } + } + + /** + * Retrieve merged hoodie record for given key. + * + * @param key of the record to retrieve + * @return {@code HoodieRecord} if key was found else {@code Option.empty()} + */ + public Option> getRecordByKey(String key) { + return Option.ofNullable((HoodieRecord) records.get(key)); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java new file mode 100644 index 0000000000000..ae471dc52e799 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +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.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.Option; + +/** + * A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is + * useful in limiting memory usage when only a small subset of updates records are to be read. + */ +public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordScanner { + // Set of all record keys that are to be read in memory + private Set mergeKeyFilter; + + public HoodieMetadataMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize, + String spillableMapBasePath, Set mergeKeyFilter) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize, + spillableMapBasePath, false); + this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet(); + + performScan(); + } + + @Override + protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { + if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieRecord.getRecordKey())) { + super.processNextRecord(hoodieRecord); + } + } + + @Override + protected void processNextDeletedKey(HoodieKey hoodieKey) { + if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieKey.getRecordKey())) { + super.processNextDeletedKey(hoodieKey); + } + } + + /** + * Retrieve a record given its key. + * + * @param key Key of the record to retrieve + * @return {@code HoodieRecord} if key was found else {@code Option.empty()} + */ + public Option> getRecordByKey(String key) { + return Option.ofNullable((HoodieRecord) records.get(key)); + } +} 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 new file mode 100644 index 0000000000000..2bd773bc7819f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class HoodieMetadataMetrics implements Serializable { + + // Metric names + public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions"; + public static final String LOOKUP_FILES_STR = "lookup_files"; + public static final String VALIDATE_PARTITIONS_STR = "validate_partitions"; + public static final String VALIDATE_FILES_STR = "validate_files"; + public static final String VALIDATE_ERRORS_STR = "validate_errors"; + public static final String SCAN_STR = "scan"; + public static final String BASEFILE_READ_STR = "basefile_read"; + public static final String INITIALIZE_STR = "initialize"; + public static final String SYNC_STR = "sync"; + + // Stats names + public static final String STAT_TOTAL_BASE_FILE_SIZE = "totalBaseFileSizeInBytes"; + public static final String STAT_TOTAL_LOG_FILE_SIZE = "totalLogFileSizeInBytes"; + public static final String STAT_COUNT_BASE_FILES = "baseFileCount"; + public static final String STAT_COUNT_LOG_FILES = "logFileCount"; + public static final String STAT_COUNT_PARTITION = "partitionCount"; + public static final String STAT_IN_SYNC = "isInSync"; + public static final String STAT_LAST_COMPACTION_TIMESTAMP = "lastCompactionTimestamp"; + + private static final Logger LOG = LogManager.getLogger(HoodieMetadataMetrics.class); + + private final Registry metricsRegistry; + + public HoodieMetadataMetrics(Registry metricsRegistry) { + this.metricsRegistry = metricsRegistry; + } + + public Map getStats(boolean detailed, HoodieTableMetaClient metaClient, HoodieTableMetadata metadata) { + try { + metaClient.reloadActiveTimeline(); + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); + return getStats(fsView, detailed, metadata); + } catch (IOException ioe) { + throw new HoodieIOException("Unable to get metadata stats.", ioe); + } + } + + private Map getStats(HoodieTableFileSystemView fsView, boolean detailed, HoodieTableMetadata tableMetadata) throws IOException { + Map stats = new HashMap<>(); + + // Total size of the metadata and count of base/log files + long totalBaseFileSizeInBytes = 0; + long totalLogFileSizeInBytes = 0; + int baseFileCount = 0; + int logFileCount = 0; + List latestSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + + for (FileSlice slice : latestSlices) { + if (slice.getBaseFile().isPresent()) { + totalBaseFileSizeInBytes += slice.getBaseFile().get().getFileStatus().getLen(); + ++baseFileCount; + } + Iterator it = slice.getLogFiles().iterator(); + while (it.hasNext()) { + totalLogFileSizeInBytes += it.next().getFileStatus().getLen(); + ++logFileCount; + } + } + + stats.put(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes)); + stats.put(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes)); + stats.put(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount)); + stats.put(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES, String.valueOf(logFileCount)); + + if (detailed) { + stats.put(HoodieMetadataMetrics.STAT_COUNT_PARTITION, String.valueOf(tableMetadata.getAllPartitionPaths().size())); + stats.put(HoodieMetadataMetrics.STAT_IN_SYNC, String.valueOf(tableMetadata.isInSync())); + } + + return stats; + } + + protected void updateMetrics(String action, long durationInMs) { + if (metricsRegistry == null) { + return; + } + + // Update sum of duration and total for count + String countKey = action + ".count"; + String durationKey = action + ".totalDuration"; + metricsRegistry.add(countKey, 1); + metricsRegistry.add(durationKey, durationInMs); + + LOG.info(String.format("Updating metadata metrics (%s=%dms, %s=1)", durationKey, durationInMs, countKey)); + } + + public void updateMetrics(long totalBaseFileSizeInBytes, long totalLogFileSizeInBytes, int baseFileCount, + int logFileCount) { + if (metricsRegistry == null) { + return; + } + + // Update sizes and count for metadata table's data files + metricsRegistry.add("basefile.size", totalBaseFileSizeInBytes); + metricsRegistry.add("logfile.size", totalLogFileSizeInBytes); + metricsRegistry.add("basefile.count", baseFileCount); + metricsRegistry.add("logfile.count", logFileCount); + + LOG.info(String.format("Updating metadata size metrics (basefile.size=%d, logfile.size=%d, basefile.count=%d, " + + "logfile.count=%d)", totalBaseFileSizeInBytes, totalLogFileSizeInBytes, baseFileCount, logFileCount)); + } + + 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 new file mode 100644 index 0000000000000..0863f7ef80f5a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.avro.model.HoodieMetadataFileInfo; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST; + +/** + * This is a payload which saves information about a single entry in the Metadata Table. + * + * The type of the entry is determined by the "type" saved within the record. The following types of entries are saved: + * + * 1. List of partitions: There is a single such record + * key="__all_partitions__" + * + * 2. List of files in a Partition: There is one such record for each partition + * key=Partition name + * + * During compaction on the table, the deletions are merged with additions and hence pruned. + * + * Metadata Table records are saved with the schema defined in HoodieMetadata.avsc. This class encapsulates the + * HoodieMetadataRecord for ease of operations. + */ +public class HoodieMetadataPayload implements HoodieRecordPayload { + // Type of the record + // This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810 + private static final int PARTITION_LIST = 1; + private static final int FILE_LIST = 2; + + private String key = null; + private int type = 0; + private Map filesystemMetadata = null; + + public HoodieMetadataPayload(Option record) { + if (record.isPresent()) { + // This can be simplified using SpecificData.deepcopy once this bug is fixed + // https://issues.apache.org/jira/browse/AVRO-1811 + key = record.get().get("key").toString(); + type = (int) record.get().get("type"); + if (record.get().get("filesystemMetadata") != null) { + filesystemMetadata = (Map) record.get().get("filesystemMetadata"); + filesystemMetadata.keySet().forEach(k -> { + GenericRecord v = filesystemMetadata.get(k); + filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long)v.get("size"), (Boolean)v.get("isDeleted"))); + }); + } + } + } + + private HoodieMetadataPayload(String key, int type, Map filesystemMetadata) { + this.key = key; + this.type = type; + this.filesystemMetadata = filesystemMetadata; + } + + /** + * Create and return a {@code HoodieMetadataPayload} to save list of partitions. + * + * @param partitions The list of partitions + */ + public static HoodieRecord createPartitionListRecord(List partitions) { + Map fileInfo = new HashMap<>(); + partitions.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false))); + + HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), PARTITION_LIST, fileInfo); + return new HoodieRecord<>(key, payload); + } + + /** + * Create and return a {@code HoodieMetadataPayload} to save list of files within a partition. + * + * @param partition The name of the partition + * @param filesAdded Mapping of files to their sizes for files which have been added to this partition + * @param filesDeleted List of files which have been deleted from this partition + */ + public static HoodieRecord createPartitionFilesRecord(String partition, + Option> filesAdded, Option> filesDeleted) { + Map fileInfo = new HashMap<>(); + filesAdded.ifPresent( + m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false)))); + filesDeleted.ifPresent( + m -> m.forEach(filename -> fileInfo.put(filename, new HoodieMetadataFileInfo(0L, true)))); + + HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.partitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), FILE_LIST, fileInfo); + return new HoodieRecord<>(key, payload); + } + + @Override + public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { + ValidationUtils.checkArgument(previousRecord.type == type, + "Cannot combine " + previousRecord.type + " with " + type); + + Map combinedFileInfo = null; + + switch (type) { + case PARTITION_LIST: + case FILE_LIST: + combinedFileInfo = combineFilesystemMetadata(previousRecord); + break; + default: + throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type); + } + + return new HoodieMetadataPayload(key, type, combinedFileInfo); + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException { + HoodieMetadataPayload anotherPayload = new HoodieMetadataPayload(Option.of((GenericRecord)oldRecord)); + HoodieRecordPayload combinedPayload = preCombine(anotherPayload); + return combinedPayload.getInsertValue(schema); + } + + @Override + public Option getInsertValue(Schema schema) throws IOException { + if (key == null) { + return Option.empty(); + } + + HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata); + return Option.of(record); + } + + /** + * Returns the list of filenames added as part of this record. + */ + public List getFilenames() { + return filterFileInfoEntries(false).map(e -> e.getKey()).sorted().collect(Collectors.toList()); + } + + /** + * Returns the list of filenames deleted as part of this record. + */ + public List getDeletions() { + return filterFileInfoEntries(true).map(Map.Entry::getKey).sorted().collect(Collectors.toList()); + } + + /** + * Returns the files added as part of this record. + */ + public FileStatus[] getFileStatuses(Path partitionPath) { + return filterFileInfoEntries(false) + .map(e -> new FileStatus(e.getValue().getSize(), false, 0, 0, 0, 0, null, null, null, + new Path(partitionPath, e.getKey()))) + .toArray(FileStatus[]::new); + } + + private Stream> filterFileInfoEntries(boolean isDeleted) { + if (filesystemMetadata == null) { + return Stream.empty(); + } + + return filesystemMetadata.entrySet().stream().filter(e -> e.getValue().getIsDeleted() == isDeleted); + } + + private Map combineFilesystemMetadata(HoodieMetadataPayload previousRecord) { + Map combinedFileInfo = new HashMap<>(); + if (previousRecord.filesystemMetadata != null) { + combinedFileInfo.putAll(previousRecord.filesystemMetadata); + } + + if (filesystemMetadata != null) { + filesystemMetadata.forEach((filename, fileInfo) -> { + // If the filename wasnt present then we carry it forward + if (!combinedFileInfo.containsKey(filename)) { + combinedFileInfo.put(filename, fileInfo); + } else { + if (fileInfo.getIsDeleted()) { + // file deletion + combinedFileInfo.remove(filename); + } else { + // file appends. + combinedFileInfo.merge(filename, fileInfo, (oldFileInfo, newFileInfo) -> { + return new HoodieMetadataFileInfo(oldFileInfo.getSize() + newFileInfo.getSize(), false); + }); + } + } + }); + } + + return combinedFileInfo; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); + sb.append("key=").append(key).append(", "); + sb.append("type=").append(type).append(", "); + sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); + sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); + 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 new file mode 100644 index 0000000000000..acb29f79ed311 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +/** + * Interface that supports querying various pieces of metadata about a hudi table. + */ +public interface HoodieTableMetadata extends Serializable { + + // Table name suffix + String METADATA_TABLE_NAME_SUFFIX = "_metadata"; + /** + * Timestamp for a commit when the base dataset had not had any commits yet. this is < than even + * {@link org.apache.hudi.common.table.timeline.HoodieTimeline#INIT_INSTANT_TS}, such that the metadata table + * can be prepped even before bootstrap is done. + */ + String SOLO_COMMIT_TIMESTAMP = "0000000000000"; + // Key for the record which saves list of all partitions + String RECORDKEY_PARTITION_LIST = "__all_partitions__"; + // The partition name used for non-partitioned tables + String NON_PARTITIONED_NAME = "."; + + // Base path of the Metadata Table relative to the dataset (.hoodie/metadata) + static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata"; + + /** + * Return the base path of the Metadata Table. + * + * @param tableBasePath The base path of the dataset + */ + static String getMetadataTableBasePath(String tableBasePath) { + return tableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH; + } + + /** + * Returns {@code True} if the given path contains a metadata table. + * + * @param basePath The base path to check + */ + static boolean isMetadataTable(String basePath) { + return basePath.endsWith(METADATA_TABLE_REL_PATH); + } + + static HoodieTableMetadata create(Configuration conf, String datasetBasePath, String spillableMapPath, boolean useFileListingFromMetadata, + boolean verifyListings, boolean enableMetrics, boolean shouldAssumeDatePartitioning) { + return new HoodieBackedTableMetadata(conf, datasetBasePath, spillableMapPath, useFileListingFromMetadata, verifyListings, + enableMetrics, shouldAssumeDatePartitioning); + } + + /** + * Fetch all the files at the given partition path, per the latest snapshot of the metadata. + */ + FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException; + + /** + * Fetch list of all partition paths, per the latest snapshot of the metadata. + */ + List getAllPartitionPaths() throws IOException; + + /** + * Get the instant time to which the metadata is synced w.r.t data timeline. + */ + Option getSyncedInstantTime(); + + boolean isInSync(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java new file mode 100644 index 0000000000000..115001a02a3a7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -0,0 +1,318 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; + +/** + * A utility to convert timeline information to metadata table records. + */ +public class HoodieTableMetadataUtil { + + private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class); + + /** + * Converts a timeline instant to metadata table records. + * + * @param datasetMetaClient The meta client associated with the timeline instant + * @param instant to fetch and convert to metadata table records + * @return a list of metadata table records + * @throws IOException + */ + public static Option> convertInstantToMetaRecords(HoodieTableMetaClient datasetMetaClient, HoodieInstant instant, Option lastSyncTs) throws IOException { + HoodieTimeline timeline = datasetMetaClient.getActiveTimeline(); + Option> records = Option.empty(); + ValidationUtils.checkArgument(instant.isCompleted(), "Only completed instants can be synced."); + + switch (instant.getAction()) { + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(datasetMetaClient, instant); + records = Option.of(convertMetadataToRecords(cleanMetadata, instant.getTimestamp())); + break; + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.COMPACTION_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + records = Option.of(convertMetadataToRecords(commitMetadata, instant.getTimestamp())); + break; + case HoodieTimeline.ROLLBACK_ACTION: + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + timeline.getInstantDetails(instant).get()); + records = Option.of(convertMetadataToRecords(rollbackMetadata, instant.getTimestamp(), lastSyncTs)); + break; + case HoodieTimeline.RESTORE_ACTION: + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( + timeline.getInstantDetails(instant).get()); + records = Option.of(convertMetadataToRecords(restoreMetadata, instant.getTimestamp(), lastSyncTs)); + break; + case HoodieTimeline.SAVEPOINT_ACTION: + // Nothing to be done here + break; + default: + throw new HoodieException("Unknown type of action " + instant.getAction()); + } + + return records; + } + + /** + * Finds all new files/partitions created as part of commit and creates metadata table records for them. + * + * @param commitMetadata + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieCommitMetadata commitMetadata, String instantTime) { + List records = new LinkedList<>(); + List allPartitions = new LinkedList<>(); + commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { + final String partition = partitionStatName.equals("") ? NON_PARTITIONED_NAME : partitionStatName; + allPartitions.add(partition); + + Map newFiles = new HashMap<>(writeStats.size()); + writeStats.forEach(hoodieWriteStat -> { + String pathWithPartition = hoodieWriteStat.getPath(); + if (pathWithPartition == null) { + // Empty partition + LOG.warn("Unable to find path in write stat to update metadata table " + hoodieWriteStat); + return; + } + + int offset = partition.equals(NON_PARTITIONED_NAME) ? 0 : partition.length() + 1; + String filename = pathWithPartition.substring(offset); + ValidationUtils.checkState(!newFiles.containsKey(filename), "Duplicate files in HoodieCommitMetadata"); + newFiles.put(filename, hoodieWriteStat.getTotalWriteBytes()); + }); + + // New files added to a partition + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord( + partition, Option.of(newFiles), Option.empty()); + records.add(record); + }); + + // New partitions created + HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(new ArrayList<>(allPartitions)); + records.add(record); + + LOG.info("Updating at " + instantTime + " from Commit/" + commitMetadata.getOperationType() + + ". #partitions_updated=" + records.size()); + return records; + } + + /** + * Finds all files that will be deleted as part of a planned clean and creates metadata table records for them. + * + * @param cleanerPlan from timeline to convert + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieCleanerPlan cleanerPlan, String instantTime) { + List records = new LinkedList<>(); + + int[] fileDeleteCount = {0}; + cleanerPlan.getFilePathsToBeDeletedPerPartition().forEach((partition, deletedPathInfo) -> { + fileDeleteCount[0] += deletedPathInfo.size(); + + // Files deleted from a partition + List deletedFilenames = deletedPathInfo.stream().map(p -> new Path(p.getFilePath()).getName()) + .collect(Collectors.toList()); + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), + Option.of(deletedFilenames)); + records.add(record); + }); + + LOG.info("Found at " + instantTime + " from CleanerPlan. #partitions_updated=" + records.size() + + ", #files_deleted=" + fileDeleteCount[0]); + return records; + } + + /** + * Finds all files that were deleted as part of a clean and creates metadata table records for them. + * + * @param cleanMetadata + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieCleanMetadata cleanMetadata, String instantTime) { + List records = new LinkedList<>(); + int[] fileDeleteCount = {0}; + + cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { + // Files deleted from a partition + List deletedFiles = partitionMetadata.getSuccessDeleteFiles(); + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), + Option.of(new ArrayList<>(deletedFiles))); + + records.add(record); + fileDeleteCount[0] += deletedFiles.size(); + }); + + LOG.info("Updating at " + instantTime + " from Clean. #partitions_updated=" + records.size() + + ", #files_deleted=" + fileDeleteCount[0]); + return records; + } + + /** + * Aggregates all files deleted and appended to from all rollbacks associated with a restore operation then + * creates metadata table records for them. + * + * @param restoreMetadata + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieRestoreMetadata restoreMetadata, String instantTime, Option lastSyncTs) { + Map> partitionToAppendedFiles = new HashMap<>(); + Map> partitionToDeletedFiles = new HashMap<>(); + restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { + rms.forEach(rm -> processRollbackMetadata(rm, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs)); + }); + + return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore"); + } + + public static List convertMetadataToRecords(HoodieRollbackMetadata rollbackMetadata, String instantTime, Option lastSyncTs) { + + Map> partitionToAppendedFiles = new HashMap<>(); + Map> partitionToDeletedFiles = new HashMap<>(); + processRollbackMetadata(rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs); + return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback"); + } + + /** + * Extracts information about the deleted and append files from the {@code HoodieRollbackMetadata}. + * + * During a rollback files may be deleted (COW, MOR) or rollback blocks be appended (MOR only) to files. This + * function will extract this change file for each partition. + * + * @param rollbackMetadata {@code HoodieRollbackMetadata} + * @param partitionToDeletedFiles The {@code Map} to fill with files deleted per partition. + * @param partitionToAppendedFiles The {@code Map} to fill with files appended per partition and their sizes. + */ + private static void processRollbackMetadata(HoodieRollbackMetadata rollbackMetadata, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + Option lastSyncTs) { + + rollbackMetadata.getPartitionMetadata().values().forEach(pm -> { + // Has this rollback produced new files? + boolean hasAppendFiles = pm.getAppendFiles().values().stream().mapToLong(Long::longValue).sum() > 0; + // If commit being rolled back has not been synced to metadata table yet then there is no need to update metadata + boolean shouldSkip = lastSyncTs.isPresent() + && HoodieTimeline.compareTimestamps(rollbackMetadata.getCommitsRollback().get(0), HoodieTimeline.GREATER_THAN, lastSyncTs.get()); + + if (!hasAppendFiles && shouldSkip) { + LOG.info(String.format("Skipping syncing of rollbackMetadata at %s, given metadata table is already synced upto to %s", + rollbackMetadata.getCommitsRollback().get(0), lastSyncTs.get())); + return; + } + + final String partition = pm.getPartitionPath(); + if (!pm.getSuccessDeleteFiles().isEmpty() && !shouldSkip) { + if (!partitionToDeletedFiles.containsKey(partition)) { + partitionToDeletedFiles.put(partition, new ArrayList<>()); + } + + // Extract deleted file name from the absolute paths saved in getSuccessDeleteFiles() + List deletedFiles = pm.getSuccessDeleteFiles().stream().map(p -> new Path(p).getName()) + .collect(Collectors.toList()); + partitionToDeletedFiles.get(partition).addAll(deletedFiles); + } + + if (!pm.getAppendFiles().isEmpty()) { + if (!partitionToAppendedFiles.containsKey(partition)) { + partitionToAppendedFiles.put(partition, new HashMap<>()); + } + + // Extract appended file name from the absolute paths saved in getAppendFiles() + pm.getAppendFiles().forEach((path, size) -> { + partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, (oldSize, newSizeCopy) -> { + return size + oldSize; + }); + }); + } + }); + } + + private static List convertFilesToRecords(Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, String instantTime, + String operation) { + List records = new LinkedList<>(); + int[] fileChangeCount = {0, 0}; // deletes, appends + + partitionToDeletedFiles.forEach((partition, deletedFiles) -> { + fileChangeCount[0] += deletedFiles.size(); + + Option> filesAdded = Option.empty(); + if (partitionToAppendedFiles.containsKey(partition)) { + filesAdded = Option.of(partitionToAppendedFiles.remove(partition)); + } + + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, filesAdded, + Option.of(new ArrayList<>(deletedFiles))); + records.add(record); + }); + + partitionToAppendedFiles.forEach((partition, appendedFileMap) -> { + fileChangeCount[1] += appendedFileMap.size(); + + // Validate that no appended file has been deleted + ValidationUtils.checkState( + !appendedFileMap.keySet().removeAll(partitionToDeletedFiles.getOrDefault(partition, Collections.emptyList())), + "Rollback file cannot both be appended and deleted"); + + // New files added to a partition + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.of(appendedFileMap), + Option.empty()); + records.add(record); + }); + + LOG.info("Found at " + instantTime + " from " + operation + ". #partitions_updated=" + records.size() + + ", #files_deleted=" + fileChangeCount[0] + ", #files_appended=" + fileChangeCount[1]); + + return records; + } +} 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 new file mode 100644 index 0000000000000..0436de707d2fd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +public enum MetadataPartitionType { + FILES("files"); + + private final String partitionPath; + + MetadataPartitionType(String partitionPath) { + this.partitionPath = partitionPath; + } + + public String partitionPath() { + return partitionPath; + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java index 1747888b15ad7..048402af1a434 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java @@ -31,7 +31,6 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.Job; -import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -97,13 +96,10 @@ public FileStatus[] listStatus(JobConf job) throws IOException { // process snapshot queries next. List snapshotPaths = inputPathHandler.getSnapshotPaths(); if (snapshotPaths.size() > 0) { - setInputPaths(job, snapshotPaths.toArray(new Path[snapshotPaths.size()])); - FileStatus[] fileStatuses = super.listStatus(job); - Map> groupedFileStatus = - HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses, HoodieFileFormat.HFILE.getFileExtension(), - tableMetaClientMap.values()); - LOG.info("Found a total of " + groupedFileStatus.size() + " groups"); - for (Map.Entry> entry : groupedFileStatus.entrySet()) { + Map> groupedPaths = + HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths); + LOG.info("Found a total of " + groupedPaths.size() + " groups"); + for (Map.Entry> entry : groupedPaths.entrySet()) { List result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue()); if (result != null) { returns.addAll(result); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 8b89949e803b6..d51aff0c0ef24 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -18,7 +18,6 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; @@ -109,13 +108,11 @@ public FileStatus[] listStatus(JobConf job) throws IOException { // process snapshot queries next. List snapshotPaths = inputPathHandler.getSnapshotPaths(); if (snapshotPaths.size() > 0) { - setInputPaths(job, snapshotPaths.toArray(new Path[snapshotPaths.size()])); - FileStatus[] fileStatuses = super.listStatus(job); - Map> groupedFileStatus = - HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses, - HoodieFileFormat.PARQUET.getFileExtension(), tableMetaClientMap.values()); - LOG.info("Found a total of " + groupedFileStatus.size() + " groups"); - for (Map.Entry> entry : groupedFileStatus.entrySet()) { + Map> groupedPaths = + HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths); + LOG.info("Found a total of " + groupedPaths.size() + " groups"); + for (Map.Entry> entry : groupedPaths.entrySet()) { + HoodieTableMetaClient metaClient = entry.getKey(); List result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue()); if (result != null) { returns.addAll(result); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java index 1e616f896bb30..baedb1688a844 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java @@ -22,9 +22,11 @@ import java.util.Set; import org.apache.hadoop.conf.Configurable; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.TableNotFoundException; @@ -43,6 +45,11 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; + /** * Given a path is a part of - Hoodie table = accepts ONLY the latest version of each path - Non-Hoodie table = then * always accept @@ -163,9 +170,13 @@ public boolean accept(Path path) { metaClientCache.put(baseDir.toString(), metaClient); } - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder)); - List latestFiles = fsView.getLatestBaseFiles().collect(Collectors.toList()); + boolean useFileListingFromMetadata = getConf().getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS); + boolean verifyFileListing = getConf().getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE); + HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(metaClient, + useFileListingFromMetadata, verifyFileListing); + String partition = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), folder); + + List latestFiles = fsView.getLatestBaseFiles(partition).collect(Collectors.toList()); // populate the cache if (!hoodiePathCache.containsKey(folder.toString())) { hoodiePathCache.put(folder.toString(), new HashSet<>()); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 9b0fbf9fc8b35..cf7da544d5c2c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; 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.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; @@ -62,6 +63,11 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + public class HoodieInputFormatUtils { // These positions have to be deterministic across all tables @@ -391,27 +397,48 @@ public static Map> groupFileStatusForSna return grouped; } + public static Map> groupSnapshotPathsByMetaClient( + Collection metaClientList, + List snapshotPaths + ) { + Map> grouped = new HashMap<>(); + metaClientList.forEach(metaClient -> grouped.put(metaClient, new ArrayList<>())); + for (Path path : snapshotPaths) { + // Find meta client associated with the input path + metaClientList.stream().filter(metaClient -> path.toString().contains(metaClient.getBasePath())) + .forEach(metaClient -> grouped.get(metaClient).add(path)); + } + return grouped; + } + /** - * Filters data files for a snapshot queried table. + * Filters data files under @param paths for a snapshot queried table. * @param job - * @param metadata - * @param fileStatuses + * @param metaClient + * @param paths * @return */ public static List filterFileStatusForSnapshotMode( - JobConf job, HoodieTableMetaClient metadata, List fileStatuses) throws IOException { - FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]); + JobConf job, HoodieTableMetaClient metaClient, List paths) throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata); + LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metaClient); } - // Get all commits, delta commits, compactions, as all of them produce a base parquet file today - HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - TableFileSystemView.BaseFileOnlyView roView = new HoodieTableFileSystemView(metadata, timeline, statuses); - // filter files on the latest commit found - List filteredFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); - LOG.info("Total paths to process after hoodie filter " + filteredFiles.size()); + + boolean useFileListingFromMetadata = job.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS); + boolean verifyFileListing = job.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE); + HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(metaClient, + useFileListingFromMetadata, verifyFileListing); + + List filteredBaseFiles = new ArrayList<>(); + for (Path p : paths) { + String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p); + List matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList()); + filteredBaseFiles.addAll(matched); + } + + LOG.info("Total paths to process after hoodie filter " + filteredBaseFiles.size()); List returns = new ArrayList<>(); - for (HoodieBaseFile filteredFile : filteredFiles) { + for (HoodieBaseFile filteredFile : filteredBaseFiles) { if (LOG.isDebugEnabled()) { LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index c8a0d7f82e96d..1b272ae89db75 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -26,6 +26,7 @@ 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.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; @@ -53,6 +54,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { private static final Logger LOG = LogManager.getLogger(HoodieRealtimeInputFormatUtils.class); @@ -63,13 +69,25 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream partitionsToMetaClient = getTableMetaClientByBasePath(conf, partitionsToParquetSplits.keySet()); + boolean useFileListingFromMetadata = conf.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS); + boolean verifyFileListing = conf.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE); + // Create file system cache so metadata table is only instantiated once. Also can benefit normal file listing if + // partition path is listed twice so file groups will already be loaded in file system + Map fsCache = new HashMap<>(); // for all unique split parents, obtain all delta files based on delta commit timeline, // grouped on file id List rtSplits = new ArrayList<>(); partitionsToParquetSplits.keySet().forEach(partitionPath -> { // for each partition path obtain the data & log file groupings, then map back to inputsplits HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); + if (!fsCache.containsKey(metaClient)) { + + HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(metaClient, + useFileListingFromMetadata, verifyFileListing); + fsCache.put(metaClient, fsView); + } + HoodieTableFileSystemView fsView = fsCache.get(metaClient); + String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); try { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index 43d5fdefaf461..a41da2c2196da 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -19,6 +19,7 @@ package org.apache.hudi.integ.testsuite.reader; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; @@ -85,7 +86,8 @@ protected List getPartitions(Option partitionsLimit) throws IOE // Using FSUtils.getFS here instead of metaClient.getFS() since we dont want to count these listStatus // calls in metrics as they are not part of normal HUDI operation. FileSystem fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf()); - List partitionPaths = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath(), false); + List partitionPaths = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath(), + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false); // Sort partition so we can pick last N partitions by default Collections.sort(partitionPaths); if (!partitionPaths.isEmpty()) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 988c9f9191e71..472f450337022 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -29,7 +29,7 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.HoodieWriteResult import org.apache.hudi.client.SparkRDDWriteClient -import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.timeline.HoodieActiveTimeline @@ -38,7 +38,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, B import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} -import org.apache.hudi.internal.{HoodieDataSourceInternalWriter, DataSourceInternalWriterHelper} +import org.apache.hudi.internal.{DataSourceInternalWriterHelper, HoodieDataSourceInternalWriter} import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.log4j.LogManager import org.apache.spark.SPARK_VERSION @@ -372,6 +372,8 @@ private[hudi] object HoodieSparkSqlWriter { ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*) hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) hiveSyncConfig.useJdbc = parameters(HIVE_USE_JDBC_OPT_KEY).toBoolean + hiveSyncConfig.useFileListingFromMetadata = parameters(HoodieMetadataConfig.METADATA_ENABLE_PROP).toBoolean + hiveSyncConfig.verifyMetadataFileListing = parameters(HoodieMetadataConfig.METADATA_VALIDATE_PROP).toBoolean hiveSyncConfig.supportTimestamp = parameters.get(HIVE_SUPPORT_TIMESTAMP).exists(r => r.toBoolean) hiveSyncConfig.decodePartition = parameters.getOrElse(URL_ENCODE_PARTITIONING_OPT_KEY, DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL).toBoolean diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 294050b6d46ca..02b5abd12c62c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -23,6 +23,11 @@ import org.apache.hudi.common.config.TypedProperties import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters.mapAsScalaMapConverter +import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE +import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE +import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP +import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP + /** * WriterUtils to assist in write path in Datasource and tests. */ @@ -46,6 +51,8 @@ object HoodieWriterUtils { RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL, PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL, KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, + METADATA_ENABLE_PROP -> DEFAULT_METADATA_ENABLE.toString, + METADATA_VALIDATE_PROP -> DEFAULT_METADATA_VALIDATE.toString, COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL, INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL, STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL, diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java index 7e13a5e4b0fa4..521ff05513a03 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java @@ -28,6 +28,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.bootstrap.FileStatusUtils; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; @@ -372,7 +373,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -390,7 +392,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -406,7 +409,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -423,7 +427,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -438,7 +443,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -455,7 +461,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 51ca72e0411f4..f315a2672298e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -22,6 +22,7 @@ import java.util.function.Supplier import java.util.stream.Stream import org.apache.hadoop.fs.Path +import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.testutils.HoodieTestDataGenerator @@ -34,6 +35,8 @@ import org.apache.spark.sql.functions.{col, lit} import org.apache.spark.sql.types.{DataTypes, DateType, IntegerType, StringType, StructField, StructType, TimestampType} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.JavaConversions._ @@ -82,13 +85,18 @@ class TestCOWDataSource extends HoodieClientTestBase { assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) } - @Test def testCopyOnWriteStorage() { + @ParameterizedTest + //TODO(metadata): Needs HUDI-1459 to be fixed + //@ValueSource(booleans = Array(true, false)) + @ValueSource(booleans = Array(false)) + def testCopyOnWriteStorage(isMetadataEnabled: Boolean) { // Insert Operation val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") .options(commonOpts) .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .mode(SaveMode.Overwrite) .save(basePath) @@ -96,7 +104,9 @@ class TestCOWDataSource extends HoodieClientTestBase { val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath) // Snapshot query - val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*") + val snapshotDF1 = spark.read.format("org.apache.hudi") + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) + .load(basePath + "/*/*/*/*") assertEquals(100, snapshotDF1.count()) // Upsert based on the written table with Hudi metadata columns @@ -120,6 +130,7 @@ class TestCOWDataSource extends HoodieClientTestBase { inputDF2.write.format("org.apache.hudi") .options(commonOpts) + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .mode(SaveMode.Append) .save(basePath) @@ -128,6 +139,7 @@ class TestCOWDataSource extends HoodieClientTestBase { // Snapshot Query val snapshotDF3 = spark.read.format("org.apache.hudi") + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .load(basePath + "/*/*/*/*") assertEquals(100, snapshotDF3.count()) // still 100, since we only updated @@ -149,6 +161,7 @@ class TestCOWDataSource extends HoodieClientTestBase { val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1)) emptyDF.write.format("org.apache.hudi") .options(commonOpts) + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .mode(SaveMode.Append) .save(basePath) diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java index 8a92de64c2ac3..5b50ada22c6c3 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java @@ -19,6 +19,8 @@ package org.apache.hudi.dla; import com.beust.jcommander.Parameter; + +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import java.io.Serializable; @@ -68,6 +70,12 @@ public class DLASyncConfig implements Serializable { @Parameter(names = {"--hive-style-partitioning"}, description = "Use DLA hive style partitioning, true if like the following style: field1=value1/field2=value2") public Boolean useDLASyncHiveStylePartitioning = false; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -88,6 +96,8 @@ public static DLASyncConfig copy(DLASyncConfig cfg) { newConfig.skipROSuffix = cfg.skipROSuffix; newConfig.skipRTSync = cfg.skipRTSync; newConfig.useDLASyncHiveStylePartitioning = cfg.useDLASyncHiveStylePartitioning; + newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; + newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; return newConfig; } @@ -99,6 +109,8 @@ public String toString() { + ", basePath='" + basePath + '\'' + ", partitionFields=" + partitionFields + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + ", assumeDatePartitioning=" + assumeDatePartitioning + ", useDLASyncHiveStylePartitioning=" + useDLASyncHiveStylePartitioning + + ", useFileListingFromMetadata=" + useFileListingFromMetadata + + ", verifyMetadataFileListing=" + verifyMetadataFileListing + ", help=" + help + '}'; } } diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java index 34a96c9ad8fe6..02c07d6e5861f 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java @@ -70,7 +70,8 @@ public class HoodieDLAClient extends AbstractSyncHoodieClient { private PartitionValueExtractor partitionValueExtractor; public HoodieDLAClient(DLASyncConfig syncConfig, FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, fs); + super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, + syncConfig.verifyMetadataFileListing, fs); this.dlaConfig = syncConfig; try { this.partitionValueExtractor = diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index 6c8fd8f916483..dd9d483fb10b7 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -18,6 +18,8 @@ package org.apache.hudi.hive; +import org.apache.hudi.common.config.HoodieMetadataConfig; + import com.beust.jcommander.Parameter; import java.io.Serializable; @@ -77,6 +79,12 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") public Boolean skipROSuffix = false; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -99,6 +107,8 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.jdbcUrl = cfg.jdbcUrl; newConfig.tableName = cfg.tableName; newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; + newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; + newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; newConfig.decodePartition = cfg.decodePartition; return newConfig; @@ -107,23 +117,25 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { @Override public String toString() { return "HiveSyncConfig{" - + "databaseName='" + databaseName + '\'' - + ", tableName='" + tableName + '\'' - + ", baseFileFormat='" + baseFileFormat + '\'' - + ", hiveUser='" + hiveUser + '\'' - + ", hivePass='" + hivePass + '\'' - + ", jdbcUrl='" + jdbcUrl + '\'' - + ", basePath='" + basePath + '\'' - + ", partitionFields=" + partitionFields - + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' - + ", assumeDatePartitioning=" + assumeDatePartitioning - + ", usePreApacheInputFormat=" + usePreApacheInputFormat - + ", useJdbc=" + useJdbc - + ", autoCreateDatabase=" + autoCreateDatabase - + ", skipROSuffix=" + skipROSuffix - + ", help=" + help - + ", supportTimestamp=" + supportTimestamp - + ", decodePartition=" + decodePartition - + '}'; + + "databaseName='" + databaseName + '\'' + + ", tableName='" + tableName + '\'' + + ", baseFileFormat='" + baseFileFormat + '\'' + + ", hiveUser='" + hiveUser + '\'' + + ", hivePass='" + hivePass + '\'' + + ", jdbcUrl='" + jdbcUrl + '\'' + + ", basePath='" + basePath + '\'' + + ", partitionFields=" + partitionFields + + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + + ", assumeDatePartitioning=" + assumeDatePartitioning + + ", usePreApacheInputFormat=" + usePreApacheInputFormat + + ", useJdbc=" + useJdbc + + ", autoCreateDatabase=" + autoCreateDatabase + + ", skipROSuffix=" + skipROSuffix + + ", help=" + help + + ", supportTimestamp=" + supportTimestamp + + ", decodePartition=" + decodePartition + + ", useFileListingFromMetadata=" + useFileListingFromMetadata + + ", verifyMetadataFileListing=" + verifyMetadataFileListing + + '}'; } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index 88f4c10611b08..5c0c1287d0590 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -76,7 +76,7 @@ public class HoodieHiveClient extends AbstractSyncHoodieClient { private HiveConf configuration; public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { - super(cfg.basePath, cfg.assumeDatePartitioning, fs); + super(cfg.basePath, cfg.assumeDatePartitioning, cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, fs); this.syncConfig = cfg; this.fs = fs; diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java index 419ea16deefd6..8c91848476b65 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java @@ -40,18 +40,25 @@ import java.util.Map; public abstract class AbstractSyncHoodieClient { + private static final Logger LOG = LogManager.getLogger(AbstractSyncHoodieClient.class); + protected final HoodieTableMetaClient metaClient; protected final HoodieTableType tableType; protected final FileSystem fs; private String basePath; private boolean assumeDatePartitioning; + private boolean useFileListingFromMetadata; + private boolean verifyMetadataFileListing; - public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, FileSystem fs) { + public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, boolean useFileListingFromMetadata, + boolean verifyMetadataFileListing, FileSystem fs) { this.metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); this.tableType = metaClient.getTableType(); this.basePath = basePath; this.assumeDatePartitioning = assumeDatePartitioning; + this.useFileListingFromMetadata = useFileListingFromMetadata; + this.verifyMetadataFileListing = verifyMetadataFileListing; this.fs = fs; } @@ -120,7 +127,7 @@ public List getPartitionsWrittenToSince(Option lastCommitTimeSyn if (!lastCommitTimeSynced.isPresent()) { LOG.info("Last commit time synced is not known, listing all partitions in " + basePath + ",FS :" + fs); try { - return FSUtils.getAllPartitionPaths(fs, basePath, assumeDatePartitioning); + return FSUtils.getAllPartitionPaths(fs, basePath, useFileListingFromMetadata, verifyMetadataFileListing, assumeDatePartitioning); } catch (IOException e) { throw new HoodieIOException("Failed to list all partitions in " + basePath, e); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index 282610893df55..0066d86d595b5 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; @@ -68,10 +69,18 @@ static class Config implements Serializable { @Parameter(names = {"--date-partitioned", "-dp"}, description = "Can we assume date partitioning?") boolean shouldAssumeDatePartitioning = false; + + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; } public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, - final boolean shouldAssumeDatePartitioning) throws IOException { + final boolean shouldAssumeDatePartitioning, + final boolean useFileListingFromMetadata, + final boolean verifyMetadataFileListing) throws IOException { FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration()); final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration()); final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); @@ -88,7 +97,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi LOG.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommitTimestamp)); - List partitions = FSUtils.getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning); + List partitions = FSUtils.getAllPartitionPaths(fs, baseDir, useFileListingFromMetadata, verifyMetadataFileListing, shouldAssumeDatePartitioning); if (partitions.size() > 0) { LOG.info(String.format("The job needs to copy %d partitions.", partitions.size())); @@ -183,7 +192,8 @@ public static void main(String[] args) throws IOException { // Copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning); + copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning, cfg.useFileListingFromMetadata, + cfg.verifyMetadataFileListing); // Stop the job jsc.stop(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index c69d0044ed933..2f5f4613ad879 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -154,7 +154,7 @@ private Option getLatestCommitTimestamp(FileSystem fs, Config cfg) { } private List getPartitions(FileSystem fs, Config cfg) throws IOException { - return FSUtils.getAllPartitionPaths(fs, cfg.sourceBasePath, false); + return FSUtils.getAllPartitionPaths(fs, cfg.sourceBasePath, true, false, false); } private void createSuccessTag(FileSystem fs, Config cfg) throws IOException { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java index f338e52cbf70b..27dc70914806f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -85,7 +86,8 @@ private void setHostAddrFromSparkConf(SparkConf sparkConf) { public void run() throws IOException { - List allPartitionPaths = FSUtils.getAllPartitionPaths(timelineServer.getFs(), cfg.basePath, true); + List allPartitionPaths = FSUtils.getAllPartitionPaths(timelineServer.getFs(), cfg.basePath, cfg.useFileListingFromMetadata, + cfg.verifyMetadataFileListing, true); Collections.shuffle(allPartitionPaths); List selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions) .collect(Collectors.toList()); @@ -294,6 +296,12 @@ public static class Config implements Serializable { @Parameter(names = {"--wait-for-manual-queries", "-ww"}) public Boolean waitForManualQueries = false; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + @Parameter(names = {"--help", "-h"}) public Boolean help = false; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java index 95af888fc48eb..0b19fa0801397 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java @@ -18,6 +18,7 @@ package org.apache.hudi.utilities.functional; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -67,7 +68,9 @@ public void testEmptySnapshotCopy() throws IOException { // Do the snapshot HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc(), basePath, outputPath, true); + copier.snapshot(jsc(), basePath, outputPath, true, + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE); // Nothing changed; we just bail out assertEquals(fs.listStatus(new Path(basePath)).length, 1); @@ -120,7 +123,8 @@ public void testSnapshotCopy() throws Exception { // Do a snapshot copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc(), basePath, outputPath, false); + copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE); // Check results assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));