diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java index 66c2eb02159e4..084d757f85250 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieSyncCommand.java @@ -89,7 +89,7 @@ public String validateSync( } private String getString(HoodieTableMetaClient target, HoodieTimeline targetTimeline, HoodieTableMetaClient source, long sourceCount, long targetCount, String sourceLatestCommit) - throws IOException { + throws IOException { List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) .getInstants().collect(Collectors.toList()); if (commitsToCatchup.isEmpty()) { diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java index 433e9df4cad3e..127cb28ad0101 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.exception.HoodieIOException; @@ -186,11 +187,11 @@ public void removeCorruptedPendingCleanAction() { CleanerUtils.getCleanerPlan(client, instant); } catch (AvroRuntimeException e) { LOG.warn("Corruption found. Trying to remove corrupted clean instant file: " + instant); - FSUtils.deleteInstantFile(client.getFs(), client.getMetaPath(), instant); + HoodieActiveTimeline.deleteInstantFile(client.getFs(), client.getMetaPath(), instant); } catch (IOException ioe) { if (ioe.getMessage().contains("Not an Avro data file")) { LOG.warn("Corruption found. Trying to remove corrupted clean instant file: " + instant); - FSUtils.deleteInstantFile(client.getFs(), client.getMetaPath(), instant); + HoodieActiveTimeline.deleteInstantFile(client.getFs(), client.getMetaPath(), instant); } else { throw new HoodieIOException(ioe.getMessage(), ioe); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index 791f4c21cf101..9732ce72b913d 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -24,6 +24,7 @@ import org.apache.hudi.cli.functional.CLIFunctionalTestHarness; import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; import org.apache.hudi.cli.testutils.HoodieTestCommitUtilities; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -71,6 +72,7 @@ public void init() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); // Create six commits diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index cdf642799438d..d71e7ec8d987d 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -25,6 +25,7 @@ import org.apache.hudi.cli.functional.CLIFunctionalTestHarness; import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; import org.apache.hudi.cli.testutils.HoodieTestReplaceCommitMetadataGenerator; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -208,6 +209,7 @@ public void testShowArchivedCommits() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath1) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); // generate data and metadata diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java index e6d1dee89aa4b..de305f404455b 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java @@ -24,6 +24,7 @@ import org.apache.hudi.cli.TableHeader; import org.apache.hudi.cli.functional.CLIFunctionalTestHarness; import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -158,6 +159,7 @@ private void generateArchive() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .forTable("test-trip-table").build(); // archive HoodieTableMetaClient metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()); 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 45c7151582281..ec586a18034c3 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 @@ -61,6 +61,7 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; @@ -241,13 +242,16 @@ void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String } } + /** + * Any pre-commit actions like conflict resolution or updating metadata table goes here. + * @param instantTime commit instant time. + * @param metadata commit metadata for which pre commit is being invoked. + */ protected void preCommit(String instantTime, HoodieCommitMetadata metadata) { - // no-op - // TODO : Conflict resolution is not supported for Flink & Java engines - } - - protected void syncTableMetadata() { - // no-op + // Create a Hoodie table after starting the transaction which encapsulated the commits and files visible. + // Important to create this after the lock to ensure latest commits show up in the timeline without need for reload + HoodieTable table = createTable(config, hadoopConf); + table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, instantTime)); } /** @@ -404,16 +408,6 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient); - this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxnAndMetadata - .isPresent() - ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); - try { - if (writeOperationType != WriteOperationType.CLUSTER && writeOperationType != WriteOperationType.COMPACT) { - syncTableMetadata(); - } - } finally { - this.txnManager.endTransaction(); - } this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this); } @@ -443,9 +437,6 @@ 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); - if (operationType != null && operationType != WriteOperationType.CLUSTER && operationType != WriteOperationType.COMPACT) { - syncTableMetadata(); - } } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } finally { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java index 80a412010203e..39f397ab170d6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java @@ -58,7 +58,7 @@ public static Option resolveWriteConflictIfAny(final Hoodi if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy(); Stream instantStream = resolutionStrategy.getCandidateInstants(table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant); - final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.get()); + final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElse(new HoodieCommitMetadata())); instantStream.forEach(instant -> { try { ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient()); 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 ead3ef1f07b8d..c9d8c4f117eaf 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 @@ -1679,10 +1679,6 @@ public boolean isMetadataTableEnabled() { return metadataConfig.enabled(); } - public boolean getFileListingMetadataVerify() { - return metadataConfig.validateFileListingMetadata(); - } - public int getMetadataInsertParallelism() { return getInt(HoodieMetadataConfig.INSERT_PARALLELISM_VALUE); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index f7979459458cd..19e9d313e3e9a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -19,7 +19,6 @@ 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; @@ -32,12 +31,17 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; 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; @@ -51,7 +55,6 @@ import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hadoop.conf.Configuration; @@ -83,11 +86,12 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadataWriter.class); protected HoodieWriteConfig metadataWriteConfig; - protected HoodieWriteConfig datasetWriteConfig; + protected HoodieWriteConfig dataWriteConfig; protected String tableName; protected HoodieBackedTableMetadata metadata; - protected HoodieTableMetaClient metaClient; + protected HoodieTableMetaClient metadataMetaClient; + protected HoodieTableMetaClient dataMetaClient; protected Option metrics; protected boolean enabled; protected SerializableConfiguration hadoopConf; @@ -95,7 +99,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { - this.datasetWriteConfig = writeConfig; + this.dataWriteConfig = writeConfig; this.engineContext = engineContext; this.hadoopConf = new SerializableConfiguration(hadoopConf); @@ -112,17 +116,9 @@ protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteC ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(), "File listing cannot be used for Metadata Table"); initRegistry(); - HoodieTableMetaClient datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(datasetWriteConfig.getBasePath()).build(); - initialize(engineContext, datasetMetaClient); - if (enabled) { - // 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())); - } + this.dataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(dataWriteConfig.getBasePath()).build(); + initialize(engineContext); + initTableMetadata(); } else { enabled = false; this.metrics = Option.empty(); @@ -165,7 +161,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) .archiveCommitsWith(minCommitsToKeep, maxCommitsToKeep) // we will trigger compaction manually, to control the instant times @@ -174,7 +170,8 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withParallelism(parallelism, parallelism) .withDeleteParallelism(parallelism) .withRollbackParallelism(parallelism) - .withFinalizeWriteParallelism(parallelism); + .withFinalizeWriteParallelism(parallelism) + .withAllowMultiWriteOnSameInstant(true); if (writeConfig.isMetricsOn()) { builder.withMetricsConfig(HoodieMetricsConfig.newBuilder() @@ -216,48 +213,43 @@ public HoodieBackedTableMetadata 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 + * Initialize the metadata table if it does not exist. * - * 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. + * If the metadata table did not exist, then file and partition listing is used to bootstrap the table. */ - protected abstract void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient); + protected abstract void initialize(HoodieEngineContext engineContext); protected void initTableMetadata() { try { if (this.metadata != null) { this.metadata.close(); } - this.metadata = new HoodieBackedTableMetadata(engineContext, datasetWriteConfig.getMetadataConfig(), - datasetWriteConfig.getBasePath(), datasetWriteConfig.getSpillableMapBasePath()); - this.metaClient = metadata.getMetaClient(); + this.metadata = new HoodieBackedTableMetadata(engineContext, dataWriteConfig.getMetadataConfig(), + dataWriteConfig.getBasePath(), dataWriteConfig.getSpillableMapBasePath()); + this.metadataMetaClient = metadata.getMetadataMetaClient(); } catch (Exception e) { throw new HoodieException("Error initializing metadata table for reads", e); } } - protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableMetaClient dataMetaClient) throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); - boolean exists = datasetMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); + boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); boolean rebootstrap = false; if (exists) { // If the un-synched instants have been archived then the metadata table will need to be bootstrapped again - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) .setBasePath(metadataWriteConfig.getBasePath()).build(); - Option latestMetadataInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); + Option latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); if (!latestMetadataInstant.isPresent()) { LOG.warn("Metadata Table will need to be re-bootstrapped as no instants were found"); rebootstrap = true; } else if (!latestMetadataInstant.get().getTimestamp().equals(SOLO_COMMIT_TIMESTAMP) - && datasetMetaClient.getActiveTimeline().isBeforeTimelineStarts(latestMetadataInstant.get().getTimestamp())) { + && dataMetaClient.getActiveTimeline().getAllCommitsTimeline().isBeforeTimelineStarts(latestMetadataInstant.get().getTimestamp())) { + // TODO: Revisit this logic and validate that filtering for all commits timeline is the right thing to do LOG.warn("Metadata Table will need to be re-bootstrapped as un-synced instants have been archived." + " latestMetadataInstant=" + latestMetadataInstant.get().getTimestamp() - + ", latestDatasetInstant=" + datasetMetaClient.getActiveTimeline().firstInstant().get().getTimestamp()); + + ", latestDataInstant=" + dataMetaClient.getActiveTimeline().firstInstant().get().getTimestamp()); rebootstrap = true; } } @@ -265,13 +257,13 @@ protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableM if (rebootstrap) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.REBOOTSTRAP_STR, 1)); LOG.info("Deleting Metadata Table directory so that it can be re-bootstrapped"); - datasetMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath()), true); + dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath()), true); exists = false; } if (!exists) { // Initialize for the first time by listing partitions and files directly from the file system - if (bootstrapFromFilesystem(engineContext, datasetMetaClient)) { + if (bootstrapFromFilesystem(engineContext, dataMetaClient)) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); } } @@ -280,23 +272,23 @@ protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableM /** * Initialize the Metadata Table by listing files and partitions from the file system. * - * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + * @param dataMetaClient {@code HoodieTableMetaClient} for the dataset. */ - private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient dataMetaClient) throws IOException { ValidationUtils.checkState(enabled, "Metadata table cannot be initialized as it is not enabled"); // We can only bootstrap if there are no pending operations on the dataset - Option pendingInstantOption = Option.fromJavaOptional(datasetMetaClient.getActiveTimeline() + Option pendingDataInstant = Option.fromJavaOptional(dataMetaClient.getActiveTimeline() .getReverseOrderedInstants().filter(i -> !i.isCompleted()).findFirst()); - if (pendingInstantOption.isPresent()) { + if (pendingDataInstant.isPresent()) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); - LOG.warn("Cannot bootstrap metadata table as operation is in progress: " + pendingInstantOption.get()); + LOG.warn("Cannot bootstrap metadata table as operation is in progress in dataset: " + pendingDataInstant.get()); return false; } // 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 latest commit timestamp. - String createInstantTime = datasetMetaClient.getActiveTimeline().getReverseOrderedInstants().findFirst() + String createInstantTime = dataMetaClient.getActiveTimeline().getReverseOrderedInstants().findFirst() .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); @@ -309,10 +301,11 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); initTableMetadata(); + initializeFileGroups(dataMetaClient, MetadataPartitionType.FILES, createInstantTime, 1); // 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); + LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); + Map> partitionToFileStatus = getPartitionsToFilesMapping(dataMetaClient); // Create a HoodieCommitMetadata with writeStats for all discovered files int[] stats = {0}; @@ -349,17 +342,17 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi /** * Function to find hoodie partitions and list files in them in parallel. * - * @param datasetMetaClient + * @param dataMetaClient * @return Map of partition names to a list of FileStatus for all the files in the partition */ - private Map> getPartitionsToFilesMapping(HoodieTableMetaClient datasetMetaClient) { + private Map> getPartitionsToFilesMapping(HoodieTableMetaClient dataMetaClient) { List pathsToList = new LinkedList<>(); - pathsToList.add(new Path(datasetWriteConfig.getBasePath())); + pathsToList.add(new Path(dataWriteConfig.getBasePath())); Map> partitionToFileStatus = new HashMap<>(); final int fileListingParallelism = metadataWriteConfig.getFileListingParallelism(); - SerializableConfiguration conf = new SerializableConfiguration(datasetMetaClient.getHadoopConf()); - final String dirFilterRegex = datasetWriteConfig.getMetadataConfig().getDirectoryFilterRegex(); + SerializableConfiguration conf = new SerializableConfiguration(dataMetaClient.getHadoopConf()); + final String dirFilterRegex = dataWriteConfig.getMetadataConfig().getDirectoryFilterRegex(); while (!pathsToList.isEmpty()) { int listingParallelism = Math.min(fileListingParallelism, pathsToList.size()); @@ -383,7 +376,7 @@ private Map> getPartitionsToFilesMapping(HoodieTableMet .collect(Collectors.toList()); if (p.getRight().length > filesInDir.size()) { - String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), p.getLeft()); + String partitionName = FSUtils.getRelativePartitionPath(new Path(dataMetaClient.getBasePath()), p.getLeft()); // deal with Non-partition table, we should exclude .hoodie partitionToFileStatus.put(partitionName, filesInDir.stream() .filter(f -> !f.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)).collect(Collectors.toList())); @@ -401,64 +394,78 @@ private Map> getPartitionsToFilesMapping(HoodieTableMet } /** - * Sync the Metadata Table from the instants created on the dataset. + * Initialize file groups for a partition. For file listing, we just have one file group. * - * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + * All FileGroups for a given metadata partition has a fixed prefix as per the {@link MetadataPartitionType#getFileIdPrefix()}. + * Each file group is suffixed with 4 digits with increments of 1 starting with 0000. + * + * Lets say we configure 10 file groups for record level index partittion, and prefix as "record-index-bucket-" + * File groups will be named as : + * record-index-bucket-0000, .... -> ..., record-index-bucket-0009 */ - private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { - ValidationUtils.checkState(enabled, "Metadata table cannot be synced as it is not enabled"); - // (re) init the metadata for reading. - initTableMetadata(); - try { - List instantsToSync = metadata.findInstantsToSyncForWriter(); - if (instantsToSync.isEmpty()) { - return; + private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, + int fileGroupCount) throws IOException { + + final HashMap blockHeader = new HashMap<>(); + blockHeader.put(HeaderMetadataType.INSTANT_TIME, instantTime); + // Archival of data table has a dependency on compaction(base files) in metadata table. + // It is assumed that as of time Tx of base instant (/compaction time) in metadata table, + // all commits in data table is in sync with metadata table. So, we always start with log file for any fileGroup. + final HoodieDeleteBlock block = new HoodieDeleteBlock(new HoodieKey[0], blockHeader); + + LOG.info(String.format("Creating %d file groups for partition %s with base fileId %s at instant time %s", + fileGroupCount, metadataPartition.partitionPath(), metadataPartition.getFileIdPrefix(), instantTime)); + for (int i = 0; i < fileGroupCount; ++i) { + final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i); + try { + HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() + .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.partitionPath())) + .withFileId(fileGroupFileId).overBaseCommit(instantTime) + .withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION) + .withFileSize(0L) + .withSizeThreshold(metadataWriteConfig.getLogFileMaxSize()) + .withFs(dataMetaClient.getFs()) + .withRolloverLogWriteToken(HoodieLogFormat.DEFAULT_WRITE_TOKEN) + .withLogWriteToken(HoodieLogFormat.DEFAULT_WRITE_TOKEN) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); + writer.appendBlock(block); + writer.close(); + } catch (InterruptedException e) { + throw new HoodieException("Failed to created fileGroup " + fileGroupFileId + " for partition " + metadataPartition.partitionPath(), e); } - - LOG.info("Syncing " + instantsToSync.size() + " instants to metadata table: " + instantsToSync); - - // Read each instant in order and sync it to metadata table - for (HoodieInstant instant : instantsToSync) { - LOG.info("Syncing instant " + instant + " to metadata table"); - - Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, - metaClient.getActiveTimeline(), instant, metadata.getUpdateTime()); - if (records.isPresent()) { - commit(records.get(), MetadataPartitionType.FILES.partitionPath(), instant.getTimestamp()); - } - } - 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 + * Interface to assist in converting commit metadata to List of HoodieRecords to be written to metadata table. + * Updates of different commit metadata uses the same method to convert to HoodieRecords and hence. */ - @Override - public void update(HoodieCommitMetadata commitMetadata, String instantTime) { - if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime); + private interface ConvertMetadataFunction { + List convertMetadata(); + } + + /** + * Processes commit metadata from data table and commits to metadata table. + * @param instantTime instant time of interest. + * @param convertMetadataFunction converter function to convert the respective metadata to List of HoodieRecords to be written to metadata table. + * @param type of commit metadata. + */ + private void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction) { + if (enabled && metadata != null) { + List records = convertMetadataFunction.convertMetadata(); commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); } } /** - * Update from {@code HoodieCleanerPlan}. + * Update from {@code HoodieCommitMetadata}. * - * @param cleanerPlan {@code HoodieCleanerPlan} - * @param instantTime Timestamp at which the clean plan was generated + * @param commitMetadata {@code HoodieCommitMetadata} + * @param instantTime Timestamp at which the commit was performed */ @Override - public void update(HoodieCleanerPlan cleanerPlan, String instantTime) { - if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanerPlan, instantTime); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); - } + public void update(HoodieCommitMetadata commitMetadata, String instantTime) { + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime)); } /** @@ -469,10 +476,7 @@ public void update(HoodieCleanerPlan cleanerPlan, String instantTime) { */ @Override public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { - if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); - } + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime)); } /** @@ -483,11 +487,8 @@ public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { */ @Override public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { - if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(metaClient.getActiveTimeline(), - restoreMetadata, instantTime, metadata.getUpdateTime()); - commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); - } + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), + restoreMetadata, instantTime, metadata.getSyncedInstantTime())); } /** @@ -498,9 +499,21 @@ public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { */ @Override public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) { - if (enabled) { - List records = HoodieTableMetadataUtil.convertMetadataToRecords(metaClient.getActiveTimeline(), - rollbackMetadata, instantTime, metadata.getUpdateTime()); + if (enabled && metadata != null) { + // Is this rollback of an instant that has been synced to the metadata table? + String rollbackInstant = rollbackMetadata.getCommitsRollback().get(0); + boolean wasSynced = metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, rollbackInstant)); + if (!wasSynced) { + // A compaction may have taken place on metadata table which would have included this instant being rolled back. + // Revisit this logic to relax the compaction fencing : https://issues.apache.org/jira/browse/HUDI-2458 + Option latestCompaction = metadata.getLatestCompactionTime(); + if (latestCompaction.isPresent()) { + wasSynced = HoodieTimeline.compareTimestamps(rollbackInstant, HoodieTimeline.LESSER_THAN_OR_EQUALS, latestCompaction.get()); + } + } + + List records = HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), rollbackMetadata, instantTime, + metadata.getSyncedInstantTime(), wasSynced); commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); } } @@ -512,13 +525,12 @@ public void close() throws Exception { } } - public HoodieBackedTableMetadata getMetadataReader() { - return metadata; - } - /** * Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit. * + * @param records The list of records to be written. + * @param partitionName The partition to which the records are to be written. + * @param instantTime The timestamp to use for the deltacommit. */ 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 index 02c5b9e646ad0..f5c4d26d0ce4d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -19,7 +19,6 @@ 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; @@ -31,13 +30,32 @@ */ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { + /** + * Update the metadata table due to a COMMIT operation. + * @param commitMetadata commit metadata of the operation of interest. + * @param instantTime instant time of the commit. + */ void update(HoodieCommitMetadata commitMetadata, String instantTime); - void update(HoodieCleanerPlan cleanerPlan, String instantTime); - + /** + * Update the metadata table due to a CLEAN operation. + * @param cleanMetadata clean metadata of the operation of interest. + * @param instantTime instant time of the commit. + */ void update(HoodieCleanMetadata cleanMetadata, String instantTime); + /** + * Update the metadata table due to a RESTORE operation. + * @param restoreMetadata restore metadata of the operation of interest. + * @param instantTime instant time of the commit. + */ void update(HoodieRestoreMetadata restoreMetadata, String instantTime); + /** + * Update the metadata table due to a ROLLBACK operation. + * @param rollbackMetadata rollback metadata of the operation of interest. + * @param instantTime instant time of the commit. + */ 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 f701e4036bdd0..a82a8bccf9bb1 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 @@ -55,6 +55,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; 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.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -63,6 +64,7 @@ import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.marker.WriteMarkers; @@ -703,4 +705,13 @@ public HoodieEngineContext getContext() { // to engine context, and it ends up being null (as its not serializable and marked transient here). return context == null ? new HoodieLocalEngineContext(hadoopConfiguration.get()) : context; } + + /** + * Fetch instance of {@link HoodieTableMetadataWriter}. + * @return instance of {@link HoodieTableMetadataWriter} + */ + public Option getMetadataWriter() { + ValidationUtils.checkArgument(!config.isMetadataTableEnabled(), "Metadata Table support not enabled in this Table"); + return Option.empty(); + } } 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 317512f766c99..d492fb6577a93 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 @@ -200,20 +200,19 @@ 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 metadata table is enabled, do not archive instants which are more recent that the last compaction on the + // metadata table. if (config.isMetadataTableEnabled()) { try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(), config.getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) { - Option lastSyncedInstantTime = tableMetadata.getUpdateTime(); - - 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"); + Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); + if (!latestCompactionTime.isPresent()) { + LOG.info("Not archiving as there is no compaction yet on the metadata table"); instants = Stream.empty(); + } else { + LOG.info("Limiting archiving of instants to latest compaction on metadata table at " + latestCompactionTime.get()); + instants = instants.filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.LESSER_THAN, + latestCompactionTime.get())); } } catch (Exception e) { throw new HoodieException("Error limiting instant archival based on metadata table", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index 6c776cfb077ae..73083cdecabd3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -21,7 +21,11 @@ import java.io.Serializable; import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -46,4 +50,36 @@ public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, } public abstract R execute(); + + /** + * Writes commits metadata to table metadata. + * @param metadata commit metadata of interest. + */ + protected final void writeTableMetadata(HoodieCommitMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } + + /** + * Writes clean metadata to table metadata. + * @param metadata clean metadata of interest. + */ + protected final void writeTableMetadata(HoodieCleanMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } + + /** + * Writes rollback metadata to table metadata. + * @param metadata rollback metadata of interest. + */ + protected final void writeTableMetadata(HoodieRollbackMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } + + /** + * Writes restore metadata to table metadata. + * @param metadata restore metadata of interest. + */ + protected final void writeTableMetadata(HoodieRestoreMetadata metadata) { + table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime)); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index a5a72d4d08787..abe88b91b4144 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -24,6 +24,7 @@ import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.CleanFileInfo; @@ -58,9 +59,11 @@ public class CleanActionExecutor extends private static final long serialVersionUID = 1L; private static final Logger LOG = LogManager.getLogger(CleanActionExecutor.class); + private final TransactionManager txnManager; public CleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { super(context, config, table, instantTime); + this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); } static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException { @@ -196,7 +199,7 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan Option.of(timer.endTimer()), cleanStats ); - + writeMetadata(metadata); table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant, TimelineMetadataUtils.serializeCleanMetadata(metadata)); LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete"); @@ -206,6 +209,19 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan } } + /** + * Update metadata table if available. Any update to metadata table happens within data table lock. + * @param cleanMetadata intance of {@link HoodieCleanMetadata} to be applied to metadata. + */ + private void writeMetadata(HoodieCleanMetadata cleanMetadata) { + try { + this.txnManager.beginTransaction(Option.empty(), Option.empty()); + writeTableMetadata(cleanMetadata); + } finally { + this.txnManager.endTransaction(); + } + } + @Override public HoodieCleanMetadata execute() { List cleanMetadataList = new ArrayList<>(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 4b519ed92b4aa..ce6ed5db303c7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -175,10 +175,6 @@ protected void finalizeWrite(String instantTime, List stats, Ho } } - protected void syncTableMetadata() { - // No Op - } - /** * By default, return the writer schema in Write Config for storing in commit. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 3b722a7a41ac9..8b0085c829df3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -27,6 +28,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.table.HoodieTable; @@ -46,6 +48,7 @@ public abstract class BaseRestoreActionExecutor executeRollback(HoodieInstant instantToRollba protected void finishRollback(HoodieInstant inflightInstant, HoodieRollbackMetadata rollbackMetadata) throws HoodieIOException { try { + writeToMetadata(rollbackMetadata); table.getActiveTimeline().transitionRollbackInflightToComplete(inflightInstant, TimelineMetadataUtils.serializeRollbackMetadata(rollbackMetadata)); LOG.info("Rollback of Commits " + rollbackMetadata.getCommitsRollback() + " is complete"); @@ -256,6 +260,19 @@ protected void finishRollback(HoodieInstant inflightInstant, HoodieRollbackMetad } } + /** + * Update metadata table if available. Any update to metadata table happens within data table lock. + * @param rollbackMetadata intance of {@link HoodieRollbackMetadata} to be applied to metadata. + */ + private void writeToMetadata(HoodieRollbackMetadata rollbackMetadata) { + try { + this.txnManager.beginTransaction(Option.empty(), Option.empty()); + writeTableMetadata(rollbackMetadata); + } finally { + this.txnManager.endTransaction(); + } + } + /** * Delete Inflight instant if enabled. * diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java index c6a1527e292b0..702a84a97b300 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java @@ -28,6 +28,7 @@ import org.apache.hudi.exception.HoodieLockException; import java.io.IOException; +import java.io.Serializable; import java.util.concurrent.TimeUnit; import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; @@ -39,12 +40,12 @@ * create operation. This lock does not support cleaning/expiring the lock after a failed write hence cannot be used * in production environments. */ -public class FileSystemBasedLockProviderTestClass implements LockProvider { +public class FileSystemBasedLockProviderTestClass implements LockProvider, Serializable { private static final String LOCK_NAME = "acquired"; private String lockPath; - private FileSystem fs; + private transient FileSystem fs; protected LockConfiguration lockConfiguration; public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfiguration, final Configuration configuration) { @@ -55,7 +56,7 @@ public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfigur public void acquireLock() { try { - fs.create(new Path(lockPath + "/" + LOCK_NAME)).close(); + fs.create(new Path(lockPath + "/" + LOCK_NAME), false).close(); } catch (IOException e) { throw new HoodieIOException("Failed to acquire lock", e); } @@ -78,7 +79,12 @@ public boolean tryLock(long time, TimeUnit unit) { && (numRetries <= lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY))) { Thread.sleep(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY)); } - acquireLock(); + synchronized (LOCK_NAME) { + if (fs.exists(new Path(lockPath + "/" + LOCK_NAME))) { + return false; + } + acquireLock(); + } return true; } catch (IOException | InterruptedException e) { throw new HoodieLockException("Failed to acquire lock", e); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java new file mode 100644 index 0000000000000..801c8463b13d2 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java @@ -0,0 +1,143 @@ +/* + * 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.testutils; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; + +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * {@link HoodieTestTable} impl used for testing metadata. This class does synchronous updates to HoodieTableMetadataWriter if non null. + */ +public class HoodieMetadataTestTable extends HoodieTestTable { + + private HoodieTableMetadataWriter writer; + + protected HoodieMetadataTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, HoodieTableMetadataWriter writer) { + super(basePath, fs, metaClient); + this.writer = writer; + } + + public static HoodieTestTable of(HoodieTableMetaClient metaClient) { + return HoodieMetadataTestTable.of(metaClient, null); + } + + public static HoodieTestTable of(HoodieTableMetaClient metaClient, HoodieTableMetadataWriter writer) { + testTableState = HoodieTestTableState.of(); + return new HoodieMetadataTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, writer); + } + + @Override + public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationType operationType, + List newPartitionsToAdd, List partitions, + int filesPerPartition, boolean bootstrap, boolean createInflightCommit) throws Exception { + HoodieCommitMetadata commitMetadata = super.doWriteOperation(commitTime, operationType, newPartitionsToAdd, partitions, filesPerPartition, bootstrap, createInflightCommit); + if (writer != null && !createInflightCommit) { + writer.update(commitMetadata, commitTime); + } + return commitMetadata; + } + + @Override + public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { + super.moveInflightCommitToComplete(instantTime, metadata); + if (writer != null) { + writer.update(metadata, instantTime); + } + return this; + } + + public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata, boolean ignoreWriter) throws IOException { + super.moveInflightCommitToComplete(instantTime, metadata); + if (!ignoreWriter && writer != null) { + writer.update(metadata, instantTime); + } + return this; + } + + @Override + public HoodieTestTable moveInflightCompactionToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { + super.moveInflightCompactionToComplete(instantTime, metadata); + if (writer != null) { + writer.update(metadata, instantTime); + } + return this; + } + + @Override + public HoodieCleanMetadata doClean(String commitTime, Map partitionFileCountsToDelete) throws IOException { + HoodieCleanMetadata cleanMetadata = super.doClean(commitTime, partitionFileCountsToDelete); + if (writer != null) { + writer.update(cleanMetadata, commitTime); + } + return cleanMetadata; + } + + public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception { + super.addCompaction(instantTime, commitMetadata); + if (writer != null) { + writer.update(commitMetadata, instantTime); + } + return this; + } + + @Override + public HoodieTestTable addRollback(String instantTime, HoodieRollbackMetadata rollbackMetadata) throws IOException { + super.addRollback(instantTime, rollbackMetadata); + if (writer != null) { + writer.update(rollbackMetadata, instantTime); + } + return this; + } + + @Override + public HoodieTestTable addRestore(String instantTime, HoodieRestoreMetadata restoreMetadata) throws IOException { + super.addRestore(instantTime, restoreMetadata); + if (writer != null) { + writer.update(restoreMetadata, instantTime); + } + return this; + } + + @Override + public HoodieTestTable addReplaceCommit( + String instantTime, + Option requestedReplaceMetadata, + Option inflightReplaceMetadata, + HoodieReplaceCommitMetadata completeReplaceMetadata) throws Exception { + super.addReplaceCommit(instantTime, requestedReplaceMetadata, inflightReplaceMetadata, completeReplaceMetadata); + if (writer != null) { + writer.update(completeReplaceMetadata, instantTime); + } + return this; + } +} 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 7140504ebcf55..e279940b66dd5 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 @@ -42,7 +42,6 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.FlinkHoodieIndex; import org.apache.hudi.index.HoodieIndex; @@ -52,8 +51,6 @@ import org.apache.hudi.io.FlinkMergeHandle; import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.io.MiniBatchHandle; -import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; @@ -390,16 +387,6 @@ protected HoodieTable>, List, List records, String partitionName, String i // 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))); + m.updateSizeMetrics(metadataMetaClient, metadata); } catch (HoodieIOException e) { LOG.error("Could not publish metadata size metrics", e); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java index 5cfd28be2c1dc..fce159ec8a408 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -141,13 +141,14 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - syncTableMetadata(); try { LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + writeTableMetadata(metadata); + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java index 67376aef587af..832db1d7d21cb 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java @@ -49,7 +49,9 @@ protected Map upgrade(HoodieTableVersion fromVersion, Ho return new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime); } else if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.TWO) { return new OneToTwoUpgradeHandler().upgrade(config, context, instantTime); - } else { + } else if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.THREE) { + return new TwoToThreeUpgradeHandler().upgrade(config, context, instantTime); + } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true); } } @@ -60,6 +62,8 @@ protected Map downgrade(HoodieTableVersion fromVersion, return new OneToZeroDowngradeHandler().downgrade(config, context, instantTime); } else if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.ONE) { return new TwoToOneDowngradeHandler().downgrade(config, context, instantTime); + } else if (fromVersion == HoodieTableVersion.THREE && toVersion == HoodieTableVersion.TWO) { + return new ThreeToTwoDowngradeHandler().downgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java new file mode 100644 index 0000000000000..e6b3c30293905 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.upgrade; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; + +import java.util.Collections; +import java.util.Map; + +/** + * Downgrade handler to assist in downgrading hoodie table from version 3 to 2. + */ +public class ThreeToTwoDowngradeHandler implements DowngradeHandler { + + @Override + public Map downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { + if (config.isMetadataTableEnabled()) { + // Metadata Table in version 3 is synchronous and in version 2 is asynchronous. Downgrading to asynchronous + // removes the checks in code to decide whether to use a LogBlock or not. Also, the schema for the + // table has been updated and is not forward compatible. Hence, we need to delete the table. + HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); + } + return Collections.emptyMap(); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java new file mode 100644 index 0000000000000..9f5644aefea36 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.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.table.upgrade; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; + +import java.util.Collections; +import java.util.Map; + +/** + * UpgradeHandler to assist in upgrading {@link org.apache.hudi.table.HoodieTable} from version 2 to 3. + */ +public class TwoToThreeUpgradeHandler implements UpgradeHandler { + @Override + public Map upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { + if (config.isMetadataTableEnabled()) { + // Metadata Table in version 2 is asynchronous and in version 3 is synchronous. Synchronous table will not + // sync any instants not already synced. So its simpler to re-bootstrap the table. Also, the schema for the + // table has been updated and is not backward compatible. + HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); + } + return Collections.emptyMap(); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index aec84a50e18af..e9abf277f6602 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -206,6 +206,8 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + writeTableMetadata(metadata); + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java index 1170f2f4eac4c..a52ab6e0f3d0c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertOverwriteTableCommitActionExecutor.java @@ -52,8 +52,7 @@ protected List getAllExistingFileIds(String partitionPath) { protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeResult) { Map> partitionToExistingFileIds = new HashMap<>(); List partitionPaths = FSUtils.getAllPartitionPaths(context, - table.getMetaClient().getBasePath(), config.isMetadataTableEnabled(), - config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()); + table.getMetaClient().getBasePath(), config.isMetadataTableEnabled(), config.shouldAssumeDatePartitioning()); if (partitionPaths != null && partitionPaths.size() > 0) { partitionToExistingFileIds = context.mapToPair(partitionPaths, diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 796d7b74a83c5..4a3f3d5bcef89 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; @@ -114,7 +115,8 @@ private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() { return HoodieWriteConfig.newBuilder() .withEngineType(EngineType.JAVA) .withPath(basePath) - .withSchema(SCHEMA.toString()); + .withSchema(SCHEMA.toString()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); } @Test 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 f3127cb364623..1c5bdf5ec7139 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 @@ -42,7 +42,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.metadata.HoodieTableMetadataWriter; @@ -79,7 +78,7 @@ public class SparkRDDWriteClient extends private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class); public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { - super(context, clientConfig); + this(context, clientConfig, Option.empty()); } @Deprecated @@ -96,6 +95,11 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeC public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, Option timelineService) { super(context, writeConfig, timelineService); + if (config.isMetadataTableEnabled()) { + // If the metadata table does not exist, it should be bootstrapped here + // TODO: Check if we can remove this requirement - auto bootstrap on commit + SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context); + } } /** @@ -299,12 +303,13 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD writeStats = writeStatuses.map(WriteStatus::getStat).collect(); + writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime)); + // commit to data table after committing to metadata table. finalizeWrite(table, compactionCommitTime, writeStats); LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata); WriteMarkersFactory.get(config.getMarkersType(), table, compactionCommitTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - if (compactionTimer != null) { long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); try { @@ -320,7 +325,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD compact(String compactionInstantTime, boolean shouldComplete) { - HoodieSparkTable table = HoodieSparkTable.create(config, context, config.isMetadataTableEnabled()); + HoodieSparkTable table = HoodieSparkTable.create(config, context, true); preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient()); HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); @@ -369,6 +374,7 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD s.getTotalWriteErrors() > 0L).map(s -> s.getFileId()).collect(Collectors.joining(","))); } + writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime)); finalizeWrite(table, clusteringCommitTime, writeStats); try { LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata); @@ -376,7 +382,7 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD>, JavaRDD, JavaRDD> table, HoodieCommitMetadata commitMetadata, + HoodieInstant hoodieInstant) { + try { + this.txnManager.beginTransaction(Option.of(hoodieInstant), Option.empty()); + // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a + // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. + table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp())); + } finally { + this.txnManager.endTransaction(); + } + } + @Override protected HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); @@ -445,24 +463,10 @@ private HoodieTable>, JavaRDD, JavaRDD ((HoodieTableMetadataWriter)w).update(metadata, instantTime)); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 7c12a9e001024..f512b8f98dcc0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -24,20 +24,15 @@ import org.apache.hudi.common.engine.HoodieEngineContext; 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.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; 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; @@ -46,8 +41,8 @@ import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; +import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { @@ -78,7 +73,7 @@ protected void initRegistry() { } @Override - protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) { + protected void initialize(HoodieEngineContext engineContext) { try { metrics.map(HoodieMetadataMetrics::registry).ifPresent(registry -> { if (registry instanceof DistributedRegistry) { @@ -88,7 +83,7 @@ protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClie }); if (enabled) { - bootstrapIfNeeded(engineContext, datasetMetaClient); + bootstrapIfNeeded(engineContext, dataMetaClient); } } catch (IOException e) { LOG.error("Failed to initialize metadata table. Disabling the writer.", e); @@ -99,83 +94,93 @@ protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClie @Override protected void commit(List records, String partitionName, String instantTime) { ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); - JavaRDD recordRDD = prepRecords(records, partitionName); + JavaRDD recordRDD = prepRecords(records, partitionName, 1); try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { - writeClient.startCommitWithTime(instantTime); + if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) { + // if this is a new commit being applied to metadata for the first time + writeClient.startCommitWithTime(instantTime); + } else { + // this code path refers to a re-attempted commit that got committed to metadata table, but failed in datatable. + // for eg, lets say compaction c1 on 1st attempt succeeded in metadata table and failed before committing to datatable. + // when retried again, data table will first rollback pending compaction. these will be applied to metadata table, but all changes + // are upserts to metadata table and so only a new delta commit will be created. + // once rollback is complete, compaction will be retried again, which will eventually hit this code block where the respective commit is + // already part of completed commit. So, we have to manually remove the completed instant and proceed. + // and it is for the same reason we enabled withAllowMultiWriteOnSameInstant for metadata table. + HoodieInstant alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)).lastInstant().get(); + HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant); + metadataMetaClient.reloadActiveTimeline(); + } 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"); + + // reload timeline + metadataMetaClient.reloadActiveTimeline(); + compactIfNecessary(writeClient, instantTime); + doClean(writeClient, instantTime); } // 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); - } - }); + metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata)); } /** - * Tag each record with the location. + * Perform a compaction on the Metadata Table. * - * Since we only read the latest base file in a partition, we tag the records with the instant time of the latest - * base file. + * Cases to be handled: + * 1. We cannot perform compaction if there are previous inflight operations on the dataset. This is because + * a compacted metadata base file at time Tx should represent all the actions on the dataset till time Tx. + * + * 2. In multi-writer scenario, a parallel operation with a greater instantTime may have completed creating a + * deltacommit. */ - 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"); - } + private void compactIfNecessary(SparkRDDWriteClient writeClient, String instantTime) { + String latestDeltacommitTime = metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant() + .get().getTimestamp(); + List pendingInstants = dataMetaClient.reloadActiveTimeline().filterInflightsAndRequested() + .findInstantsBefore(latestDeltacommitTime).getInstants().collect(Collectors.toList()); + + if (!pendingInstants.isEmpty()) { + LOG.info(String.format("Cannot compact metadata table as there are %d inflight instants before latest deltacommit %s: %s", + pendingInstants.size(), latestDeltacommitTime, Arrays.toString(pendingInstants.toArray()))); + return; } - 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(); + // Trigger 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. + final String compactionInstantTime = latestDeltacommitTime + "001"; + if (writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())) { + writeClient.compact(compactionInstantTime); } + } + + private void doClean(SparkRDDWriteClient writeClient, String instantTime) { + // Trigger cleaning 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. + writeClient.clean(instantTime + "002"); + } + + /** + * Tag each record with the location in the given partition. + * + * The record is tagged with respective file slice's location based on its record key. + */ + private JavaRDD prepRecords(List records, String partitionName, int numFileGroups) { + List fileSlices = HoodieTableMetadataUtil.loadPartitionFileGroupsWithLatestFileSlices(metadataMetaClient, partitionName); + ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups)); - return jsc.parallelize(records, 1).map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))); + JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext(); + return jsc.parallelize(records, 1).map(r -> { + FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups)); + r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId())); + return r; + }); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index e252cabba9052..a7b14be5f5c38 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -29,14 +29,24 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; +import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; +import java.io.IOException; + public abstract class HoodieSparkTable extends HoodieTable>, JavaRDD, JavaRDD> { + private boolean isMetadataAvailabilityUpdated = false; + private boolean isMetadataTableAvailable; + protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { super(config, context, metaClient); } @@ -85,4 +95,31 @@ public static HoodieSparkTable create(HoodieW protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return SparkHoodieIndex.createIndex(config); } + + /** + * Fetch instance of {@link HoodieTableMetadataWriter}. + * + * @return instance of {@link HoodieTableMetadataWriter} + */ + @Override + public Option getMetadataWriter() { + synchronized (this) { + if (!isMetadataAvailabilityUpdated) { + // this code assumes that if metadata availability is updated once it will not change. please revisit this logic if that's not the case. + // this is done to avoid repeated calls to fs.exists(). + try { + isMetadataTableAvailable = config.isMetadataTableEnabled() + && metaClient.getFs().exists(new Path(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()))); + } catch (IOException e) { + throw new HoodieMetadataException("Checking existence of metadata table failed", e); + } + isMetadataAvailabilityUpdated = true; + } + } + if (isMetadataTableAvailable) { + return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context)); + } else { + return Option.empty(); + } + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 821b3071e145c..457fdaee52ede 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -59,13 +59,10 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieKeyGeneratorException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.execution.SparkBoundedInMemoryExecutor; import org.apache.hudi.io.HoodieBootstrapHandle; import org.apache.hudi.keygen.KeyGeneratorInterface; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -226,17 +223,6 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta LOG.info("Committing metadata bootstrap !!"); } - @Override - protected void syncTableMetadata() { - // Open up the metadata table again, for syncing - try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { - LOG.info("Successfully synced to metadata table"); - } catch (Exception e) { - throw new HoodieMetadataException("Error syncing to metadata table.", e); - } - } - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { String actionType = table.getMetaClient().getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); @@ -252,7 +238,6 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta // Finalize write finalizeWrite(instantTime, stats, result); - syncTableMetadata(); // add in extra metadata if (extraMetadata.isPresent()) { extraMetadata.get().forEach(metadata::addMetadata); @@ -260,6 +245,8 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); metadata.setOperationType(operationType); + writeTableMetadata(metadata); + try { activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 2bc1f0302798e..1935a3e5c56bc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -40,7 +40,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.SparkLazyInsertIterable; import org.apache.hudi.io.CreateHandleFactory; @@ -49,8 +48,6 @@ import org.apache.hudi.io.storage.HoodieConcatHandle; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; -import org.apache.hudi.metadata.HoodieTableMetadataWriter; -import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -239,7 +236,7 @@ protected JavaRDD updateIndex(JavaRDD writeStatusRDD, result.setWriteStatuses(statuses); return statuses; } - + protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { updateIndex(writeStatusRDD, result); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); @@ -264,13 +261,11 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta result.setWriteStats(writeStats); // Finalize write finalizeWrite(instantTime, writeStats, result); - syncTableMetadata(); try { - LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); - + writeTableMetadata(metadata); activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); @@ -354,17 +349,6 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, } } - @Override - public void syncTableMetadata() { - // Open up the metadata table again, for syncing - try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { - LOG.info("Successfully synced to metadata table"); - } catch (Exception e) { - throw new HoodieMetadataException("Error syncing to metadata table.", e); - } - } - @Override public Iterator> handleInsert(String idPfx, Iterator> recordItr) throws Exception { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java index 1f4c2038283bc..7fb286eb125a0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToTwoUpgradeHandler.java @@ -30,4 +30,4 @@ public class OneToTwoUpgradeHandler extends BaseOneToTwoUpgradeHandler { String getPartitionColumns(HoodieWriteConfig config) { return HoodieSparkUtils.getPartitionColumns(config.getProps()); } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java index 7284db5df4293..83f29b544aee2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java @@ -53,6 +53,8 @@ protected Map upgrade(HoodieTableVersion fromVersion, Ho return new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime); } else if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.TWO) { return new OneToTwoUpgradeHandler().upgrade(config, context, instantTime); + } else if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.THREE) { + return new TwoToThreeUpgradeHandler().upgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true); } @@ -64,6 +66,8 @@ protected Map downgrade(HoodieTableVersion fromVersion, return new OneToZeroDowngradeHandler().downgrade(config, context, instantTime); } else if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.ONE) { return new TwoToOneDowngradeHandler().downgrade(config, context, instantTime); + } else if (fromVersion == HoodieTableVersion.THREE && toVersion == HoodieTableVersion.TWO) { + return new ThreeToTwoDowngradeHandler().downgrade(config, context, instantTime); } else { throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java new file mode 100644 index 0000000000000..9211144d0c889 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ThreeToTwoDowngradeHandler.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.upgrade; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; + +import java.util.Collections; +import java.util.Map; + +/** + * Downgrade handler to assist in downgrading hoodie table from version 3 to 2. + */ +public class ThreeToTwoDowngradeHandler implements DowngradeHandler { + + @Override + public Map downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { + if (config.isMetadataTableEnabled()) { + // Metadata Table in version 3 is synchronous and in version 2 is asynchronous. Downgrading to asynchronous + // removes the checks in code to decide whether to use a LogBlock or not. Also, the schema for the + // table has been updated and is not forward compatible. Hence, we need to delete the table. + HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); + } + return Collections.emptyMap(); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java new file mode 100644 index 0000000000000..278e413849c22 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.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.table.upgrade; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; + +import java.util.Collections; +import java.util.Map; + +/** + * UpgradeHandler to assist in upgrading {@link org.apache.hudi.table.HoodieTable} from version 2 to 3. + */ +public class TwoToThreeUpgradeHandler implements UpgradeHandler { + @Override + public Map upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) { + if (config.isMetadataTableEnabled()) { + // Metadata Table in version 2 is asynchronous and in version 3 is synchronous. Synchronous table will not + // sync any instants not already synced. So its simpler to re-bootstrap the table. Also, the schema for the + // table has been updated and is not backward compatible. + HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); + } + return Collections.emptyMap(); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index c7599aac0c83d..c70a2cf6a5832 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -23,6 +23,7 @@ import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; +import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; @@ -42,6 +43,8 @@ import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -123,15 +126,27 @@ public void testHoodieClientBasicMultiWriter(HoodieTableType tableType) throws E } } - @ParameterizedTest - @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) - public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception { + @Disabled + public void testMultiWriterWithAsyncTableServicesWithConflictCOW() throws Exception { + testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.COPY_ON_WRITE); + } + + @Test + public void testMultiWriterWithAsyncTableServicesWithConflictMOR() throws Exception { + testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.MERGE_ON_READ); + } + + private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception { // create inserts X 1 if (tableType == HoodieTableType.MERGE_ON_READ) { setUpMORTestTable(); } Properties properties = new Properties(); properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000"); // Disabling embedded timeline server, it doesn't work with multiwriter HoodieWriteConfig cfg = getConfigBuilder() .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java index 83761c985b040..457b8b526aa04 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java @@ -18,6 +18,7 @@ package org.apache.hudi.client; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieRecord; @@ -30,6 +31,7 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieClientTestUtils; @@ -67,9 +69,15 @@ public void tearDown() throws Exception { } protected HoodieWriteConfig getHoodieWriteConfig(String basePath) { + return getHoodieWriteConfig(basePath, HoodieMetadataConfig.ENABLE.defaultValue()); + } + + protected HoodieWriteConfig getHoodieWriteConfig(String basePath, boolean enableMetadata) { return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable(tableName) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()) + .build(); } @Test @@ -82,8 +90,17 @@ public void readLocalWriteHDFS() throws Exception { .initTable(hadoopConf, dfsBasePath); // Create write client to write some records in - HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath); - HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath); + HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath, false); + HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath, false); + + HoodieTableMetaClient.withPropertyBuilder() + .setTableType(tableType) + .setTableName(tableName) + .setPayloadClass(HoodieAvroPayload.class) + .setRecordKeyFields(localConfig.getProps().getProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())) + .setPartitionFields(localConfig.getProps().getProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())) + .initTable(hadoopConf, tablePath); + try (SparkRDDWriteClient hdfsWriteClient = getHoodieWriteClient(cfg); SparkRDDWriteClient localWriteClient = getHoodieWriteClient(localConfig)) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java index 3ad777475f496..bd0961d227476 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java @@ -21,6 +21,7 @@ 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.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -335,7 +336,8 @@ public void testTagLocationAndPartitionPathUpdateWithExplicitRollback() throws E @Test public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { // Load to memory - HoodieWriteConfig config = getConfig(); + HoodieWriteConfig config = getConfigBuilder(100, false, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); @@ -422,7 +424,8 @@ public void testSimpleTagLocationWithInvalidCommit() throws Exception { @Test public void testEnsureTagLocationUsesCommitTimeline() throws Exception { // Load to memory - HoodieWriteConfig config = getConfig(); + HoodieWriteConfig config = getConfigBuilder(100, false, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); SparkRDDWriteClient writeClient = getHoodieWriteClient(config); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index ea59738143997..f7cb22cda1cb8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -21,8 +21,11 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.LockConfiguration; 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.metrics.Registry; import org.apache.hudi.common.model.FileSlice; @@ -30,14 +33,20 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieFileGroupId; 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.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.FileCreateUtils; @@ -45,12 +54,12 @@ 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.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.metrics.HoodieMetricsConfig; +import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.FileSystemBackedTableMetadata; @@ -61,18 +70,19 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.Time; 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.Assertions; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; @@ -83,17 +93,23 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.stream.Collectors; import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; +import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; -import static org.apache.hudi.common.model.WriteOperationType.BULK_INSERT; import static org.apache.hudi.common.model.WriteOperationType.DELETE; import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; @@ -105,33 +121,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @Tag("functional") -public class TestHoodieBackedMetadata extends HoodieClientTestHarness { +public class TestHoodieBackedMetadata extends TestHoodieMetadataBase { private static final Logger LOG = LogManager.getLogger(TestHoodieBackedMetadata.class); - private static HoodieTestTable testTable; - private String metadataTableBasePath; - private HoodieTableType tableType; - private HoodieWriteConfig writeConfig; - - 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); - writeConfig = getWriteConfig(true, true); - testTable = HoodieTestTable.of(metaClient); - } - - @AfterEach - public void clean() throws IOException { - cleanupResources(); - } - public static List bootstrapAndTableOperationTestArgs() { return asList( Arguments.of(COPY_ON_WRITE, true), @@ -147,26 +140,29 @@ public static List bootstrapAndTableOperationTestArgs() { @ParameterizedTest @MethodSource("bootstrapAndTableOperationTestArgs") public void testMetadataTableBootstrap(HoodieTableType tableType, boolean addRollback) throws Exception { - init(tableType); + init(tableType, false); // bootstrap with few commits - doWriteOperationsAndBootstrapMetadata(testTable); + doPreBootstrapOperations(testTable); + writeConfig = getWriteConfig(true, true); + initWriteConfigAndMetatableWriter(writeConfig, true); + syncTableMetadata(writeConfig); + validateMetadata(testTable); + doWriteInsertAndUpsert(testTable); + validateMetadata(testTable); if (addRollback) { // trigger an UPSERT that will be rolled back - testTable.doWriteOperation("003", UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 2); - syncTableMetadata(writeConfig); + doWriteOperationAndValidate(testTable, "0000003"); + // rollback last commit - testTable = testTable.doRollback("003", "004"); - syncAndValidate(testTable); + doRollbackAndValidate(testTable, "0000003", "0000004"); } - testTable.doWriteOperation("005", INSERT, asList("p1", "p2"), 4); - syncAndValidate(testTable); + doWriteOperationAndValidate(testTable, "0000005"); // trigger an upsert and validate - testTable.doWriteOperation("006", UPSERT, singletonList("p3"), - asList("p1", "p2", "p3"), 4); - syncAndValidate(testTable, true); + doWriteOperation(testTable, "0000006"); + validateMetadata(testTable, true); } /** @@ -176,7 +172,7 @@ public void testMetadataTableBootstrap(HoodieTableType tableType, boolean addRol @EnumSource(HoodieTableType.class) public void testOnlyValidPartitionsAdded(HoodieTableType tableType) throws Exception { // This test requires local file system - init(tableType); + init(tableType, false); // 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)); @@ -189,12 +185,12 @@ public void testOnlyValidPartitionsAdded(HoodieTableType tableType) throws Excep // Create some commits testTable.withPartitionMetaFiles("p1", "p2", filteredDirectoryOne, filteredDirectoryTwo, filteredDirectoryThree) - .addCommit("001").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10) - .addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10); + .addCommit("0000001").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10) + .addCommit("0000002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10); writeConfig = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withDirectoryFilterRegex(filterDirRegex).build()).build(); - testTable.doWriteOperation("003", UPSERT, emptyList(), asList("p1", "p2"), 1, true); + testTable.doWriteOperation("0000003", UPSERT, emptyList(), asList("p1", "p2"), 1, true); syncTableMetadata(writeConfig); List partitions = metadataWriter(writeConfig).metadata().getAllPartitionPaths(); @@ -224,199 +220,116 @@ public void testOnlyValidPartitionsAdded(HoodieTableType tableType) throws Excep * Test various table operations sync to Metadata Table correctly. */ @ParameterizedTest - @MethodSource("bootstrapAndTableOperationTestArgs") - public void testTableOperations(HoodieTableType tableType, boolean doNotSyncFewCommits) throws Exception { + @EnumSource(HoodieTableType.class) + public void testTableOperations(HoodieTableType tableType) throws Exception { init(tableType); - // bootstrap w/ 2 commits - doWriteOperationsAndBootstrapMetadata(testTable); + doWriteInsertAndUpsert(testTable); // trigger an upsert - testTable.doWriteOperation("003", UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 3); - syncAndValidate(testTable); + doWriteOperationAndValidate(testTable, "0000003"); // trigger compaction if (MERGE_ON_READ.equals(tableType)) { - testTable = testTable.doCompaction("004", asList("p1", "p2")); - syncAndValidate(testTable); + doCompactionAndValidate(testTable, "0000004"); } // trigger an upsert - testTable.doWriteOperation("005", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - if (doNotSyncFewCommits) { - syncAndValidate(testTable, emptyList(), true, false, true); - } + doWriteOperationAndValidate(testTable, "0000005"); // trigger clean - testTable.doCleanBasedOnCommits("006", singletonList("001")); - if (doNotSyncFewCommits) { - syncAndValidate(testTable, emptyList(), true, false, false); - } + doCleanAndValidate(testTable, "0000006", singletonList("0000001")); - // trigger delete - testTable.doWriteOperation("007", DELETE, emptyList(), asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable, emptyList(), true, true, false); + // trigger few upserts and validate + doWriteOperation(testTable, "0000007"); + doWriteOperation(testTable, "0000008"); + validateMetadata(testTable, emptyList(), true); } - /** - * Test several table operations with restore. This test uses SparkRDDWriteClient. - * Once the restore support is ready in HoodieTestTable, then rewrite this test. - */ @ParameterizedTest @EnumSource(HoodieTableType.class) - public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exception { + public void testMetadataInsertUpsertClean(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); + doWriteOperation(testTable, "0000001", INSERT); + doWriteOperation(testTable, "0000002"); + doCleanAndValidate(testTable, "0000003", Arrays.asList("0000001")); + if (tableType == MERGE_ON_READ) { + doCompactionAndValidate(testTable, "0000004"); + } + doWriteOperation(testTable, "0000005"); + validateMetadata(testTable, emptyList(), true); + } - // Restore - client.restoreToInstant("006"); - validateMetadata(client); + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testInsertUpsertCluster(HoodieTableType tableType) throws Exception { + init(tableType); + doWriteOperation(testTable,"0000001", INSERT); + doWriteOperation(testTable, "0000002"); + doClusterAndValidate(testTable, "0000003"); + if (tableType == MERGE_ON_READ) { + doCompaction(testTable, "0000004"); } + validateMetadata(testTable, emptyList(), true); } /** - * Tests rollback of a commit with metadata enabled. + * Test rollback of various table operations sync to Metadata Table correctly. */ @ParameterizedTest @EnumSource(HoodieTableType.class) public void testRollbackOperations(HoodieTableType tableType) throws Exception { init(tableType); - // bootstrap w/ 2 commits - doWriteOperationsAndBootstrapMetadata(testTable); + doWriteInsertAndUpsert(testTable); // trigger an upsert - testTable.doWriteOperation("003", UPSERT, emptyList(), asList("p1", "p2"), 2); - syncAndValidate(testTable); + doWriteOperationAndValidate(testTable, "0000003"); // trigger a commit and rollback - testTable.doWriteOperation("004", UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 3); - syncTableMetadata(writeConfig); - // rollback last commit - testTable = testTable.doRollback("004", "005"); - syncAndValidate(testTable); + doWriteOperationAndValidate(testTable, "0000004"); + doRollbackAndValidate(testTable, "0000004", "0000005"); // trigger few upserts and validate for (int i = 6; i < 10; i++) { - testTable.doWriteOperation("00" + i, UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); + doWriteOperation(testTable, "000000" + i); } - syncAndValidate(testTable); + validateMetadata(testTable); - testTable.doWriteOperation("010", UPSERT, emptyList(), asList("p1", "p2", "p3"), 3); - syncAndValidate(testTable); + doWriteOperationAndValidate(testTable, "0000010"); - // rollback last commit. sync and validate. - testTable.doRollback("010", "011"); - syncTableMetadata(writeConfig); + // rollback last commit. and validate. + doRollbackAndValidate(testTable, "0000010", "0000011"); // rollback of compaction if (MERGE_ON_READ.equals(tableType)) { - testTable = testTable.doCompaction("012", asList("p1", "p2")); - syncTableMetadata(writeConfig); - testTable.doRollback("012", "013"); - syncTableMetadata(writeConfig); + doCompactionAndValidate(testTable, "0000012"); + doRollbackAndValidate(testTable, "0000012", "0000013"); } // roll back of delete - testTable.doWriteOperation("014", DELETE, emptyList(), asList("p1", "p2", "p3"), 2); - syncTableMetadata(writeConfig); - testTable.doRollback("014", "015"); - syncTableMetadata(writeConfig); + doWriteOperationAndValidate(testTable, "0000014", DELETE); + doRollbackAndValidate(testTable, "0000014", "0000015"); // rollback partial commit writeConfig = getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).build(); - testTable.doWriteOperation("016", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - testTable.doRollback("016", "017"); - syncTableMetadata(writeConfig); + doWriteOperation(testTable, "0000016"); + testTable.doRollback("0000016", "0000017"); + validateMetadata(testTable); // marker-based rollback of partial commit writeConfig = getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(true).build(); - testTable.doWriteOperation("018", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - testTable.doRollback("018", "019"); - syncAndValidate(testTable, true); + doWriteOperation(testTable, "0000018"); + testTable.doRollback("0000018", "0000019"); + validateMetadata(testTable, true); } /** * Test that manual rollbacks work correctly and enough timeline history is maintained on the metadata table * timeline. */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testManualRollbacks(HoodieTableType tableType) throws Exception { - init(tableType); - doWriteOperationsAndBootstrapMetadata(testTable); - + @Test + public void testManualRollbacks() throws Exception { + HoodieTableType tableType = COPY_ON_WRITE; + init(tableType, false); // Setting to archive more aggressively on the Metadata Table than the Dataset final int maxDeltaCommitsBeforeCompaction = 4; final int minArchiveCommitsMetadata = 2; @@ -428,25 +341,24 @@ public void testManualRollbacks(HoodieTableType tableType) throws Exception { .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveCommitsDataset, minArchiveCommitsDataset + 1) .retainCommits(1).retainFileVersions(1).withAutoClean(false).withAsyncClean(true).build()) .build(); + + initWriteConfigAndMetatableWriter(writeConfig, true); + doWriteInsertAndUpsert(testTable, "000001", "000002"); + for (int i = 3; i < 10; i++) { - if (i == 3) { - testTable.doWriteOperation("00" + i, UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 2); - syncTableMetadata(writeConfig); - } else { - testTable.doWriteOperation("00" + i, UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - } + doWriteOperation(testTable, "00000" + i); + archiveDataTable(writeConfig, metaClient); } - syncAndValidate(testTable, true); + validateMetadata(testTable); // We can only rollback those commits whose deltacommit have not been archived yet. int numRollbacks = 0; boolean exceptionRaised = false; - List allInstants = metaClient.reloadActiveTimeline().getCommitsTimeline().getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant instantToRollback : allInstants) { try { testTable.doRollback(instantToRollback.getTimestamp(), String.valueOf(Time.now())); - syncTableMetadata(writeConfig); + validateMetadata(testTable); ++numRollbacks; } catch (HoodieMetadataException e) { exceptionRaised = true; @@ -467,350 +379,657 @@ public void testManualRollbacks(HoodieTableType tableType) throws Exception { @ParameterizedTest @EnumSource(HoodieTableType.class) public void testSync(HoodieTableType tableType) throws Exception { - init(tableType); + init(tableType, false); // Initial commits without metadata table enabled writeConfig = getWriteConfigBuilder(true, false, false).build(); - testTable.doWriteOperation("001", BULK_INSERT, asList("p1", "p2"), asList("p1", "p2"), 1); - testTable.doWriteOperation("002", BULK_INSERT, asList("p1", "p2"), 1); + doPreBootstrapOperations(testTable, "00000001", "00000002"); + // Enable metadata table so it initialized by listing from file system - testTable.doWriteOperation("003", INSERT, asList("p1", "p2"), 1); - syncAndValidate(testTable, emptyList(), true, true, true); - // Various table operations without metadata table enabled - testTable.doWriteOperation("004", UPSERT, asList("p1", "p2"), 1); - testTable.doWriteOperation("005", UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 3); - syncAndValidate(testTable, emptyList(), false, true, true); + writeConfig = getWriteConfigBuilder(true, true, false).build(); + + initWriteConfigAndMetatableWriter(writeConfig, true); + syncTableMetadata(writeConfig); + validateMetadata(testTable); + + doWriteOperationAndValidate(testTable, "00000003", INSERT); + doWriteOperationAndValidate(testTable, "00000004", UPSERT); + doWriteOperationAndValidate(testTable, "00000005", UPSERT); // trigger compaction if (MERGE_ON_READ.equals(tableType)) { - testTable = testTable.doCompaction("006", asList("p1", "p2")); - syncAndValidate(testTable, emptyList(), false, true, true); + doCompactionAndValidate(testTable, "00000006"); } // trigger an upsert - testTable.doWriteOperation("008", UPSERT, asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable, emptyList(), false, true, true); - - // savepoint - if (COPY_ON_WRITE.equals(tableType)) { - testTable.doSavepoint("008"); - syncAndValidate(testTable, emptyList(), false, true, true); - } - + doWriteOperationAndValidate(testTable, "00000008"); // trigger delete - testTable.doWriteOperation("009", DELETE, emptyList(), asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable, emptyList(), false, true, true); - + doWriteOperationAndValidate(testTable, "00000009", DELETE); // trigger clean - testTable.doCleanBasedOnCommits("010", asList("001", "002")); - syncAndValidate(testTable, emptyList(), false, true, true); - + doCleanAndValidate(testTable, "00000010", asList("00000003", "00000004")); // trigger another upsert - testTable.doWriteOperation("011", UPSERT, asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable, emptyList(), false, true, true); - + doWriteOperationAndValidate(testTable, "00000011"); // trigger clustering - testTable.doCluster("012", new HashMap<>()); - syncAndValidate(testTable, emptyList(), false, true, true); + doClusterAndValidate(testTable, "00000012"); // If there is an inflight operation, the Metadata Table is not updated beyond that operations but the // in-memory merge should consider all the completed operations. - HoodieCommitMetadata inflightCommitMeta = testTable.doWriteOperation("007", UPSERT, emptyList(), - asList("p1", "p2", "p3"), 2, false, true); + HoodieCommitMetadata inflightCommitMeta = testTable.doWriteOperation("00000007", UPSERT, emptyList(), + asList("p1", "p2"), 2, false, true); // trigger upsert - testTable.doWriteOperation("013", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); + doWriteOperation(testTable, "00000013"); // testTable validation will fetch only files pertaining to completed commits. So, validateMetadata() will skip files for 007 // while validating against actual metadata table. - syncAndValidate(testTable, singletonList("007"), true, true, false); + validateMetadata(testTable, singletonList("00000007")); + // Remove the inflight instance holding back table sync - testTable.moveInflightCommitToComplete("007", inflightCommitMeta); - syncTableMetadata(writeConfig); + testTable.moveInflightCommitToComplete("00000007", inflightCommitMeta); + validateMetadata(testTable); // A regular commit should get synced - testTable.doWriteOperation("014", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable, emptyList(), true, true, true); + doWriteOperation(testTable, "00000014"); + validateMetadata(testTable, emptyList(), true); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType) throws Exception { + init(tableType); + for (int i = 1; i < 25; i += 7) { + String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i; + String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1); + String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2); + String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3); + String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4); + String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5); + String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6); + doWriteOperation(testTable, commitTime1, INSERT); + doWriteOperation(testTable, commitTime2); + doClean(testTable, commitTime3, Arrays.asList(commitTime1)); + doWriteOperation(testTable, commitTime4); + if (tableType == MERGE_ON_READ) { + doCompaction(testTable, commitTime5); + } + doWriteOperation(testTable, commitTime6); + doRollback(testTable, commitTime6, commitTime7); + } + validateMetadata(testTable, emptyList(), true); + } + + // Some operations are not feasible with test table infra. hence using write client to test those cases. + + /** + * Test several table operations with restore. This test uses SparkRDDWriteClient. + * Once the restore support is ready in HoodieTestTable, then rewrite this test. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - /* TODO: Restore to savepoint, enable metadata table and ensure it is synced try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { - client.restoreToInstant(restoreToInstant); - assertFalse(metadata(client).isInSync()); - newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 1 (Bulk insert) + String newCommitTime = "0000001"; + 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 = "0000002"; + 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 = "0000003"; + 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 = "0000004"; client.startCommitWithTime(newCommitTime); - client.syncTableMetadata(); + 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 = "0000005"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Write 5 (updates and inserts) + newCommitTime = "0000006"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 5); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); validateMetadata(client); - assertTrue(metadata(client).isInSync()); - }*/ + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "0000007"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Deletes + newCommitTime = "0000009"; + 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 = "0000009"; + client.clean(newCommitTime); + validateMetadata(client); + + // Restore + client.restoreToInstant("0000006"); + validateMetadata(client); + } } /** - * Instants on Metadata Table should be archived as per config but we always keep atlest the number of instants - * as on the dataset. Metadata Table should be automatically compacted as per config. + * Test multi-writer on metadata table with optimistic concurrency. */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - @Disabled - public void testCleaningArchivingAndCompaction(HoodieTableType tableType) throws Exception { - init(tableType); - doWriteOperationsAndBootstrapMetadata(testTable); + @Test + public void testMetadataMultiWriter() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - final int maxDeltaCommitsBeforeCompaction = 4; - final int minArchiveLimit = 4; - final int maxArchiveLimit = 6; - writeConfig = getWriteConfigBuilder(true, true, false) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) - .archiveCommitsWith(minArchiveLimit - 2, maxArchiveLimit - 2).retainCommits(1) - .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(minArchiveLimit, maxArchiveLimit) - .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(true).build()) + Properties properties = new Properties(); + properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000"); + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class).build()) + .withProperties(properties) .build(); - for (int i = 3; i < 10; i++) { - if (i == 3) { - testTable.doWriteOperation("00" + i, UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 2); - syncTableMetadata(writeConfig); - } else { - testTable.doWriteOperation("00" + i, UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - } + + ExecutorService executors = Executors.newFixedThreadPool(dataGen.getPartitionPaths().length); + // Create clients in advance + SparkRDDWriteClient[] writeClients = new SparkRDDWriteClient[dataGen.getPartitionPaths().length]; + for (int i = 0; i < dataGen.getPartitionPaths().length; i++) { + writeClients[i] = new SparkRDDWriteClient(engineContext, writeConfig); } - syncAndValidate(testTable, true); + // Parallel commits for separate partitions + List futures = new LinkedList<>(); + for (int i = 0; i < dataGen.getPartitionPaths().length; ++i) { + final int index = i; + String newCommitTime = "000000" + (index + 1); + Future future = executors.submit(() -> { + List records = dataGen.generateInsertsForPartition(newCommitTime, 100, dataGen.getPartitionPaths()[index]); + SparkRDDWriteClient writeClient = writeClients[index]; + writeClient.startCommitWithTime(newCommitTime); + List writeStatuses = writeClient.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + }); + futures.add(future); + } + + // Wait for all commits to complete + for (Future future : futures) { + future.get(); + } + + // Ensure all commits were synced to the Metadata Table HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); - HoodieTableMetaClient datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(writeConfig.getBasePath()).build(); - HoodieActiveTimeline metadataTimeline = metadataMetaClient.getActiveTimeline(); - // check that there are compactions. - assertTrue(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants() > 0); - // check that cleaning has, once after each compaction. - assertTrue(metadataTimeline.getCleanerTimeline().filterCompletedInstants().countInstants() > 0); - // ensure archiving has happened - long numDataCompletedInstants = datasetMetaClient.getActiveTimeline().filterCompletedInstants().countInstants(); - long numDeltaCommits = metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(); - assertTrue(numDeltaCommits >= minArchiveLimit); - assertTrue(numDeltaCommits < numDataCompletedInstants, "Must have less delta commits than total completed instants on data timeline."); + assertEquals(metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 4); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000001"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000002"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000003"))); + + // Compaction may occur if the commits completed in order + assertTrue(metadataMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().countInstants() <= 1); + + // Validation + validateMetadata(writeClients[0]); } /** - * Test various error scenarios. + * Lets say clustering commit succeeded in metadata table, but failed before committing to datatable. + * Next time, when clustering kicks in, hudi will rollback pending clustering and re-attempt the clustering with same instant time. + * So, this test ensures the 2nd attempt succeeds with metadata enabled. + * This is applicable to any table service where instant time is fixed. So, how many ever times the operation fails, re attempt will + * be made with same commit time. + * Tests uses clustering to test out the scenario. */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testErrorCases(HoodieTableType tableType) throws Exception { + @Test + public void testReattemptOfFailedClusteringCommit() throws Exception { + tableType = HoodieTableType.COPY_ON_WRITE; init(tableType); - // 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. - testTable.doWriteOperation("001", UPSERT, asList("p1", "p2"), asList("p1", "p2"), 1); - syncAndValidate(testTable); - testTable.doWriteOperation("002", BULK_INSERT, emptyList(), asList("p1", "p2"), 1); - syncAndValidate(testTable); - // 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("002"); - assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, - commitInstantFileName), false)); - // Next upsert - testTable.doWriteOperation("003", UPSERT, emptyList(), asList("p1", "p2"), 1); - // Post rollback commit and metadata should be valid - syncTableMetadata(writeConfig); - HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); - HoodieActiveTimeline timeline = metadataMetaClient.getActiveTimeline(); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "001"))); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "003"))); + context = new HoodieSparkEngineContext(jsc); + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, TRIP_EXAMPLE_SCHEMA, 10, false); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // Write 1 (Bulk insert) + String newCommitTime = "0000001"; + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 2 (inserts) + newCommitTime = "0000002"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // setup clustering config. + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringSortColumns("_row_key") + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + + HoodieWriteConfig newWriteConfig = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER) + .withAutoCommit(false) + .withClusteringConfig(clusteringConfig).build(); + + // trigger clustering + SparkRDDWriteClient newClient = getHoodieWriteClient(newWriteConfig); + String clusteringCommitTime = newClient.scheduleClustering(Option.empty()).get().toString(); + HoodieWriteMetadata> clusterMetadata = newClient.cluster(clusteringCommitTime, true); + + // collect replaceFileIds for validation later. + Set replacedFileIds = new HashSet<>(); + clusterMetadata.getPartitionToReplaceFileIds().entrySet().forEach(partitionFiles -> + partitionFiles.getValue().stream().forEach(file -> + replacedFileIds.add(new HoodieFileGroupId(partitionFiles.getKey(), file)))); + + // trigger new write to mimic other writes succeeding before re-attempt. + newCommitTime = "0000003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // manually remove clustering completed instant from .hoodie folder and to mimic succeeded clustering in metadata table, but failed in data table. + FileCreateUtils.deleteReplaceCommit(basePath, clusteringCommitTime); + HoodieWriteMetadata> updatedClusterMetadata = newClient.cluster(clusteringCommitTime, true); + + metaClient.reloadActiveTimeline(); + Set updatedReplacedFileIds = new HashSet<>(); + updatedClusterMetadata.getPartitionToReplaceFileIds().entrySet().forEach(partitionFiles -> + partitionFiles.getValue().stream().forEach(file -> + updatedReplacedFileIds.add(new HoodieFileGroupId(partitionFiles.getKey(), file)))); + assertEquals(replacedFileIds, updatedReplacedFileIds); + validateMetadata(client); } /** - * Test non-partitioned datasets. + * Ensure that the reader only reads completed instants. + * + * @throws IOException */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testNonPartitioned(HoodieTableType tableType) throws Exception { - init(tableType); - // Non-partitioned bulk insert - testTable.doWriteOperation("001", BULK_INSERT, emptyList(), 1); - syncTableMetadata(writeConfig); - List metadataPartitions = metadata(writeConfig, context).getAllPartitionPaths(); - assertTrue(metadataPartitions.isEmpty(), "Must contain empty partition"); + @Test + public void testReader() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + List records; + List writeStatuses; + String[] commitTimestamps = {HoodieActiveTimeline.createNewInstantTime(), HoodieActiveTimeline.createNewInstantTime(), + HoodieActiveTimeline.createNewInstantTime(), HoodieActiveTimeline.createNewInstantTime()}; + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + for (int i = 0; i < commitTimestamps.length; ++i) { + records = dataGen.generateInserts(commitTimestamps[i], 5); + client.startCommitWithTime(commitTimestamps[i]); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamps[i]).collect(); + assertNoWriteErrors(writeStatuses); + } + + // Ensure we can see files from each commit + Set timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), commitTimestamps.length); + for (int i = 0; i < commitTimestamps.length; ++i) { + assertTrue(timelineTimestamps.contains(commitTimestamps[i])); + } + + // mark each commit as incomplete and ensure files are not seen + for (int i = 0; i < commitTimestamps.length; ++i) { + FileCreateUtils.deleteCommit(basePath, commitTimestamps[i]); + timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), commitTimestamps.length - 1); + for (int j = 0; j < commitTimestamps.length; ++j) { + assertTrue(j == i || timelineTimestamps.contains(commitTimestamps[j])); + } + FileCreateUtils.createCommit(basePath, commitTimestamps[i]); + } + + // Test multiple incomplete commits + FileCreateUtils.deleteCommit(basePath, commitTimestamps[0]); + FileCreateUtils.deleteCommit(basePath, commitTimestamps[2]); + timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), commitTimestamps.length - 2); + for (int j = 0; j < commitTimestamps.length; ++j) { + assertTrue(j == 0 || j == 2 || timelineTimestamps.contains(commitTimestamps[j])); + } + + // Test no completed commits + for (int i = 0; i < commitTimestamps.length; ++i) { + FileCreateUtils.deleteCommit(basePath, commitTimestamps[i]); + } + timelineTimestamps = getAllFiles(metadata(client)).stream().map(p -> p.getName()).map(n -> FSUtils.getCommitTime(n)).collect(Collectors.toSet()); + assertEquals(timelineTimestamps.size(), 0); + } } /** - * Test various metrics published by metadata table. + * Instants on Metadata Table should be archived as per config but we always keep atlest the number of instants + * as on the dataset. + *

+ * Metadata Table should be automatically compacted as per config. */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testMetadataMetrics(HoodieTableType tableType) throws Exception { - init(tableType); - writeConfig = getWriteConfigBuilder(true, true, true).build(); - testTable.doWriteOperation(HoodieActiveTimeline.createNewInstantTime(), INSERT, asList("p1", "p2"), - asList("p1", "p2"), 2, true); - syncTableMetadata(writeConfig); - Registry metricsRegistry = Registry.getRegistry("HoodieMetadata"); - assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count")); - assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration")); - assertTrue(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 + public void testCleaningArchivingAndCompaction() throws Exception { + init(HoodieTableType.COPY_ON_WRITE, false); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + final int maxDeltaCommitsBeforeCompaction = 3; + HoodieWriteConfig config = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) + .archiveCommitsWith(40, 60).retainCommits(1) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER) + .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(false).build()) + .build(); + + List records; + String newCommitTime; + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, config)) { + // Some initial commits so compaction is not triggered. + // 1 deltacommit will be from bootstrap. So we can perform maxDeltaCommitsBeforeCompaction - 2 more commits before + // compaction will be attempted. + for (int i = 0; i < maxDeltaCommitsBeforeCompaction - 2; ++i) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + } + + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); + HoodieTableMetaClient datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()).build(); + + // There should not be any compaction yet and we have not performed more than maxDeltaCommitsBeforeCompaction + // deltacommits (1 will be due to bootstrap) + HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 0); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), maxDeltaCommitsBeforeCompaction - 1); + assertEquals(datasetMetaClient.getArchivedTimeline().reload().countInstants(), 0); + + // Next commit will initiate a compaction + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 1); + assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), maxDeltaCommitsBeforeCompaction + 1); + assertEquals(datasetMetaClient.getArchivedTimeline().reload().countInstants(), 0); + + // More than maxDeltaCommitsBeforeCompaction commits + String inflightCommitTime = newCommitTime; + for (int i = 0; i < maxDeltaCommitsBeforeCompaction + 1; ++i) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + if (i == 0) { + // Mark this commit inflight so compactions dont take place + FileCreateUtils.deleteCommit(basePath, newCommitTime); + FileCreateUtils.createInflightCommit(basePath, newCommitTime); + inflightCommitTime = newCommitTime; + } + } + + // Ensure no more compactions took place due to the leftover inflight commit + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 1); + assertEquals(metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(), + ((2 * maxDeltaCommitsBeforeCompaction) + (maxDeltaCommitsBeforeCompaction /* clean from dataset */) + 1)/* clean in metadata table */); + + // Complete commit + FileCreateUtils.createCommit(basePath, inflightCommitTime); + + // Next commit should lead to compaction + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + + // Ensure compactions took place + metadataTimeline = metadataMetaClient.reloadActiveTimeline(); + assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 2); + assertEquals(metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(), + ((2 * maxDeltaCommitsBeforeCompaction) + (maxDeltaCommitsBeforeCompaction + 1 /* clean from dataset */) + 2 /* clean in metadata table */)); + assertTrue(datasetMetaClient.getArchivedTimeline().reload().countInstants() > 0); + + validateMetadata(client); + } } - /** - * Test when reading from metadata table which is out of sync with dataset that results are still consistent. - */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testMetadataOutOfSync(HoodieTableType tableType) throws Exception { - init(tableType); - testTable.doWriteOperation("001", BULK_INSERT, asList("p1", "p2"), asList("p1", "p2"), 1); - // Enable metadata so table is initialized but do not sync - syncAndValidate(testTable, emptyList(), true, false, false); - // Perform an insert and upsert - testTable.doWriteOperation("002", INSERT, asList("p1", "p2"), 1); - testTable.doWriteOperation("003", UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 1); - // Run compaction for MOR table - if (MERGE_ON_READ.equals(tableType)) { - testTable = testTable.doCompaction("004", asList("p1", "p2")); + @Test + public void testUpgradeDowngrade() throws IOException { + init(HoodieTableType.COPY_ON_WRITE, false); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Perform a commit. This should bootstrap the metadata table with latest version. + List records; + List writeStatuses; + String commitTimestamp = HoodieActiveTimeline.createNewInstantTime(); + HoodieWriteConfig writeConfig = getWriteConfig(true, true); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + records = dataGen.generateInserts(commitTimestamp, 5); + client.startCommitWithTime(commitTimestamp); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect(); + assertNoWriteErrors(writeStatuses); } - assertFalse(metadata(writeConfig, context).isInSync()); - testTable.doWriteOperation("005", UPSERT, asList("p1", "p2", "p3"), 1); - if (MERGE_ON_READ.equals(tableType)) { - testTable = testTable.doCompaction("006", asList("p1", "p2")); + + // Metadata table should have been bootstrapped + assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should exist"); + FileStatus oldStatus = fs.getFileStatus(new Path(metadataTableBasePath)); + + // set hoodie.table.version to 2 in hoodie.properties file + changeTableVersion(HoodieTableVersion.TWO); + + // With next commit the table should be deleted (as part of upgrade) + commitTimestamp = HoodieActiveTimeline.createNewInstantTime(); + metaClient.reloadActiveTimeline(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + records = dataGen.generateInserts(commitTimestamp, 5); + client.startCommitWithTime(commitTimestamp); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect(); + assertNoWriteErrors(writeStatuses); } - testTable.doCleanBasedOnCommits("007", singletonList("001")); - /* TODO: Perform restore with metadata disabled - try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { - client.restoreToInstant("004"); - }*/ - assertFalse(metadata(writeConfig, context).isInSync()); - syncAndValidate(testTable, emptyList(), true, true, true, true); - } + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist"); - /** - * Test that failure to perform deltacommit on the metadata table does not lead to missed sync. - */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testMetdataTableCommitFailure(HoodieTableType tableType) throws Exception { - init(tableType); - testTable.doWriteOperation("001", INSERT, asList("p1", "p2"), asList("p1", "p2"), 2, true); - syncTableMetadata(writeConfig); - testTable.doWriteOperation("002", INSERT, asList("p1", "p2"), 2, true); - syncTableMetadata(writeConfig); + // With next commit the table should be re-bootstrapped (currently in the constructor. To be changed) + commitTimestamp = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + records = dataGen.generateInserts(commitTimestamp, 5); + client.startCommitWithTime(commitTimestamp); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commitTimestamp).collect(); + assertNoWriteErrors(writeStatuses); + } - // At this time both commits 001 and 002 must be synced to the metadata table - HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); - HoodieActiveTimeline timeline = metadataMetaClient.getActiveTimeline(); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "001"))); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); - - // Delete the 002 deltacommit completed instant to make it inflight - FileCreateUtils.deleteDeltaCommit(metadataTableBasePath, "002"); - timeline = metadataMetaClient.reloadActiveTimeline(); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "001"))); - assertTrue(timeline.containsInstant(new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); - - // In this commit deltacommit "002" will be rolled back and attempted again. - testTable.doWriteOperation("003", BULK_INSERT, singletonList("p3"), asList("p1", "p2", "p3"), 2); - syncTableMetadata(writeConfig); + initMetaClient(); + assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.THREE.versionCode()); + assertTrue(fs.exists(new Path(metadataTableBasePath)), "Metadata table should exist"); + FileStatus newStatus = fs.getFileStatus(new Path(metadataTableBasePath)); + assertTrue(oldStatus.getModificationTime() < newStatus.getModificationTime()); - timeline = metadataMetaClient.reloadActiveTimeline(); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "001"))); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "003"))); - assertEquals(1, timeline.getRollbackTimeline().countInstants()); + // Test downgrade by running the downgrader + new SparkUpgradeDowngrade(metaClient, writeConfig, context).run(metaClient, HoodieTableVersion.TWO, writeConfig, context, null); + + assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.TWO.versionCode()); + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist"); } /** - * Tests that if timeline has an inflight commit midway, metadata syncs only completed commits (including later to inflight commit). + * Test various error scenarios. */ - @ParameterizedTest - @EnumSource(HoodieTableType.class) - public void testInFlightCommit(HoodieTableType tableType) throws Exception { - init(tableType); - // bootstrap w/ 2 commits - doWriteOperationsAndBootstrapMetadata(testTable); - - // trigger an upsert - testTable.doWriteOperation("003", UPSERT, singletonList("p3"), asList("p1", "p2", "p3"), 3); - syncAndValidate(testTable); + @Test + public void testErrorCases() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - // trigger an upsert - testTable.doWriteOperation("005", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable); + // 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); - // create an inflight commit. - HoodieCommitMetadata inflightCommitMeta = testTable.doWriteOperation("006", UPSERT, emptyList(), - asList("p1", "p2", "p3"), 2, false, true); + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 5); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); - // trigger upsert - testTable.doWriteOperation("007", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - // testTable validation will fetch only files pertaining to completed commits. So, validateMetadata() will skip files for 006 - // while validating against actual metadata table. - syncAndValidate(testTable, singletonList("006"), writeConfig.isMetadataTableEnabled(), writeConfig.getMetadataConfig().enableSync(), false); + // 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)); + } - // Remove the inflight instance holding back table sync - testTable.moveInflightCommitToComplete("006", inflightCommitMeta); - syncTableMetadata(writeConfig); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + String newCommitTime = client.startCommit(); + // Next insert + List records = dataGen.generateInserts(newCommitTime, 5); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); - // A regular commit should get synced - testTable.doWriteOperation("008", UPSERT, emptyList(), asList("p1", "p2", "p3"), 2); - syncAndValidate(testTable, true); + // Post rollback commit and metadata should be valid + validateMetadata(client); + } } - private void doWriteOperationsAndBootstrapMetadata(HoodieTestTable testTable) throws Exception { - testTable.doWriteOperation("001", INSERT, asList("p1", "p2"), asList("p1", "p2"), - 2, true); - testTable.doWriteOperation("002", UPSERT, asList("p1", "p2"), - 2, true); - syncAndValidate(testTable); - } + @Test + public void testNonPartitioned() throws Exception { + init(HoodieTableType.COPY_ON_WRITE, false); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - private void syncAndValidate(HoodieTestTable testTable) throws IOException { - syncAndValidate(testTable, emptyList(), writeConfig.isMetadataTableEnabled(), writeConfig.getMetadataConfig().enableSync(), true); + HoodieTestDataGenerator nonPartitionedGenerator = new HoodieTestDataGenerator(new String[] {""}); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Write 1 (Bulk insert) + String newCommitTime = "0000001"; + 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"); + } } - private void syncAndValidate(HoodieTestTable testTable, boolean doFullValidation) throws IOException { - syncAndValidate(testTable, emptyList(), writeConfig.isMetadataTableEnabled(), writeConfig.getMetadataConfig().enableSync(), true, doFullValidation); + /** + * Test various metrics published by metadata table. + */ + @Test + public void testMetadataMetrics() throws Exception { + init(HoodieTableType.COPY_ON_WRITE, false); + 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")); + assertTrue(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count") >= 1L); + final String prefix = MetadataPartitionType.FILES.partitionPath() + "."; + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)); + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_COUNT_LOG_FILES)); + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)); + assertTrue(metricsRegistry.getAllCounts().containsKey(prefix + HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)); + } } - private void syncAndValidate(HoodieTestTable testTable, List inflightCommits, boolean enableMetadata, - boolean enableMetadataSync, boolean enableValidation) throws IOException { - syncAndValidate(testTable, inflightCommits, enableMetadata, enableMetadataSync, enableValidation, false); + private void doPreBootstrapOperations(HoodieTestTable testTable) throws Exception { + doPreBootstrapOperations(testTable, "0000001", "0000002"); } - private void syncAndValidate(HoodieTestTable testTable, List inflightCommits, boolean enableMetadata, - boolean enableMetadataSync, boolean enableValidation, boolean doFullValidation) throws IOException { - writeConfig.getMetadataConfig().setValue(HoodieMetadataConfig.ENABLE, String.valueOf(enableMetadata)); - writeConfig.getMetadataConfig().setValue(HoodieMetadataConfig.SYNC_ENABLE, String.valueOf(enableMetadataSync)); - syncTableMetadata(writeConfig); - validateMetadata(testTable, inflightCommits, writeConfig, metadataTableBasePath, doFullValidation); + private void doPreBootstrapOperations(HoodieTestTable testTable, String commit1, String commit2) throws Exception { + testTable.doWriteOperation(commit1, INSERT, asList("p1", "p2"), asList("p1", "p2"), + 2, true); + testTable.doWriteOperation(commit2, UPSERT, asList("p1", "p2"), + 2, true); + validateMetadata(testTable); } - private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) { - return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build(); + private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception { + doWriteInsertAndUpsert(testTable, "0000001", "0000002"); } - private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { - return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics); + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, boolean mergeAllowDuplicateInserts) { + HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER); + return builder + .withCompactionConfig( + HoodieCompactionConfig.newBuilder() + .compactionSmallFileSize(smallFileSize) + // Set rollback to LAZY so no inflights are deleted + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .insertSplitSize(insertSplitSize).build()) + .withStorageConfig( + HoodieStorageConfig.newBuilder() + .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) + .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) + .withMergeAllowDuplicateOnInserts(mergeAllowDuplicateInserts) + .build(); } - private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, 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) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) - .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) - .withFailedWritesCleaningPolicy(policy) - .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) - .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) - .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") - .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() - .withEnableBackupForRemoteFileSystemView(false).build()) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder() - .enable(useFileListingMetadata) - .enableMetrics(enableMetrics).build()) - .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) - .withExecutorMetrics(true).build()) - .withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() - .usePrefix("unit-test").build()); + public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex.IndexType indexType, + HoodieFailedWritesCleaningPolicy cleaningPolicy) { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr) + .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2) + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withWriteStatusClass(MetadataMergeWriteStatus.class) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy) + .compactionSmallFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build()) + .forTable("test-trip-table") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) + .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server + .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); } private void validateMetadata(SparkRDDWriteClient testClient) throws IOException { @@ -916,7 +1135,6 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException .sum(); assertEquals(metadataFilenames.size(), numFiles); } catch (IOException e) { - // TODO Auto-generated catch block e.printStackTrace(); assertTrue(false, "Exception should not be raised: " + e); } @@ -928,10 +1146,8 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException // Validate write config for metadata table HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig(); assertFalse(metadataWriteConfig.isMetadataTableEnabled(), "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 = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); // Metadata table is MOR @@ -945,8 +1161,8 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath), - false, false, false); - Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); + 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 @@ -963,6 +1179,23 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException LOG.info("Validation time=" + timer.endTimer()); } + /** + * Returns the list of all files in the dataset by iterating over the metadata table. + * + * @throws IOException + * @throws IllegalArgumentException + */ + private List getAllFiles(HoodieTableMetadata metadata) throws Exception { + List allfiles = new LinkedList<>(); + for (String partition : metadata.getAllPartitionPaths()) { + for (FileStatus status : metadata.getAllFilesInPartition(new Path(basePath, partition))) { + allfiles.add(status.getPath()); + } + } + + return allfiles; + } + private HoodieBackedTableMetadataWriter metadataWriter(SparkRDDWriteClient client) { return (HoodieBackedTableMetadataWriter) SparkHoodieBackedTableMetadataWriter .create(hadoopConf, client.getConfig(), new HoodieSparkEngineContext(jsc)); @@ -974,6 +1207,14 @@ private HoodieTableMetadata metadata(SparkRDDWriteClient client) { clientConfig.getSpillableMapBasePath()); } + private void changeTableVersion(HoodieTableVersion version) throws IOException { + metaClient.getTableConfig().setTableVersion(version); + Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE); + try (FSDataOutputStream os = metaClient.getFs().create(propertyFile)) { + metaClient.getTableConfig().getProps().store(os, ""); + } + } + @Override protected HoodieTableType getTableType() { return tableType; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java new file mode 100644 index 0000000000000..5242e9f33766d --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.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.client.functional; + +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase { + + private static final Logger LOG = LogManager.getLogger(TestHoodieBackedTableMetadata.class); + + @Test + public void testTableOperations() throws Exception { + HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE; + init(tableType); + doWriteInsertAndUpsert(testTable); + + // trigger an upsert + doWriteOperation(testTable, "0000003"); + verifyBaseMetadataTable(); + } + + private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception { + doWriteInsertAndUpsert(testTable, "0000001", "0000002"); + } + + private void verifyBaseMetadataTable() throws IOException { + HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false); + assertTrue(tableMetadata.enabled()); + List fsPartitionPaths = testTable.getAllPartitionPaths(); + List fsPartitions = new ArrayList<>(); + fsPartitionPaths.forEach(entry -> fsPartitions.add(entry.getFileName().toString())); + List metadataPartitions = tableMetadata.getAllPartitionPaths(); + + Collections.sort(fsPartitions); + Collections.sort(metadataPartitions); + + assertEquals(fsPartitions.size(), metadataPartitions.size(), "Partitions should match"); + assertEquals(fsPartitions, metadataPartitions, "Partitions should match"); + + // Files within each partition should match + HoodieTable table = HoodieSparkTable.create(writeConfig, context, true); + TableFileSystemView tableView = table.getHoodieView(); + List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList()); + Map partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths); + assertEquals(fsPartitions.size(), partitionToFilesMap.size()); + + fsPartitions.forEach(partition -> { + try { + validateFilesPerPartition(testTable, tableMetadata, tableView, partitionToFilesMap, partition); + } catch (IOException e) { + fail("Exception should not be raised: " + e); + } + }); + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index de4f42177429a..f712201ee8089 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -2147,19 +2147,18 @@ public void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMetaFi assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 0); } - @ParameterizedTest - @MethodSource("populateMetaFieldsParams") - public void testParallelInsertAndCleanPreviousFailedCommits(boolean populateMetaFields) throws Exception { + @Test + public void testParallelInsertAndCleanPreviousFailedCommits() throws Exception { HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; ExecutorService service = Executors.newFixedThreadPool(2); HoodieTestUtils.init(hadoopConf, basePath); // Perform 2 failed writes to table - SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)); writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, false); client.close(); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)); writeBatch(client, "200", "200", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, false); @@ -2167,7 +2166,7 @@ public void testParallelInsertAndCleanPreviousFailedCommits(boolean populateMeta // refresh data generator to delete records generated from failed commits dataGen = new HoodieTestDataGenerator(); // Create a succesful commit - Future> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)), + Future> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)), "300", "200", Option.of(Arrays.asList("300")), "200", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); commit3.get(); @@ -2177,17 +2176,17 @@ public void testParallelInsertAndCleanPreviousFailedCommits(boolean populateMeta CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0); assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2); assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1); - client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); + client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)); // Await till enough time passes such that the first 2 failed commits heartbeats are expired boolean conditionMet = false; while (!conditionMet) { conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200"); Thread.sleep(2000); } - Future> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)), + Future> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)), "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true)); - Future clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)).clean()); + Future clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)).clean()); commit4.get(); clean1.get(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 9c4059a519504..824d742181f62 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.functional; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; @@ -97,6 +98,10 @@ private static Stream indexTypeParams() { private HoodieWriteConfig config; private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception { + setUp(indexType, populateMetaFields, true); + } + + private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadata) throws Exception { this.indexType = indexType; initPath(); initSparkContexts(); @@ -107,7 +112,7 @@ private void setUp(IndexType indexType, boolean populateMetaFields) throws Excep config = getConfigBuilder() .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) - .build()).withAutoCommit(false).build(); + .build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()).build(); writeClient = getHoodieWriteClient(config); this.index = writeClient.getIndex(); } @@ -220,7 +225,7 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul @ParameterizedTest @MethodSource("indexTypeParams") public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception { - setUp(indexType, populateMetaFields); + setUp(indexType, populateMetaFields, false); String newCommitTime = writeClient.startCommit(); int totalRecords = 20 + random.nextInt(20); List records = dataGen.generateInserts(newCommitTime, totalRecords); @@ -367,7 +372,8 @@ public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() thro .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType) .withGlobalSimpleIndexUpdatePartitionPath(true) .withBloomIndexUpdatePartitionPath(true) - .build()).build(); + .build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); writeClient = getHoodieWriteClient(config); index = writeClient.getIndex(); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java new file mode 100644 index 0000000000000..85f869f7835b3 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.functional; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.testutils.HoodieMetadataTestTable; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.metrics.HoodieMetricsConfig; +import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.HoodieTimelineArchiveLog; +import org.apache.hudi.testutils.HoodieClientTestHarness; + +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.AfterEach; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.apache.hudi.common.model.WriteOperationType.INSERT; +import static org.apache.hudi.common.model.WriteOperationType.UPSERT; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; + +public class TestHoodieMetadataBase extends HoodieClientTestHarness { + + private static final Logger LOG = LogManager.getLogger(TestHoodieMetadataBase.class); + + protected static HoodieTestTable testTable; + protected String metadataTableBasePath; + protected HoodieTableType tableType; + protected HoodieWriteConfig writeConfig; + protected HoodieTableMetadataWriter metadataWriter; + + public void init(HoodieTableType tableType) throws IOException { + init(tableType, true); + } + + public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException { + this.tableType = tableType; + initPath(); + initSparkContexts("TestHoodieMetadata"); + initFileSystem(); + fs.mkdirs(new Path(basePath)); + initMetaClient(tableType); + initTestDataGenerator(); + metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + writeConfig = getWriteConfig(true, enableMetadataTable); + initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable); + } + + protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) { + this.writeConfig = writeConfig; + if (enableMetadataTable) { + metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context); + testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + } else { + testTable = HoodieTestTable.of(metaClient); + } + } + + @AfterEach + public void clean() throws Exception { + cleanupResources(); + } + + protected void doWriteInsertAndUpsert(HoodieTestTable testTable, String commit1, String commit2) throws Exception { + testTable.doWriteOperation(commit1, INSERT, asList("p1", "p2"), asList("p1", "p2"), + 4, false); + testTable.doWriteOperation(commit2, UPSERT, asList("p1", "p2"), + 4, false); + validateMetadata(testTable); + } + + protected void doWriteOperationAndValidateMetadata(HoodieTestTable testTable, String commitTime) throws Exception { + doWriteOperation(testTable, commitTime); + validateMetadata(testTable); + } + + protected void doWriteOperation(HoodieTestTable testTable, String commitTime) throws Exception { + doWriteOperation(testTable, commitTime, UPSERT); + } + + protected void doWriteOperationAndValidate(HoodieTestTable testTable, String commitTime) throws Exception { + doWriteOperationAndValidate(testTable, commitTime, UPSERT); + } + + protected void doWriteOperationAndValidate(HoodieTestTable testTable, String commitTime, WriteOperationType operationType) throws Exception { + doWriteOperation(testTable, commitTime, operationType); + validateMetadata(testTable); + } + + protected void doWriteOperation(HoodieTestTable testTable, String commitTime, WriteOperationType operationType) throws Exception { + testTable.doWriteOperation(commitTime, operationType, emptyList(), asList("p1", "p2"), 3); + } + + protected void doClean(HoodieTestTable testTable, String commitTime, List commitsToClean) throws IOException { + doCleanInternal(testTable, commitTime, commitsToClean, false); + } + + protected void doCleanAndValidate(HoodieTestTable testTable, String commitTime, List commitsToClean) throws IOException { + doCleanInternal(testTable, commitTime, commitsToClean, true); + } + + private void doCleanInternal(HoodieTestTable testTable, String commitTime, List commitsToClean, boolean validate) throws IOException { + testTable.doCleanBasedOnCommits(commitTime, commitsToClean); + if (validate) { + validateMetadata(testTable); + } + } + + protected void doCompaction(HoodieTestTable testTable, String commitTime) throws Exception { + doCompactionInternal(testTable, commitTime, false); + } + + protected void doCompactionAndValidate(HoodieTestTable testTable, String commitTime) throws Exception { + doCompactionInternal(testTable, commitTime, true); + } + + private void doCompactionInternal(HoodieTestTable testTable, String commitTime, boolean validate) throws Exception { + testTable.doCompaction(commitTime, asList("p1", "p2")); + if (validate) { + validateMetadata(testTable); + } + } + + protected void doCluster(HoodieTestTable testTable, String commitTime) throws Exception { + doClusterInternal(testTable, commitTime, false); + } + + protected void doClusterAndValidate(HoodieTestTable testTable, String commitTime) throws Exception { + doClusterInternal(testTable, commitTime, true); + } + + protected void doClusterInternal(HoodieTestTable testTable, String commitTime, boolean validate) throws Exception { + testTable.doCluster(commitTime, new HashMap<>(), Arrays.asList("p1", "p2"), 2); + if (validate) { + validateMetadata(testTable); + } + } + + protected void doRollback(HoodieTestTable testTable, String commitToRollback, String rollbackTime) throws Exception { + doRollbackInternal(testTable, commitToRollback, rollbackTime, false); + } + + protected void doRollbackAndValidate(HoodieTestTable testTable, String commitToRollback, String rollbackTime) throws Exception { + doRollbackInternal(testTable, commitToRollback, rollbackTime, true); + } + + private void doRollbackInternal(HoodieTestTable testTable, String commitToRollback, String rollbackTime, boolean validate) throws Exception { + testTable.doRollback(commitToRollback, rollbackTime); + if (validate) { + validateMetadata(testTable); + } + } + + protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, String commitTime) throws Exception { + doPreBootstrapWriteOperation(testTable, UPSERT, commitTime); + } + + protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, WriteOperationType writeOperationType, String commitTime) throws Exception { + doPreBootstrapWriteOperation(testTable, writeOperationType, commitTime, 2); + } + + protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, WriteOperationType writeOperationType, String commitTime, int filesPerPartition) throws Exception { + testTable.doWriteOperation(commitTime, writeOperationType, asList("p1", "p2"), asList("p1", "p2"), + filesPerPartition, true); + } + + protected void doPreBootstrapClean(HoodieTestTable testTable, String commitTime, List commitsToClean) throws Exception { + testTable.doCleanBasedOnCommits(commitTime, commitsToClean); + } + + protected void doPreBootstrapRollback(HoodieTestTable testTable, String rollbackTime, String commitToRollback) throws Exception { + testTable.doRollback(commitToRollback, rollbackTime); + } + + protected void doPrebootstrapCompaction(HoodieTestTable testTable, String commitTime) throws Exception { + doPrebootstrapCompaction(testTable, commitTime, Arrays.asList("p1", "p2")); + } + + protected void doPrebootstrapCompaction(HoodieTestTable testTable, String commitTime, List partitions) throws Exception { + testTable.doCompaction(commitTime, partitions); + } + + protected void doPreBootstrapCluster(HoodieTestTable testTable, String commitTime) throws Exception { + testTable.doCluster(commitTime, new HashMap<>(), Arrays.asList("p1", "p2"), 2); + } + + protected void doPreBootstrapRestore(HoodieTestTable testTable, String restoreTime, String commitToRestore) throws Exception { + testTable.doRestore(commitToRestore, restoreTime); + } + + protected void archiveDataTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) throws IOException { + HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table); + archiveLog.archiveIfRequired(context); + } + + protected void validateMetadata(HoodieTestTable testTable) throws IOException { + validateMetadata(testTable, emptyList()); + } + + protected void validateMetadata(HoodieTestTable testTable, boolean doFullValidation) throws IOException { + validateMetadata(testTable, emptyList(), doFullValidation); + } + + protected void validateMetadata(HoodieTestTable testTable, List inflightCommits) throws IOException { + validateMetadata(testTable, inflightCommits, false); + } + + protected void validateMetadata(HoodieTestTable testTable, List inflightCommits, boolean doFullValidation) throws IOException { + validateMetadata(testTable, inflightCommits, writeConfig, metadataTableBasePath, doFullValidation); + } + + protected HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) { + return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build(); + } + + protected HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { + return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics); + } + + protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, 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) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .withFailedWritesCleaningPolicy(policy) + .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) + .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") + .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() + .withEnableBackupForRemoteFileSystemView(false).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(useFileListingMetadata) + .enableMetrics(enableMetrics).build()) + .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) + .withExecutorMetrics(true).build()) + .withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() + .usePrefix("unit-test").build()); + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java new file mode 100644 index 0000000000000..12c8410c35e02 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBootstrap.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.functional; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.testutils.HoodieMetadataTestTable; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.util.Arrays; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; +import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; +import static org.apache.hudi.common.model.WriteOperationType.INSERT; +import static org.apache.hudi.common.model.WriteOperationType.UPSERT; + +@Tag("functional") +public class TestHoodieMetadataBootstrap extends TestHoodieMetadataBase { + + private static final Logger LOG = LogManager.getLogger(TestHoodieMetadataBootstrap.class); + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapInsertUpsert(HoodieTableType tableType) throws Exception { + init(tableType, false); + doPreBootstrapWriteOperation(testTable, INSERT, "0000001"); + doPreBootstrapWriteOperation(testTable, "0000002"); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, "0000003"); + } + bootstrapAndVerify(); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapInsertUpsertClean(HoodieTableType tableType) throws Exception { + init(tableType, false); + doPreBootstrapWriteOperation(testTable, INSERT, "0000001"); + doPreBootstrapWriteOperation(testTable, "0000002"); + doPreBootstrapClean(testTable, "0000003", Arrays.asList("0000001")); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, "0000004"); + } + doPreBootstrapWriteOperation(testTable, "0000005"); + bootstrapAndVerify(); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapInsertUpsertRollback(HoodieTableType tableType) throws Exception { + init(tableType, false); + doPreBootstrapWriteOperation(testTable, INSERT, "0000001"); + doPreBootstrapWriteOperation(testTable, "0000002"); + doPreBootstrapRollback(testTable, "0000003", "0000002"); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, "0000004"); + } + bootstrapAndVerify(); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapInsertUpsertCluster(HoodieTableType tableType) throws Exception { + init(tableType, false); + doPreBootstrapWriteOperation(testTable, INSERT, "0000001"); + doPreBootstrapWriteOperation(testTable, "0000002"); + doPreBootstrapCluster(testTable, "0000003"); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, "0000004"); + } + bootstrapAndVerify(); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType) throws Exception { + init(tableType, false); + for (int i = 1; i < 25; i += 7) { + String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i; + String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1); + String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2); + String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3); + String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4); + String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5); + String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6); + doPreBootstrapWriteOperation(testTable, INSERT, commitTime1); + doPreBootstrapWriteOperation(testTable, commitTime2); + doPreBootstrapClean(testTable, commitTime3, Arrays.asList(commitTime1)); + doPreBootstrapWriteOperation(testTable, commitTime4); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, commitTime5); + } + doPreBootstrapWriteOperation(testTable, commitTime6); + doPreBootstrapRollback(testTable, commitTime7, commitTime6); + } + bootstrapAndVerify(); + } + + @Test + public void testMetadataBootstrapInflightCommit() throws Exception { + HoodieTableType tableType = COPY_ON_WRITE; + init(tableType, false); + + doPreBootstrapWriteOperation(testTable, INSERT, "0000001"); + doPreBootstrapWriteOperation(testTable, "0000002"); + // add an inflight commit + HoodieCommitMetadata inflightCommitMeta = testTable.doWriteOperation("00000007", UPSERT, emptyList(), + asList("p1", "p2"), 2, true, true); + // bootstrap and following validation should fail. bootstrap should not happen. + bootstrapAndVerifyFailure(); + + // once the commit is complete, metadata should get fully synced. + // in prod code path, SparkHoodieBackedTableMetadataWriter.create() will be called for every commit, + // which may not be the case here if we directly call HoodieBackedTableMetadataWriter.update() + // hence lets first move the commit to complete and invoke sync directly + ((HoodieMetadataTestTable) testTable).moveInflightCommitToComplete("00000007", inflightCommitMeta, true); + syncTableMetadata(writeConfig); + validateMetadata(testTable); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapArchival(HoodieTableType tableType) throws Exception { + init(tableType, false); + writeConfig = getWriteConfig(2, 4); + for (int i = 1; i < 13; i += 7) { + String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i; + String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1); + String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2); + String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3); + String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4); + String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5); + String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6); + doPreBootstrapWriteOperation(testTable, INSERT, commitTime1); + doPreBootstrapWriteOperation(testTable, commitTime2); + doPreBootstrapClean(testTable, commitTime3, Arrays.asList(commitTime1)); + doPreBootstrapWriteOperation(testTable, commitTime4); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, commitTime5); + } + doPreBootstrapWriteOperation(testTable, commitTime6); + doPreBootstrapRollback(testTable, commitTime7, commitTime6); + } + // archive and then bootstrap + archiveDataTable(writeConfig, metaClient); + bootstrapAndVerify(); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapAfterRestore(HoodieTableType tableType) throws Exception { + init(tableType, false); + testRestore(false); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testMetadataBootstrapAfterRestoreAndUpserts(HoodieTableType tableType) throws Exception { + init(tableType, false); + testRestore(true); + } + + private void testRestore(boolean addUpsertsAfterRestore) throws Exception { + doPreBootstrapWriteOperation(testTable, INSERT, "0000001"); + doPreBootstrapWriteOperation(testTable, "0000002"); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, "0000003"); + } + doPreBootstrapWriteOperation(testTable, "0000004"); + doPreBootstrapWriteOperation(testTable, "0000005"); + doPreBootstrapWriteOperation(testTable, "0000006"); + doPreBootstrapRestore(testTable, "0000007", "0000004"); + + if (addUpsertsAfterRestore) { + doPreBootstrapWriteOperation(testTable, "0000008"); + doPreBootstrapWriteOperation(testTable, "0000009"); + if (tableType == MERGE_ON_READ) { + doPrebootstrapCompaction(testTable, "0000010"); + } + } + bootstrapAndVerify(); + } + + private void bootstrapAndVerify() throws Exception { + writeConfig = getWriteConfig(true, true); + initWriteConfigAndMetatableWriter(writeConfig, true); + syncTableMetadata(writeConfig); + validateMetadata(testTable); + // after bootstrap do two writes and validate its still functional. + doWriteInsertAndUpsert(testTable); + validateMetadata(testTable); + } + + private void bootstrapAndVerifyFailure() throws Exception { + writeConfig = getWriteConfig(true, true); + initWriteConfigAndMetatableWriter(writeConfig, true); + syncTableMetadata(writeConfig); + try { + validateMetadata(testTable); + Assertions.fail("Should have failed"); + } catch (IllegalStateException e) { + // expected + } + } + + private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception { + doWriteInsertAndUpsert(testTable, "0000100", "0000101"); + } + + private HoodieWriteConfig getWriteConfig(int minArchivalCommits, int maxArchivalCommits) throws Exception { + return HoodieWriteConfig.newBuilder().withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .forTable("test-trip-table").build(); + } + + @Override + protected HoodieTableType getTableType() { + return tableType; + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java index cdd3fa526b914..53f0cdde3da86 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java @@ -24,23 +24,28 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.HoodieCleanStat; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; +import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.HoodieTimelineArchiveLog; @@ -48,19 +53,23 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +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; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -71,262 +80,137 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { + private static final Logger LOG = LogManager.getLogger(TestHoodieTimelineArchiveLog.class); + private Configuration hadoopConf; private HoodieWrapperFileSystem wrapperFs; + private HoodieTableMetadataWriter metadataWriter; + private HoodieTestTable testTable; - @BeforeEach public void init() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + } + + public void init(HoodieTableType tableType) throws Exception { initPath(); initSparkContexts(); initMetaClient(); hadoopConf = context.getHadoopConf().get(); metaClient.getFs().mkdirs(new Path(basePath)); - metaClient = HoodieTestUtils.init(hadoopConf, basePath); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); wrapperFs = metaClient.getFs(); hadoopConf.addResource(wrapperFs.getConf()); } + private void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) { + if (enableMetadataTable) { + metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context); + testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + } else { + testTable = HoodieTestTable.of(metaClient); + } + } + @AfterEach public void clean() throws IOException { cleanupResources(); } - @Test - public void testArchiveEmptyTable() throws IOException { - HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table").build(); - metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - boolean result = archiveLog.archiveIfRequired(context); - assertTrue(result); + private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, int maxDeltaCommitsMetadataTable) throws Exception { + return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE); } - @Test - public void testArchiveTableWithArchival() throws IOException { - HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) + private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, int maxDeltaCommitsMetadataTable, + HoodieTableType tableType) throws Exception { + init(tableType); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 4).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsMetadataTable).build()) .forTable("test-trip-table").build(); - HoodieTestUtils.init(hadoopConf, basePath); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); - - metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - - assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); - - createCleanMetadata("100", false); - createCleanMetadata("101", false); - createCleanMetadata("102", false); - createCleanMetadata("103", false); - createCleanMetadata("104", false); - createCleanMetadata("105", false); - createCleanMetadata("106", true); - createCleanMetadata("107", true); - - // reload the timeline and get all the commmits before archive - timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); - List originalCommits = timeline.getInstants().collect(Collectors.toList()); - - assertEquals(12, timeline.countInstants(), "Loaded 6 commits and the count should match"); - - // verify in-flight instants before archive - verifyInflightInstants(metaClient, 2); - - metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - assertTrue(archiveLog.archiveIfRequired(context)); - - // reload the timeline and remove the remaining commits - timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); - originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); - - // Check compaction instants - List instants = metaClient.scanHoodieInstantsFromFileSystem( - new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false); - assertEquals(4, instants.size(), "Should delete all compaction instants < 104"); - assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")), - "Requested Compaction must be absent for 100"); - assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")), - "Inflight Compaction must be absent for 100"); - assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")), - "Requested Compaction must be absent for 101"); - assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")), - "Inflight Compaction must be absent for 101"); - assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")), - "Requested Compaction must be absent for 102"); - assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")), - "Inflight Compaction must be absent for 102"); - assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")), - "Requested Compaction must be absent for 103"); - assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")), - "Inflight Compaction must be absent for 103"); - assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")), - "Requested Compaction must be present for 104"); - assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104")), - "Inflight Compaction must be present for 104"); - assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105")), - "Requested Compaction must be present for 105"); - assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105")), - "Inflight Compaction must be present for 105"); - - // read the file - HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient); - assertEquals(24, archivedTimeline.countInstants(), - "Total archived records and total read records are the same count"); - - //make sure the archived commits are the same as the (originalcommits - commitsleft) - Set readCommits = - archivedTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); - assertEquals(originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits, - "Read commits map should match the originalCommits - commitsLoadedFromArchival"); - - // verify in-flight instants after archive - verifyInflightInstants(metaClient, 2); + initWriteConfigAndMetatableWriter(writeConfig, enableMetadata); + return writeConfig; } @Test - public void testArchiveTableWithNoArchival() throws IOException { - HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) - .build(); + public void testArchiveEmptyTable() throws Exception { + init(); + HoodieWriteConfig cfg = + HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).forTable("test-trip-table").build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); - // Requested Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); - // Inflight Compaction - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); - - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match"); boolean result = archiveLog.archiveIfRequired(context); assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); - assertEquals(4, timeline.countInstants(), "Should not archive commits when maxCommitsToKeep is 5"); - - List instants = metaClient.scanHoodieInstantsFromFileSystem( - new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false); - assertEquals(8, instants.size(), "Should not delete any aux compaction files when maxCommitsToKeep is 5"); - assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")), - "Requested Compaction must be present for 100"); - assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")), - "Inflight Compaction must be present for 100"); - assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")), - "Requested Compaction must be present for 101"); - assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")), - "Inflight Compaction must be present for 101"); - assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")), - "Requested Compaction must be present for 102"); - assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")), - "Inflight Compaction must be present for 102"); - assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")), - "Requested Compaction must be present for 103"); - assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")), - "Inflight Compaction must be present for 103"); } - @Test - public void testArchiveCommitSafety() throws IOException { - HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) - .build(); - metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testArchiveTableWithArchival(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2); + + // min archival commits is 2 and max archival commits is 4. and so, after 5th commit, 3 commits will be archived. + // 1,2,3,4,5 : after archival -> 4,5 + // after 3 more commits, earliest 3 will be archived + // 4,5,6,7,8 : after archival -> 7, 8 + // after 9 no-op wrt archival. + for (int i = 1; i < 10; i++) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // trigger archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + if (i < 5) { + assertEquals(originalCommits, commitsAfterArchival); + } else if (i == 5) { + // archival should have kicked in. + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003")), getActiveCommitInstants(Arrays.asList("00000004", "00000005")), commitsAfterArchival); + } else if (i < 8) { + assertEquals(originalCommits, commitsAfterArchival); + } else if (i == 8) { + // archival should have kicked in. + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006")), + getActiveCommitInstants(Arrays.asList("00000007", "00000008")), commitsAfterArchival); + } else { + assertEquals(originalCommits, commitsAfterArchival); + } + } + } - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); - boolean result = archiveLog.archiveIfRequired(context); - assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); - assertTrue(timeline.containsOrBeforeTimelineStarts("100"), "Archived commits should always be safe"); - assertTrue(timeline.containsOrBeforeTimelineStarts("101"), "Archived commits should always be safe"); - assertTrue(timeline.containsOrBeforeTimelineStarts("102"), "Archived commits should always be safe"); - assertTrue(timeline.containsOrBeforeTimelineStarts("103"), "Archived commits should always be safe"); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testNoArchivalUntilMaxArchiveConfigWithExtraInflightCommits(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 5, 2); + + // when max archival commits is set to 5, until 6th commit there should not be any archival. + for (int i = 1; i < 6; i++) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2); + // archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); + } + + // add couple of inflight. no archival should kick in. + testTable.doWriteOperation("00000006", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2, false, true); + testTable.doWriteOperation("00000007", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2, false, true); + + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); } @Test - public void testArchiveCommitSavepointNoHole() throws IOException { + public void testArchiveCommitSavepointNoHole() throws Exception { + init(); HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); @@ -353,89 +237,97 @@ public void testArchiveCommitSavepointNoHole() throws IOException { "Archived commits should always be safe"); } - @Test - public void testArchiveRollbacks() throws IOException { - HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) - .build(); - - createCommitAndRollbackFile("100", "101", false); - createCommitAndRollbackFile("102", "103", false); - createCommitAndRollbackFile("104", "105", false); - createCommitAndRollbackFile("106", "107", false); - - HoodieTable table = HoodieSparkTable.create(cfg, context); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - - assertTrue(archiveLog.archiveIfRequired(context)); - HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); - assertEquals(2, timeline.countInstants(), - "first two commits must have been archived"); - assertFalse(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "101")), - "first rollback must have been archived"); - assertFalse(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "103")), - "second rollback must have been archived"); - assertTrue(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "105")), - "first rollback must have been archived"); - assertTrue(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "107")), - "second rollback must have been archived"); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testArchiveRollbacksTestTable(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 3, 2); + + for (int i = 1; i < 9; i += 2) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + testTable.doRollback("0000000" + i, "0000000" + (i + 1)); + + // trigger archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + + if (i != 7) { + assertEquals(originalCommits, commitsAfterArchival); + } else { + // only time when archival will kick in + List expectedArchivedInstants = new ArrayList<>(); + expectedArchivedInstants.addAll(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003"))); + expectedArchivedInstants.addAll(getAllArchivedCommitInstants(Arrays.asList("00000002", "00000004"), HoodieTimeline.ROLLBACK_ACTION)); + List expectedActiveInstants = new ArrayList<>(); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000005", "00000007"))); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000006", "00000008"), HoodieTimeline.ROLLBACK_ACTION)); + verifyArchival(expectedArchivedInstants, expectedActiveInstants, commitsAfterArchival); + } + } } - @Test - public void testArchiveCommitCompactionNoHole() throws IOException { - HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) - .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) - .build(); - HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); - HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", wrapperFs.getConf()); - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); - HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", wrapperFs.getConf()); - HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "106", wrapperFs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "107", wrapperFs.getConf()); - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2, + HoodieTableType.MERGE_ON_READ); + + // when max archival commits is set to 4, even after 7 commits, if there is an inflight compaction in the middle, archival should not kick in. + HoodieCommitMetadata inflightCompactionMetadata = null; + for (int i = 1; i < 8; i++) { + if (i == 2) { + inflightCompactionMetadata = testTable.doCompaction("0000000" + i, Arrays.asList("p1", "p2"), true); + } else { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + } + + // archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + if (i != 6) { + assertEquals(originalCommits, commitsAfterArchival); + } else { + // on 6th commit, archival will kick in. but will archive only one commit since 2nd compaction commit is inflight. + assertEquals(originalCommits.size() - commitsAfterArchival.size(), 1); + for (int j = 1; j <= 6; j++) { + if (j == 1) { + // first commit should be archived + assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); + } else if (j == 2) { + // 2nd compaction should not be archived + assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "0000000" + j))); + } else { + // every other commit should not be archived + assertTrue(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); + } + } + } + } - HoodieTimeline timeline = metaClient.getActiveTimeline().getWriteTimeline(); - assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match"); - boolean result = archiveLog.archiveIfRequired(context); - assertTrue(result); - timeline = metaClient.getActiveTimeline().reload().getWriteTimeline(); - assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")), - "Instants before oldest pending compaction can be removed"); - assertEquals(7, timeline.countInstants(), - "Since we have a pending compaction at 101, we should never archive any commit " - + "after 101 (we only archive 100)"); - assertTrue(timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")), - "Requested Compaction must still be present"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")), - "Instants greater than oldest pending compaction must be present"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")), - "Instants greater than oldest pending compaction must be present"); - assertTrue(timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")), - "Instants greater than oldest pending compaction must be present"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")), - "Instants greater than oldest pending compaction must be present"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106")), - "Instants greater than oldest pending compaction must be present"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")), - "Instants greater than oldest pending compaction must be present"); + // move inflight compaction to complete. archival should archive more commits. + // before this move, timeline 2_inflight_compaction, 3,4,5,6,7. + // after this move. 6,7. (2,3,4,5 will be archived) + testTable.moveInflightCompactionToComplete("00000002", inflightCompactionMetadata); + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + + List archivedInstants = getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003", "00000004", "00000005"), HoodieTimeline.DELTA_COMMIT_ACTION); + archivedInstants.add(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "00000002")); + archivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000002")); + verifyArchival(archivedInstants, getActiveCommitInstants(Arrays.asList("00000006", "00000007"), HoodieTimeline.DELTA_COMMIT_ACTION), commitsAfterArchival); } @Test - public void testArchiveCommitTimeline() throws IOException { + public void testArchiveCommitTimeline() throws Exception { + init(); HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) - .build(); + HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).forTable("test-trip-table") + .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "1", wrapperFs.getConf()); @@ -470,7 +362,8 @@ private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expect } @Test - public void testConvertCommitMetadata() { + public void testConvertCommitMetadata() throws Exception { + init(); HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata(); hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT); @@ -481,110 +374,93 @@ public void testConvertCommitMetadata() { assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString()); } - @Test - public void testArchiveCompletedClean() throws IOException { - HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) - .build(); - metaClient = HoodieTableMetaClient.reload(metaClient); - - createCleanMetadata("10", false); - createCleanMetadata("11", false); - HoodieInstant notArchivedInstant1 = createCleanMetadata("12", false); - HoodieInstant notArchivedInstant2 = createCleanMetadata("13", false); - - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - - archiveLog.archiveIfRequired(context); - - List notArchivedInstants = metaClient.getActiveTimeline().reload().getInstants().collect(Collectors.toList()); - //There will be 3 * 2 files but due to TimelineLayoutV1 this will show as 2. - assertEquals(2, notArchivedInstants.size(), "Not archived instants should be 2"); - assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2), ""); - } - - @Test - public void testArchiveCompletedRollback() throws IOException { - HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) - .build(); - metaClient = HoodieTableMetaClient.reload(metaClient); - - createCommitAndRollbackFile("6", "10", false); - createCommitAndRollbackFile("8", "11", false); - createCommitAndRollbackFile("7", "12", false); - HoodieInstant notArchivedInstant1 = new HoodieInstant(State.COMPLETED, "rollback", "12"); - - createCommitAndRollbackFile("5", "13", false); - HoodieInstant notArchivedInstant2 = new HoodieInstant(State.COMPLETED, "rollback", "13"); - - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - - archiveLog.archiveIfRequired(context); - - List notArchivedInstants = metaClient.getActiveTimeline().reload().getRollbackTimeline().getInstants().collect(Collectors.toList()); - //There will be 2 * 2 files but due to TimelineLayoutV1 this will show as 2. - assertEquals(2, notArchivedInstants.size(), "Not archived instants should be 2"); - assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2), ""); - } - - @Test - public void testArchiveCompletedShouldRetainMinInstantsIfInstantsGreaterThanMaxtoKeep() throws IOException { - int minInstants = 2; - int maxInstants = 10; - HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstants, maxInstants).build()) - .build(); - metaClient = HoodieTableMetaClient.reload(metaClient); - for (int i = 0; i < maxInstants + 2; i++) { - createCleanMetadata(i + "", false); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testArchiveTableWithCleanCommits(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2); + + // min archival commits is 2 and max archival commits is 4(either clean commits has to be > 4 or commits has to be greater than 4. and so, after 5th commit, 3 commits will be archived. + // 1,2,3,4,5,6 : after archival -> 1,5,6 (because, 2,3,4,5 and 6 are clean commits and are eligible for archival) + // after 7th and 8th commit no-op wrt archival. + Map cleanStats = new HashMap<>(); + cleanStats.put("p1", 1); + cleanStats.put("p2", 2); + for (int i = 1; i < 9; i++) { + if (i == 1) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 10); + } else if (i < 7) { + testTable.doClean("0000000" + i, cleanStats); + } else { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + } + // trigger archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + if (i < 6) { + assertEquals(originalCommits, commitsAfterArchival); + } else if (i == 6) { + // 1,2,3,4,5,6 : after archival -> 1,5,6 (bcoz, 2,3,4,5 and 6 are clean commits and are eligible for archival) + List expectedActiveInstants = new ArrayList<>(); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000001"))); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000005", "00000006"), HoodieTimeline.CLEAN_ACTION)); + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000002", "00000003", "00000004"), HoodieTimeline.CLEAN_ACTION), expectedActiveInstants, commitsAfterArchival); + } else { + assertEquals(originalCommits, commitsAfterArchival); + } } - - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); - - archiveLog.archiveIfRequired(context); - assertEquals(minInstants, metaClient.getActiveTimeline().reload().getInstants().count()); } @Test - public void testArchiveCompletedShouldNotArchiveIfInstantsLessThanMaxtoKeep() throws IOException { - int minInstants = 2; - int maxInstants = 10; - HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstants, maxInstants).build()) - .build(); - metaClient = HoodieTableMetaClient.reload(metaClient); - for (int i = 0; i < maxInstants; i++) { - createCleanMetadata(i + "", false); + public void testArchiveRollbacksAndCleanTestTable() throws Exception { + boolean enableMetadata = false; + int minArchiveCommits = 2; + int maxArchiveCommits = 9; + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, minArchiveCommits, maxArchiveCommits, 2); + + // trigger 1 commit to add lot of files so that future cleans can clean them up + testTable.doWriteOperation("00000001", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 20); + + Map partitionToFileDeleteCount = new HashMap<>(); + partitionToFileDeleteCount.put("p1", 1); + partitionToFileDeleteCount.put("p2", 1); + // we are triggering 10 clean commits. (1 is commit, 2 -> 11 is clean) + for (int i = 2; i <= (maxArchiveCommits + 2); i++) { + testTable.doClean((i > 9 ? ("000000") : ("0000000")) + i, partitionToFileDeleteCount); } - HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table); + // we are triggering 7 commits and 7 rollbacks for the same + for (int i = 12; i <= (2 * maxArchiveCommits); i += 2) { + testTable.doWriteOperation("000000" + i, WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + testTable.doRollback("000000" + i, "000000" + (i + 1)); + } - archiveLog.archiveIfRequired(context); - assertEquals(maxInstants, metaClient.getActiveTimeline().reload().getInstants().count()); + // trigger archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + + // out of 10 clean commits, 8 will be archived. 2 to 9. 10 and 11 will be active. + // wrt regular commits, there aren't 9 commits yet and so all of them will be active. + List expectedActiveInstants = new ArrayList<>(); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000010", "00000011"), HoodieTimeline.CLEAN_ACTION)); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000001", "00000012", "00000014", "00000016", "00000018"))); + expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000013", "00000015", "00000017", "00000019"), HoodieTimeline.ROLLBACK_ACTION)); + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000002", "00000003", "00000004", "00000005", "00000006", "00000007", "00000008", "00000009"), + HoodieTimeline.CLEAN_ACTION), expectedActiveInstants, commitsAfterArchival); } @Test - public void testArchiveCompletedRollbackAndClean() throws IOException { + public void testArchiveCompletedRollbackAndClean() throws Exception { + init(); int minInstantsToKeep = 2; int maxInstantsToKeep = 10; HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) - .build(); + HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).forTable("test-trip-table") + .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); metaClient = HoodieTableMetaClient.reload(metaClient); int startInstant = 1; @@ -612,12 +488,14 @@ public void testArchiveCompletedRollbackAndClean() throws IOException { } @Test - public void testArchiveInflightClean() throws IOException { + public void testArchiveInflightClean() throws Exception { + init(); HoodieWriteConfig cfg = - HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) - .withParallelism(2, 2).forTable("test-trip-table") - .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) - .build(); + HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).forTable("test-trip-table") + .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); metaClient = HoodieTableMetaClient.reload(metaClient); createCleanMetadata("10", false); @@ -636,6 +514,137 @@ public void testArchiveInflightClean() throws IOException { assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2, notArchivedInstant3), ""); } + @Test + public void testArchiveTableWithMetadataTableCompaction() throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 2, 4, 7); + + // min archival commits is 2 and max archival commits is 4. and so, after 5th commit, ideally archival should kick in. but max delta commits in metadata table is set to 6. and so + // archival will kick in only by 7th commit in datatable(1 commit for bootstrap + 6 commits from data table). + // and then 2nd compaction will take place + for (int i = 1; i < 6; i++) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // trigger archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); + } + + // one more commit will trigger compaction in metadata table and will let archival move forward. + testTable.doWriteOperation("00000006", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // trigger archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + // before archival 1,2,3,4,5,6 + // after archival 5,6 + assertEquals(originalCommits.size() - commitsAfterArchival.size(), 4); + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004")), getActiveCommitInstants(Arrays.asList("00000005", "00000006")), commitsAfterArchival); + + // 3 more commits, 5 and 6 will be archived. but will not move after 6 since compaction has to kick in in metadata table. + testTable.doWriteOperation("00000007", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + testTable.doWriteOperation("00000008", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // trigger archival + commitsList = archiveAndGetCommitsList(writeConfig); + originalCommits = commitsList.getKey(); + commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); + + // ideally, this will archive commits 5, 6, 7, but since compaction in metadata is until 6, only 5 and 6 will get archived, + testTable.doWriteOperation("00000009", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + commitsList = archiveAndGetCommitsList(writeConfig); + originalCommits = commitsList.getKey(); + commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits.size() - commitsAfterArchival.size(), 2); + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006")), + getActiveCommitInstants(Arrays.asList("00000007", "00000008", "00000009")), commitsAfterArchival); + + // and then 2nd compaction will take place at 12th commit + for (int i = 10; i < 13; i++) { + testTable.doWriteOperation("000000" + i, WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // trigger archival + commitsList = archiveAndGetCommitsList(writeConfig); + originalCommits = commitsList.getKey(); + commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); + } + + // one more commit will trigger compaction in metadata table and will let archival move forward. + testTable.doWriteOperation("00000013", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // trigger archival + commitsList = archiveAndGetCommitsList(writeConfig); + originalCommits = commitsList.getKey(); + commitsAfterArchival = commitsList.getValue(); + // before archival 5,6,7,8,9,10,11,12,13 + // after archival 12,13 + assertEquals(originalCommits.size() - commitsAfterArchival.size(), 5); + verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006", "00000007", "00000008", + "00000009", "00000010", "00000011")), getActiveCommitInstants(Arrays.asList("00000012", "00000013")), commitsAfterArchival); + } + + private Pair, List> archiveAndGetCommitsList(HoodieWriteConfig writeConfig) throws IOException { + metaClient.reloadActiveTimeline(); + HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); + List originalCommits = timeline.getInstants().collect(Collectors.toList()); + HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table); + archiveLog.archiveIfRequired(context); + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); + List commitsAfterArchival = timeline.getInstants().collect(Collectors.toList()); + return Pair.of(originalCommits, commitsAfterArchival); + } + + private void verifyArchival(List expectedArchivedInstants, List expectedActiveInstants, List commitsAfterArchival) { + Collections.sort(expectedActiveInstants, Comparator.comparing(HoodieInstant::getTimestamp)); + Collections.sort(commitsAfterArchival, Comparator.comparing(HoodieInstant::getTimestamp)); + assertEquals(expectedActiveInstants, commitsAfterArchival); + expectedArchivedInstants.forEach(entry -> assertFalse(commitsAfterArchival.contains(entry))); + HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient); + List actualArchivedInstants = archivedTimeline.getInstants().collect(Collectors.toList()); + Collections.sort(actualArchivedInstants, Comparator.comparing(HoodieInstant::getTimestamp)); + Collections.sort(expectedArchivedInstants, Comparator.comparing(HoodieInstant::getTimestamp)); + assertEquals(actualArchivedInstants, expectedArchivedInstants); + + HoodieTimeline timeline = metaClient.getActiveTimeline(); + expectedArchivedInstants.forEach(entry -> { + // check safety + if (entry.getAction() != HoodieTimeline.ROLLBACK_ACTION) { + assertTrue(timeline.containsOrBeforeTimelineStarts(entry.getTimestamp()), "Archived commits should always be safe"); + } + } + ); + } + + private List getArchivedInstants(HoodieInstant instant) { + List instants = new ArrayList<>(); + if (instant.getAction() == HoodieTimeline.COMMIT_ACTION || instant.getAction() == HoodieTimeline.DELTA_COMMIT_ACTION || instant.getAction() == HoodieTimeline.CLEAN_ACTION) { + instants.add(new HoodieInstant(State.REQUESTED, instant.getAction(), instant.getTimestamp())); + } + instants.add(new HoodieInstant(State.INFLIGHT, instant.getAction(), instant.getTimestamp())); + instants.add(new HoodieInstant(State.COMPLETED, instant.getAction(), instant.getTimestamp())); + return instants; + } + + private List getAllArchivedCommitInstants(List commitTimes) { + return getAllArchivedCommitInstants(commitTimes, HoodieTimeline.COMMIT_ACTION); + } + + private List getAllArchivedCommitInstants(List commitTimes, String action) { + List allInstants = new ArrayList<>(); + commitTimes.forEach(entry -> allInstants.addAll(getArchivedInstants(new HoodieInstant(State.COMPLETED, action, entry)))); + return allInstants; + } + + private List getActiveCommitInstants(List commitTimes) { + return getActiveCommitInstants(commitTimes, HoodieTimeline.COMMIT_ACTION); + } + + private List getActiveCommitInstants(List commitTimes, String action) { + List allInstants = new ArrayList<>(); + commitTimes.forEach(entry -> allInstants.add(new HoodieInstant(State.COMPLETED, action, entry))); + return allInstants; + } + private HoodieInstant createCleanMetadata(String instantTime, boolean inflightOnly) throws IOException { HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(), CleanPlanV2MigrationHandler.VERSION, new HashMap<>()); @@ -665,14 +674,14 @@ private HoodieInstant createRollbackMetadata(String rollbackTime, String commitT HoodieTestTable.of(metaClient).addInflightRollback(rollbackTime); } else { HoodieRollbackMetadata hoodieRollbackMetadata = HoodieRollbackMetadata.newBuilder() - .setVersion(1) - .setStartRollbackTime(rollbackTime) - .setTotalFilesDeleted(1) - .setTimeTakenInMillis(1000) - .setCommitsRollback(Collections.singletonList(commitToRollback)) - .setPartitionMetadata(Collections.emptyMap()) - .setInstantsRollback(Collections.emptyList()) - .build(); + .setVersion(1) + .setStartRollbackTime(rollbackTime) + .setTotalFilesDeleted(1) + .setTimeTakenInMillis(1000) + .setCommitsRollback(Collections.singletonList(commitToRollback)) + .setPartitionMetadata(Collections.emptyMap()) + .setInstantsRollback(Collections.emptyList()) + .build(); HoodieTestTable.of(metaClient).addRollback(rollbackTime, hoodieRollbackMetadata); } return new HoodieInstant(inflight, "rollback", rollbackTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java index 56f4eeef3402a..76a91ef124bb7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieRowCreateHandle.java @@ -19,6 +19,7 @@ package org.apache.hudi.io.storage.row; import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -168,7 +169,8 @@ public void testGlobalFailure() throws Exception { @Test public void testInstantiationFailure() throws IOException { // init config and table - HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withPath("/dummypath/abc/").build(); + HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .withPath("/dummypath/abc/").build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); try { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 2eb2f380e0bfa..973200eb48309 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -266,6 +266,7 @@ private void testInsertAndCleanByVersions( .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { @@ -298,7 +299,7 @@ private void testInsertAndCleanByVersions( .map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList()); HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Option.empty(), Option.empty()); - List instantTimes = makeIncrementalCommitTimes(9); + List instantTimes = makeIncrementalCommitTimes(9, 1, 10); String compactionTime = instantTimes.get(0); table.getActiveTimeline().saveToCompactionRequested( new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime), @@ -435,6 +436,7 @@ private void testInsertAndCleanByCommits( .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); @@ -511,6 +513,7 @@ private void testFailedInsertAndCleanByCommits( .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build()) .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); @@ -639,7 +642,7 @@ private List runCleaner(HoodieWriteConfig config, boolean simul public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) @@ -731,7 +734,7 @@ public void testKeepLatestFileVersionsMOR() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); @@ -769,7 +772,7 @@ public void testKeepLatestCommitsMOR() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) .build(); @@ -809,7 +812,7 @@ public void testKeepLatestCommitsMOR() throws Exception { @Test public void testCleanWithReplaceCommits() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1116,7 +1119,7 @@ private static Stream argumentsForTestKeepLatestCommits() { @MethodSource("argumentsForTestKeepLatestCommits") public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) @@ -1295,7 +1298,7 @@ public void testCleanMarkerDataFilesOnRollback() throws Exception { @Test public void testCleaningWithZeroPartitionPaths() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1317,7 +1320,7 @@ public void testCleaningWithZeroPartitionPaths() throws Exception { @Test public void testKeepLatestCommitsWithPendingCompactions() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index f0046afe03a70..595d4df2a93a5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -193,7 +194,7 @@ public void testUpsertPartitioner(boolean populateMetaFields) throws Exception { @ValueSource(booleans = {true, false}) public void testLogFileCountsAfterCompaction(boolean populateMetaFields) throws Exception { // insert 100 records - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); HoodieWriteConfig config = cfgBuilder.build(); @@ -524,6 +525,7 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception { JavaRDD deleteRDD = jsc().parallelize(fewRecordsForDelete, 1); // initialize partitioner + hoodieTable.getHoodieView().sync(); AbstractSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable, newDeleteTime, deleteRDD); actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD))); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java index 79c415a4bc268..608d9ca07aa8c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java @@ -52,7 +52,7 @@ public class TestAsyncCompaction extends CompactionTestBase { private HoodieWriteConfig getConfig(Boolean autoCommit) { return getConfigBuilder(autoCommit) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).validate(true).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()) .build(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 14e26b9d4dcbf..cad426492618e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +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; @@ -159,7 +160,10 @@ public void testScheduleCompactionWithInflightInstant() { @Test public void testWriteStatusContentsAfterCompaction() throws Exception { // insert 100 records - HoodieWriteConfig config = getConfig(); + HoodieWriteConfig config = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) { String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index af77dc75378ed..06f70f21ce866 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata; import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; @@ -157,7 +158,7 @@ public void testFailForCompletedInstants() { @Test public void testRollbackWhenFirstCommitFail() throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { client.startCommitWithTime("001"); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java index 03dd3b055f342..a22a04075ea80 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -131,6 +131,7 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese } HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); + hoodieTable.getHoodieView().sync(); FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable); // expect 2 base files for each partition assertEquals(dataGen.getPartitionPaths().length * 2, allFiles.length); @@ -146,6 +147,7 @@ void testClustering(boolean doUpdates, boolean populateMetaFields, boolean prese metaClient = HoodieTableMetaClient.reload(metaClient); final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient); + clusteredTable.getHoodieView().sync(); Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); // verify there should be only one base file per partition after clustering. diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java index e7e7074094250..fd2f63a26c638 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableIncrementalRead.java @@ -20,6 +20,7 @@ package org.apache.hudi.table.functional; import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -83,7 +84,7 @@ public void testIncrementalReadsWithCompaction() throws Exception { Properties props = new Properties(); props.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieFileFormat.PARQUET.toString()); HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, props); - HoodieWriteConfig cfg = getConfig(true); + HoodieWriteConfig cfg = getConfigBuilder(true).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { /* diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java index caecbef52031a..fb44c14f59ad2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java @@ -109,6 +109,7 @@ public void testSimpleInsertAndUpdate(HoodieFileFormat fileFormat, boolean popul client.compact(compactionCommitTime); HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient); + hoodieTable.getHoodieView().sync(); FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable); HoodieTableFileSystemView tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); Stream dataFilesToRead = tableView.getLatestBaseFiles(); @@ -238,6 +239,7 @@ public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception { writeClient.commit(newCommitTime, statuses); HoodieTable table = HoodieSparkTable.create(config, context(), metaClient); + table.getHoodieView().sync(); TableFileSystemView.SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index 7ab5a80e5f446..b1aebc74859ca 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; @@ -51,9 +52,8 @@ import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.File; @@ -136,23 +136,14 @@ void testCOWToMORConvertedTableRollback(boolean rollbackUsingMarkers) throws Exc } } - private static Stream testRollbackWithDeltaAndCompactionCommit() { - return Stream.of( - Arguments.of(true, true), - Arguments.of(true, false), - Arguments.of(false, true), - Arguments.of(false, false) - ); - } - @ParameterizedTest - @MethodSource - void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers, boolean populateMetaFields) throws Exception { + @ValueSource(booleans = {true, false}) + void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) throws Exception { HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false, rollbackUsingMarkers, HoodieIndex.IndexType.SIMPLE); - addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); + addConfigsForPopulateMetaFields(cfgBuilder, true); HoodieWriteConfig cfg = cfgBuilder.build(); - Properties properties = populateMetaFields ? new Properties() : getPropertiesForKeyGen(); + Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString()); HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties); @@ -300,10 +291,10 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers, bool } } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testMultiRollbackWithDeltaAndCompactionCommit(boolean populateMetaFields) throws Exception { - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false); + @Test + void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { + boolean populateMetaFields = true; + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); HoodieWriteConfig cfg = cfgBuilder.build(); @@ -352,7 +343,9 @@ void testMultiRollbackWithDeltaAndCompactionCommit(boolean populateMetaFields) t */ newCommitTime = "002"; // WriteClient with custom config (disable small file handling) - try (SparkRDDWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(populateMetaFields))) { + HoodieWriteConfig smallFileWriteConfig = getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build(); + try (SparkRDDWriteClient nClient = getHoodieWriteClient(smallFileWriteConfig)) { nClient.startCommitWithTime(newCommitTime); List copyOfRecords = new ArrayList<>(records); @@ -463,6 +456,10 @@ void testMultiRollbackWithDeltaAndCompactionCommit(boolean populateMetaFields) t } private HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff(boolean populateMetaFields) { + return getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields).build(); + } + + private HoodieWriteConfig.Builder getHoodieWriteConfigWithSmallFileHandlingOffBuilder(boolean populateMetaFields) { HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder().withPath(basePath()).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2) .withAutoCommit(false) @@ -474,7 +471,7 @@ private HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff(boolean p if (!populateMetaFields) { addConfigsForPopulateMetaFields(cfgBuilder, false); } - return cfgBuilder.build(); + return cfgBuilder; } @ParameterizedTest @@ -592,6 +589,7 @@ void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(boolean rollbackUsi metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(config, context(), metaClient); + table.getHoodieView().sync(); TableFileSystemView.SliceView tableRTFileSystemView = table.getSliceView(); long numLogFiles = 0; 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 c734daecddf2a..03c68eca42ff2 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 @@ -46,6 +46,7 @@ import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.metadata.FileSystemBackedTableMetadata; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; @@ -453,10 +454,14 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom String metadataTableBasePath, boolean doFullValidation) throws IOException { HoodieTableMetadata tableMetadata = metadata(writeConfig, context); assertNotNull(tableMetadata, "MetadataReader should have been initialized"); - if (!writeConfig.isMetadataTableEnabled() || !writeConfig.getMetadataConfig().validateFileListingMetadata()) { + if (!writeConfig.isMetadataTableEnabled()) { return; } + if (!tableMetadata.getSyncedInstantTime().isPresent() || tableMetadata instanceof FileSystemBackedTableMetadata) { + throw new IllegalStateException("Metadata should have synced some commits or tableMetadata should not be an instance " + + "of FileSystemBackedTableMetadata"); + } assertEquals(inflightCommits, testTable.inflightCommits()); HoodieTimer timer = new HoodieTimer().startTimer(); @@ -476,7 +481,7 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom // Files within each partition should match metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext); + HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext, true); TableFileSystemView tableView = table.getHoodieView(); List fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList()); Map partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths); @@ -497,7 +502,7 @@ public void validateMetadata(HoodieTestTable testTable, List inflightCom } public void syncTableMetadata(HoodieWriteConfig writeConfig) { - if (!writeConfig.getMetadataConfig().enableSync()) { + if (!writeConfig.getMetadataConfig().enabled()) { return; } // Open up the metadata table again, for syncing @@ -518,8 +523,8 @@ public HoodieTableMetadata metadata(HoodieWriteConfig clientConfig, HoodieEngine clientConfig.getSpillableMapBasePath()); } - private void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMetadata tableMetadata, TableFileSystemView tableView, - Map partitionToFilesMap, String partition) throws IOException { + protected void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMetadata tableMetadata, TableFileSystemView tableView, + Map partitionToFilesMap, String partition) throws IOException { Path partitionPath; if (partition.equals("")) { // Should be the non-partitioned case @@ -537,8 +542,6 @@ private void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMet Collections.sort(fsFileNames); Collections.sort(metadataFilenames); - assertEquals(fsStatuses.length, partitionToFilesMap.get(basePath + "/" + partition).length); - 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())); @@ -554,6 +557,7 @@ private void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMet } } } + assertEquals(fsStatuses.length, partitionToFilesMap.get(basePath + "/" + partition).length); // Block sizes should be valid Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getBlockSize() > 0)); @@ -586,8 +590,6 @@ private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTab HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig(); assertFalse(metadataWriteConfig.isMetadataTableEnabled(), "No metadata table for metadata table"); - // Metadata table should be in sync with the dataset - assertTrue(metadata(writeConfig, engineContext).isInSync()); HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build(); // Metadata table is MOR @@ -601,7 +603,7 @@ private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTab // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory // in the .hoodie folder. List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath), - false, false, false); + false, false); Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); // Metadata table should automatically compact and clean diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index f55f46bb69488..d085f2c92e763 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -41,23 +41,9 @@ public final class HoodieMetadataConfig extends HoodieConfig { // Enable the internal Metadata Table which saves file listings public static final ConfigProperty ENABLE = ConfigProperty .key(METADATA_PREFIX + ".enable") - .defaultValue(false) - .sinceVersion("0.7.0") - .withDocumentation("Enable the internal metadata table which serves table metadata like level file listings"); - - // Enable syncing the Metadata Table - public static final ConfigProperty SYNC_ENABLE = ConfigProperty - .key(METADATA_PREFIX + ".sync.enable") .defaultValue(true) - .sinceVersion("0.9.0") - .withDocumentation("Enable syncing of metadata table from actions on the dataset"); - - // Validate contents of Metadata Table on each access against the actual filesystem - public static final ConfigProperty VALIDATE_ENABLE = ConfigProperty - .key(METADATA_PREFIX + ".validate") - .defaultValue(false) .sinceVersion("0.7.0") - .withDocumentation("Validate contents of metadata table on each access; e.g against the actual listings from lake storage"); + .withDocumentation("Enable the internal metadata table which serves table metadata like level file listings"); public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false; @@ -149,14 +135,6 @@ public boolean enabled() { return getBoolean(ENABLE); } - public boolean enableSync() { - return enabled() && getBoolean(HoodieMetadataConfig.SYNC_ENABLE); - } - - public boolean validateFileListingMetadata() { - return getBoolean(VALIDATE_ENABLE); - } - public boolean enableMetrics() { return getBoolean(METRICS_ENABLE); } @@ -186,21 +164,11 @@ public Builder enable(boolean enable) { return this; } - public Builder enableSync(boolean enable) { - metadataConfig.setValue(SYNC_ENABLE, String.valueOf(enable)); - return this; - } - public Builder enableMetrics(boolean enableMetrics) { metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics)); return this; } - public Builder validate(boolean validate) { - metadataConfig.setValue(VALIDATE_ENABLE, String.valueOf(validate)); - return this; - } - public Builder withInsertParallelism(int parallelism) { metadataConfig.setValue(INSERT_PARALLELISM_VALUE, String.valueOf(parallelism)); return this; @@ -258,16 +226,6 @@ public HoodieMetadataConfig build() { */ @Deprecated public static final boolean DEFAULT_METADATA_ENABLE = ENABLE.defaultValue(); - /** - * @deprecated Use {@link #VALIDATE_ENABLE} and its methods. - */ - @Deprecated - public static final String METADATA_VALIDATE_PROP = VALIDATE_ENABLE.key(); - /** - * @deprecated Use {@link #VALIDATE_ENABLE} and its methods. - */ - @Deprecated - public static final boolean DEFAULT_METADATA_VALIDATE = VALIDATE_ENABLE.defaultValue(); /** * @deprecated Use {@link #METRICS_ENABLE} and its methods. 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 e96fcce6cde21..8273ca7f35f80 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -271,11 +270,10 @@ public static void processFiles(FileSystem fs, String basePathStr, Function getAllPartitionPaths(HoodieEngineContext engineContext, String basePathStr, - boolean useFileListingFromMetadata, boolean verifyListings, + boolean useFileListingFromMetadata, boolean assumeDatePartitioning) { HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() .enable(useFileListingFromMetadata) - .validate(verifyListings) .withAssumeDatePartitioning(assumeDatePartitioning) .build(); try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, @@ -537,15 +535,6 @@ public static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final return recovered; } - public static void deleteInstantFile(FileSystem fs, String metaPath, HoodieInstant instant) { - try { - LOG.warn("try to delete instant file: " + instant); - fs.delete(new Path(metaPath, instant.getFileName()), false); - } catch (IOException e) { - throw new HoodieIOException("Could not delete instant file" + instant.getFileName(), e); - } - } - public static void createPathIfNotExists(FileSystem fs, Path partitionPath) throws IOException { if (!fs.exists(partitionPath)) { fs.mkdirs(partitionPath); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java index 6bbc02d82a4b6..122c387756e88 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java @@ -32,7 +32,9 @@ public enum HoodieTableVersion { // 0.6.0 onwards ONE(1), // 0.9.0 onwards - TWO(2); + TWO(2), + // 0.10.0 onwards + THREE(3); private final int versionCode; @@ -45,7 +47,7 @@ public int versionCode() { } public static HoodieTableVersion current() { - return TWO; + return THREE; } public static HoodieTableVersion versionFromCode(int versionCode) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java index 9b643ec6e16c8..c566788fd1667 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java @@ -56,6 +56,8 @@ public interface HoodieLogFormat { String UNKNOWN_WRITE_TOKEN = "1-0-1"; + String DEFAULT_WRITE_TOKEN = "0-0-0"; + /** * Writer interface to allow appending block to this file format. */ 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 a68c8f17b1a28..18b267294aa4a 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 @@ -181,52 +181,61 @@ public static class Builder extends AbstractHoodieLogRecordScanner.Builder { protected ExternalSpillableMap.DiskMapType diskMapType = HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue(); protected boolean isBitCaskDiskMapCompressionEnabled = HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue(); // incremental filtering - private Option instantRange = Option.empty(); + protected Option instantRange = Option.empty(); // auto scan default true private boolean autoScan = true; // operation field default false private boolean withOperationField = false; + @Override public Builder withFileSystem(FileSystem fs) { this.fs = fs; return this; } + @Override public Builder withBasePath(String basePath) { this.basePath = basePath; return this; } + @Override public Builder withLogFilePaths(List logFilePaths) { this.logFilePaths = logFilePaths; return this; } + @Override public Builder withReaderSchema(Schema schema) { this.readerSchema = schema; return this; } + @Override public Builder withLatestInstantTime(String latestInstantTime) { this.latestInstantTime = latestInstantTime; return this; } + @Override public Builder withReadBlocksLazily(boolean readBlocksLazily) { this.readBlocksLazily = readBlocksLazily; return this; } + @Override public Builder withReverseReader(boolean reverseReader) { this.reverseReader = reverseReader; return this; } + @Override public Builder withBufferSize(int bufferSize) { this.bufferSize = bufferSize; return this; } + @Override public Builder withInstantRange(Option instantRange) { this.instantRange = instantRange; return this; 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 5b60b033c25bc..eca25e1e39f01 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 @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -172,6 +173,14 @@ public void deletePending(HoodieInstant instant) { deleteInstantFile(instant); } + public static void deleteInstantFile(FileSystem fs, String metaPath, HoodieInstant instant) { + try { + fs.delete(new Path(metaPath, instant.getFileName()), false); + } catch (IOException e) { + throw new HoodieIOException("Could not delete instant file" + instant.getFileName(), e); + } + } + public void deletePendingIfExists(HoodieInstant.State state, String action, String instantStr) { HoodieInstant instant = new HoodieInstant(state, action, instantStr); ValidationUtils.checkArgument(!instant.isCompleted()); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index d4bf2dd209f5d..552adfa8f3490 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -229,7 +229,14 @@ public HoodieTimeline getCleanerTimeline() { */ public HoodieTimeline getRollbackTimeline() { return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); + } + + /** + * Get only the rollback and restore action (inflight and completed) in the active timeline. + */ + public HoodieTimeline getRollbackAndRestoreTimeline() { + return getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION, RESTORE_ACTION)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 44850b9e5acfa..1690c9a6be66b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -26,12 +26,8 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecordPayload; 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.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieMetadataException; @@ -42,13 +38,10 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public abstract class BaseTableMetadata implements HoodieTableMetadata { @@ -59,22 +52,21 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { protected final transient HoodieEngineContext engineContext; protected final SerializableConfiguration hadoopConf; - protected final String datasetBasePath; - protected final HoodieTableMetaClient datasetMetaClient; + protected final String dataBasePath; + protected final HoodieTableMetaClient dataMetaClient; protected final Option metrics; protected final HoodieMetadataConfig metadataConfig; // Directory used for Spillable Map when merging records protected final String spillableMapDirectory; protected boolean enabled; - private TimelineMergedTableMetadata timelineMergedMetadata; protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, - String datasetBasePath, String spillableMapDirectory) { + String dataBasePath, String spillableMapDirectory) { this.engineContext = engineContext; this.hadoopConf = new SerializableConfiguration(engineContext.getHadoopConf()); - this.datasetBasePath = datasetBasePath; - this.datasetMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(datasetBasePath).build(); + this.dataBasePath = dataBasePath; + this.dataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(dataBasePath).build(); this.spillableMapDirectory = spillableMapDirectory; this.metadataConfig = metadataConfig; @@ -104,7 +96,7 @@ public List getAllPartitionPaths() throws IOException { throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); } } - return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, + return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, dataBasePath, metadataConfig.shouldAssumeDatePartitioning()).getAllPartitionPaths(); } @@ -129,7 +121,7 @@ public FileStatus[] getAllFilesInPartition(Path partitionPath) } } - return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()) + return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, dataBasePath, metadataConfig.shouldAssumeDatePartitioning()) .getAllFilesInPartition(partitionPath); } @@ -149,7 +141,7 @@ public Map getAllFilesInPartitions(List partitionP } } - return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()) + return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, dataBasePath, metadataConfig.shouldAssumeDatePartitioning()) .getAllFilesInPartitions(partitionPaths); } @@ -158,7 +150,7 @@ public Map getAllFilesInPartitions(List partitionP */ protected List fetchAllPartitionPaths() throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); - Option> hoodieRecord = getMergedRecordByKey(RECORDKEY_PARTITION_LIST); + Option> hoodieRecord = getRecordByKeyFromMetadata(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer())); List partitions = Collections.emptyList(); @@ -176,28 +168,6 @@ protected List fetchAllPartitionPaths() throws IOException { } } - if (metadataConfig.validateFileListingMetadata()) { - // Validate the Metadata Table data by listing the partitions from the file system - timer.startTimer(); - FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(getEngineContext(), - hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()); - 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; } @@ -208,13 +178,13 @@ protected List fetchAllPartitionPaths() throws IOException { * @param partitionPath The absolute path of the partition */ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { - String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), partitionPath); + String partitionName = FSUtils.getRelativePartitionPath(new Path(dataBasePath), partitionPath); if (partitionName.isEmpty()) { partitionName = NON_PARTITIONED_NAME; } HoodieTimer timer = new HoodieTimer().startTimer(); - Option> hoodieRecord = getMergedRecordByKey(partitionName); + Option> hoodieRecord = getRecordByKeyFromMetadata(partitionName, MetadataPartitionType.FILES.partitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); FileStatus[] statuses = {}; @@ -226,101 +196,11 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath); } - if (metadataConfig.validateFileListingMetadata()) { - // Validate the Metadata Table data by listing the partitions from the file system - timer.startTimer(); - - String partitionPathStr = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), partitionPath); - String latestDataInstantTime = getLatestDatasetInstantTime(); - HoodieTableFileSystemView dataFsView = new HoodieTableFileSystemView(datasetMetaClient, datasetMetaClient.getActiveTimeline()); - List directStatuses = dataFsView.getAllFileSlices(partitionPathStr).flatMap(slice -> { - List paths = new ArrayList<>(); - slice.getBaseFile().ifPresent(baseFile -> { - if (HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, latestDataInstantTime)) { - paths.add(baseFile.getFileStatus()); - } - }); - //TODO(metadata): this will remain problematic; no way to know the commit time based on log file written - slice.getLogFiles().forEach(logFile -> paths.add(logFile.getFileStatus())); - return paths.stream(); - }).collect(Collectors.toList()); - - List directFilenames = directStatuses.stream() - .map(fileStatus -> fileStatus.getPath().getName()).sorted() - .collect(Collectors.toList()); - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer())); - - 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.toArray(new FileStatus[0]); - } - 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) { - Option> mergedRecord; - Option> metadataHoodieRecord = getRecordByKeyFromMetadata(key); - // Retrieve record from unsynced timeline instants - Option> timelineHoodieRecord = timelineMergedMetadata.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); - - protected void openTimelineScanner(HoodieActiveTimeline metadataTableTimeline) { - if (timelineMergedMetadata == null) { - List unSyncedInstants = findInstantsToSyncForReader(); - timelineMergedMetadata = - new TimelineMergedTableMetadata(datasetMetaClient, metadataTableTimeline, unSyncedInstants, getUpdateTime(), null); - } - } - - /** - * Return the instants which are not-synced to the {@code HoodieTableMetadata}. - * - * This is the list of all completed but un-synched instants. - */ - protected abstract List findInstantsToSyncForReader(); - - /** - * Return the instants which are not-synced to the {@code HoodieTableMetadataWriter}. - * - * This is the list of all completed but un-synched instants which do not have any incomplete instants in between them. - */ - protected abstract List findInstantsToSyncForWriter(); - - @Override - public boolean isInSync() { - return enabled && findInstantsToSyncForWriter().isEmpty(); - } + protected abstract Option> getRecordByKeyFromMetadata(String key, String partitionName); protected HoodieEngineContext getEngineContext() { return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get()); @@ -330,15 +210,8 @@ public HoodieMetadataConfig getMetadataConfig() { return metadataConfig; } - protected String getLatestDatasetInstantTime() { - return datasetMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() + protected String getLatestDataInstantTime() { + return dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); } - - public Option getReaderTime() { - if (timelineMergedMetadata == null) { - return Option.empty(); - } - return timelineMergedMetadata.getSyncedInstantTime(); - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java index bb3115ae30c39..f5e14ba1dd34d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -126,13 +126,13 @@ public Map getAllFilesInPartitions(List partitionP } @Override - public Option getUpdateTime() { + public Option getSyncedInstantTime() { throw new UnsupportedOperationException(); } @Override - public boolean isInSync() { - return true; + public Option getLatestCompactionTime() { + throw new UnsupportedOperationException(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 554a165623b22..b0940a7f3469c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -20,6 +20,8 @@ import org.apache.hudi.avro.HoodieAvroUtils; 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.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; @@ -32,9 +34,9 @@ import org.apache.hudi.common.table.HoodieTableConfig; 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.view.HoodieTableFileSystemView; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; @@ -42,6 +44,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; @@ -56,15 +59,15 @@ 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.Set; +import java.util.concurrent.ConcurrentHashMap; 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 { @@ -72,16 +75,13 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { private String metadataBasePath; // Metadata table's timeline and metaclient - private HoodieTableMetaClient metaClient; - private HoodieTableConfig tableConfig; - private List latestFileSystemMetadataSlices; + private HoodieTableMetaClient metadataMetaClient; + private HoodieTableConfig metadataTableConfig; // should we reuse the open file handles, across calls private final boolean reuse; - - // Readers for the base and log file which store the metadata - private transient HoodieFileReader baseFileReader; - private transient HoodieMetadataMergedLogRecordScanner logRecordScanner; + // Readers for latest file slice corresponding to file groups in the metadata partition of interest + private Map> partitionReaders = new ConcurrentHashMap<>(); public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, String spillableMapDirectory) { @@ -96,40 +96,37 @@ public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetada } private void initIfNeeded() { + this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath); if (!enabled) { - LOG.info("Metadata table is disabled for " + datasetBasePath); - } else if (this.metaClient == null) { - this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath); + if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) { + LOG.info("Metadata table is disabled."); + } + } else if (this.metadataMetaClient == null) { try { - this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build(); - this.tableConfig = metaClient.getTableConfig(); - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); - latestFileSystemMetadataSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build(); + this.metadataTableConfig = metadataMetaClient.getTableConfig(); } catch (TableNotFoundException e) { LOG.warn("Metadata table was not found at path " + metadataBasePath); this.enabled = false; - this.metaClient = null; - this.tableConfig = null; + this.metadataMetaClient = null; + this.metadataTableConfig = null; } catch (Exception e) { LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e); this.enabled = false; - this.metaClient = null; - this.tableConfig = null; - } - - if (enabled) { - openTimelineScanner(metaClient.getActiveTimeline()); + this.metadataMetaClient = null; + this.metadataTableConfig = null; } } } @Override - protected Option> getRecordByKeyFromMetadata(String key) { - - openReadersIfNeededOrThrow(); + protected Option> getRecordByKeyFromMetadata(String key, String partitionName) { + Pair readers = openReadersIfNeeded(key, partitionName); try { List timings = new ArrayList<>(); HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieFileReader baseFileReader = readers.getKey(); + HoodieMetadataMergedLogRecordScanner logRecordScanner = readers.getRight(); // Retrieve record from base file HoodieRecord hoodieRecord = null; @@ -137,10 +134,10 @@ protected Option> getRecordByKeyFromMetadata HoodieTimer readTimer = new HoodieTimer().startTimer(); Option baseRecord = baseFileReader.getRecordByKey(key); if (baseRecord.isPresent()) { - hoodieRecord = tableConfig.populateMetaFields() - ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), tableConfig.getPayloadClass(), tableConfig.getPreCombineField(), false) - : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), tableConfig.getPayloadClass(), tableConfig.getPreCombineField(), - Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp()), false); + hoodieRecord = metadataTableConfig.populateMetaFields() + ? SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), false) + : SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), + Pair.of(metadataTableConfig.getRecordKeyFieldProp(), metadataTableConfig.getPartitionFieldProp()), false); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); } } @@ -167,173 +164,173 @@ protected Option> getRecordByKeyFromMetadata throw new HoodieIOException("Error merging records from metadata table for key :" + key, ioe); } finally { if (!reuse) { - closeOrThrow(); + close(partitionName); } } } - private void openReadersIfNeededOrThrow() { - try { - openReadersIfNeeded(); - } catch (IOException e) { - throw new HoodieIOException("Error opening readers to the Metadata Table: ", e); - } - } - /** * Returns a new pair of readers to the base and log files. */ - private void openReadersIfNeeded() throws IOException { - if (reuse && (baseFileReader != null || logRecordScanner != null)) { - // quickly exit out without synchronizing if reusing and readers are already open - return; - } - - // we always force synchronization, if reuse=false, to handle concurrent close() calls as well. - synchronized (this) { - if (baseFileReader != null || logRecordScanner != null) { - return; - } - - final long baseFileOpenMs; - final long logScannerOpenMs; - - // Metadata is in sync till the latest completed instant on the dataset - HoodieTimer timer = new HoodieTimer().startTimer(); - String latestInstantTime = getLatestDatasetInstantTime(); - ValidationUtils.checkArgument(latestFileSystemMetadataSlices.size() == 1, "must be at-least one valid metadata file slice"); - - // If the base file is present then create a reader - Option basefile = latestFileSystemMetadataSlices.get(0).getBaseFile(); - if (basefile.isPresent()) { - String basefilePath = basefile.get().getPath(); - baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); - baseFileOpenMs = timer.endTimer(); - LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", basefilePath, - basefile.get().getCommitTime(), baseFileOpenMs)); - } else { - baseFileOpenMs = 0; - timer.endTimer(); + private Pair openReadersIfNeeded(String key, String partitionName) { + return partitionReaders.computeIfAbsent(partitionName, k -> { + try { + final long baseFileOpenMs; + final long logScannerOpenMs; + HoodieFileReader baseFileReader = null; + HoodieMetadataMergedLogRecordScanner logRecordScanner = null; + + // Metadata is in sync till the latest completed instant on the dataset + HoodieTimer timer = new HoodieTimer().startTimer(); + List latestFileSlices = HoodieTableMetadataUtil.loadPartitionFileGroupsWithLatestFileSlices(metadataMetaClient, partitionName); + ValidationUtils.checkArgument(latestFileSlices.size() == 1, String.format("Invalid number of file slices: found=%d, required=%d", latestFileSlices.size(), 1)); + final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key, latestFileSlices.size())); + + // Open base file reader + Pair baseFileReaderOpenTimePair = getBaseFileReader(slice, timer); + baseFileReader = baseFileReaderOpenTimePair.getKey(); + baseFileOpenMs = baseFileReaderOpenTimePair.getValue(); + + // Open the log record scanner using the log files from the latest file slice + Pair logRecordScannerOpenTimePair = getLogRecordScanner(slice); + logRecordScanner = logRecordScannerOpenTimePair.getKey(); + logScannerOpenMs = logRecordScannerOpenTimePair.getValue(); + + metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs)); + return Pair.of(baseFileReader, logRecordScanner); + } catch (IOException e) { + throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e); } - - // Open the log record scanner using the log files from the latest file slice - timer.startTimer(); - List logFilePaths = latestFileSystemMetadataSlices.get(0).getLogFiles() - .sorted(HoodieLogFile.getLogFileComparator()) - .map(o -> o.getPath().toString()) - .collect(Collectors.toList()); - Option lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); - String latestMetaInstantTimestamp = lastInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); - - // Load the schema - Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); - HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(metadataConfig.getProps()).build(); - logRecordScanner = HoodieMetadataMergedLogRecordScanner.newBuilder() - .withFileSystem(metaClient.getFs()) - .withBasePath(metadataBasePath) - .withLogFilePaths(logFilePaths) - .withReaderSchema(schema) - .withLatestInstantTime(latestMetaInstantTimestamp) - .withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableMapDirectory) - .withDiskMapType(commonConfig.getSpillableDiskMapType()) - .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled()) - .build(); - - logScannerOpenMs = timer.endTimer(); - LOG.info(String.format("Opened metadata log files from %s at instant (dataset instant=%s, metadata instant=%s) in %d ms", - logFilePaths, latestInstantTime, latestMetaInstantTimestamp, logScannerOpenMs)); - - metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, baseFileOpenMs + logScannerOpenMs)); - } + }); } - private void close(HoodieFileReader localFileReader, HoodieMetadataMergedLogRecordScanner localLogScanner) { - try { - if (localFileReader != null) { - localFileReader.close(); - } - if (localLogScanner != null) { - localLogScanner.close(); - } - } catch (Exception e) { - throw new HoodieException("Error closing resources during metadata table merge", e); + private Pair getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException { + HoodieFileReader baseFileReader = null; + Long baseFileOpenMs; + // If the base file is present then create a reader + Option basefile = slice.getBaseFile(); + if (basefile.isPresent()) { + String basefilePath = basefile.get().getPath(); + baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); + baseFileOpenMs = timer.endTimer(); + LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", basefilePath, + basefile.get().getCommitTime(), baseFileOpenMs)); + } else { + baseFileOpenMs = 0L; + timer.endTimer(); } + return Pair.of(baseFileReader, baseFileOpenMs); } - private void closeOrThrow() { - try { - close(); - } catch (Exception e) { - throw new HoodieException("Error closing metadata table readers", e); - } + private Set getValidInstantTimestamps() { + // Only those log files which have a corresponding completed instant on the dataset should be read + // This is because the metadata table is updated before the dataset instants are committed. + HoodieActiveTimeline datasetTimeline = dataMetaClient.getActiveTimeline(); + Set validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstants() + .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + + // For any rollbacks and restores, we cannot neglect the instants that they are rolling back. + // The rollback instant should be more recent than the start of the timeline for it to have rolled back any + // instant which we have a log block for. + final String earliestInstantTime = validInstantTimestamps.isEmpty() ? SOLO_COMMIT_TIMESTAMP : Collections.min(validInstantTimestamps); + datasetTimeline.getRollbackAndRestoreTimeline().filterCompletedInstants().getInstants() + .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN, earliestInstantTime)) + .forEach(instant -> { + validInstantTimestamps.addAll(getRollbackedCommits(instant, datasetTimeline)); + }); + + // SOLO_COMMIT_TIMESTAMP is used during bootstrap so it is a valid timestamp + validInstantTimestamps.add(SOLO_COMMIT_TIMESTAMP); + return validInstantTimestamps; } - @Override - public synchronized void close() throws Exception { - close(baseFileReader, logRecordScanner); - baseFileReader = null; - logRecordScanner = null; + private Pair getLogRecordScanner(FileSlice slice) { + HoodieTimer timer = new HoodieTimer().startTimer(); + List logFilePaths = slice.getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(o -> o.getPath().toString()) + .collect(Collectors.toList()); + + // Only those log files which have a corresponding completed instant on the dataset should be read + // This is because the metadata table is updated before the dataset instants are committed. + Set validInstantTimestamps = getValidInstantTimestamps(); + + Option latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); + String latestMetadataInstantTime = latestMetadataInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + + // Load the schema + Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(metadataConfig.getProps()).build(); + HoodieMetadataMergedLogRecordScanner logRecordScanner = HoodieMetadataMergedLogRecordScanner.newBuilder() + .withFileSystem(metadataMetaClient.getFs()) + .withBasePath(metadataBasePath) + .withLogFilePaths(logFilePaths) + .withReaderSchema(schema) + .withLatestInstantTime(latestMetadataInstantTime) + .withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES) + .withBufferSize(BUFFER_SIZE) + .withSpillableMapBasePath(spillableMapDirectory) + .withDiskMapType(commonConfig.getSpillableDiskMapType()) + .withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled()) + .withLogBlockTimestamps(validInstantTimestamps) + .build(); + + Long logScannerOpenMs = timer.endTimer(); + LOG.info(String.format("Opened %d metadata log files (dataset instant=%s, metadata instant=%s) in %d ms", + logFilePaths.size(), getLatestDataInstantTime(), latestMetadataInstantTime, logScannerOpenMs)); + return Pair.of(logRecordScanner, logScannerOpenMs); } /** - * Return the timestamp of the latest synced instant. + * Returns a list of commits which were rolled back as part of a Rollback or Restore operation. + * + * @param instant The Rollback operation to read + * @param timeline instant of timeline from dataset. */ - @Override - public Option getUpdateTime() { - if (!enabled) { - return Option.empty(); - } - - HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); - return timeline.getDeltaCommitTimeline().filterCompletedInstants() - .lastInstant().map(HoodieInstant::getTimestamp); - } + private List getRollbackedCommits(HoodieInstant instant, HoodieActiveTimeline timeline) { + try { + if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + timeline.getInstantDetails(instant).get()); + return rollbackMetadata.getCommitsRollback(); + } - /** - * Return an ordered list of instants which have not been synced to the Metadata Table. - */ - @Override - protected List findInstantsToSyncForReader() { - return findInstantsToSync(true); + List rollbackedCommits = new LinkedList<>(); + if (instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)) { + // Restore is made up of several rollbacks + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( + timeline.getInstantDetails(instant).get()); + restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { + rms.forEach(rm -> rollbackedCommits.addAll(rm.getCommitsRollback())); + }); + } + return rollbackedCommits; + } catch (IOException e) { + throw new HoodieMetadataException("Error retrieving rollback commits for instant " + instant, e); + } } - /** - * Return an ordered list of instants which have not been synced to the Metadata Table. - */ @Override - protected List findInstantsToSyncForWriter() { - return findInstantsToSync(false); - } - - /** - * Return an ordered list of instants which have not been synced to the Metadata Table. - */ - private List findInstantsToSync(boolean ignoreIncompleteInstants) { - initIfNeeded(); - - // if there are no instants yet, return empty list, since there is nothing to sync here. - if (!enabled || !metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().isPresent()) { - return Collections.EMPTY_LIST; + public void close() { + for (String partitionName : partitionReaders.keySet()) { + close(partitionName); } + partitionReaders.clear(); + } - // All instants on the data timeline, which are greater than the last deltacommit instant on metadata timeline - // are candidates for sync. We only consider delta-commit instants as each actions on dataset leads to a - // deltacommit on the metadata table. - String latestMetadataInstantTime = metaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() - .lastInstant().get().getTimestamp(); - HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE); - Option earliestIncompleteInstant = ignoreIncompleteInstants ? Option.empty() - : 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()); + private synchronized void close(String partitionName) { + Pair readers = partitionReaders.remove(partitionName); + if (readers != null) { + try { + if (readers.getKey() != null) { + readers.getKey().close(); + } + if (readers.getValue() != null) { + readers.getValue().close(); + } + } catch (Exception e) { + throw new HoodieException("Error closing resources during metadata table merge", e); + } } } @@ -345,11 +342,33 @@ public SerializableConfiguration getHadoopConf() { return hadoopConf; } - public HoodieTableMetaClient getMetaClient() { - return metaClient; + public HoodieTableMetaClient getMetadataMetaClient() { + return metadataMetaClient; } public Map stats() { - return metrics.map(m -> m.getStats(true, metaClient, this)).orElse(new HashMap<>()); + return metrics.map(m -> m.getStats(true, metadataMetaClient, this)).orElse(new HashMap<>()); + } + + @Override + public Option getSyncedInstantTime() { + if (metadataMetaClient != null) { + Option latestInstant = metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant(); + if (latestInstant.isPresent()) { + return Option.of(latestInstant.get().getTimestamp()); + } + } + return Option.empty(); + } + + @Override + public Option getLatestCompactionTime() { + if (metadataMetaClient != null) { + Option latestCompaction = metadataMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant(); + if (latestCompaction.isPresent()) { + return Option.of(latestCompaction.get().getTimestamp()); + } + } + return Option.empty(); } } 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 index a3c3e086f24c9..3132ea6346f3b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; @@ -37,15 +38,17 @@ * 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; private HoodieMetadataMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize, String spillableMapBasePath, Set mergeKeyFilter, - ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { + ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled, + Option instantRange) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize, - spillableMapBasePath, Option.empty(), false, diskMapType, isBitCaskDiskMapCompressionEnabled, false); + spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false); this.mergeKeyFilter = mergeKeyFilter; performScan(); @@ -88,59 +91,71 @@ public Option> getRecordByKey(String key) { public static class Builder extends HoodieMergedLogRecordScanner.Builder { private Set mergeKeyFilter = Collections.emptySet(); + @Override public Builder withFileSystem(FileSystem fs) { this.fs = fs; return this; } + @Override public Builder withBasePath(String basePath) { this.basePath = basePath; return this; } + @Override public Builder withLogFilePaths(List logFilePaths) { this.logFilePaths = logFilePaths; return this; } + @Override public Builder withReaderSchema(Schema schema) { this.readerSchema = schema; return this; } + @Override public Builder withLatestInstantTime(String latestInstantTime) { this.latestInstantTime = latestInstantTime; return this; } + @Override public Builder withReadBlocksLazily(boolean readBlocksLazily) { throw new UnsupportedOperationException(); } + @Override public Builder withReverseReader(boolean reverseReader) { throw new UnsupportedOperationException(); } + @Override public Builder withBufferSize(int bufferSize) { this.bufferSize = bufferSize; return this; } + @Override public Builder withMaxMemorySizeInBytes(Long maxMemorySizeInBytes) { this.maxMemorySizeInBytes = maxMemorySizeInBytes; return this; } + @Override public Builder withSpillableMapBasePath(String spillableMapBasePath) { this.spillableMapBasePath = spillableMapBasePath; return this; } + @Override public Builder withDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) { this.diskMapType = diskMapType; return this; } + @Override public Builder withBitCaskDiskMapCompressionEnabled(boolean isBitCaskDiskMapCompressionEnabled) { this.isBitCaskDiskMapCompressionEnabled = isBitCaskDiskMapCompressionEnabled; return this; @@ -151,11 +166,33 @@ public Builder withMergeKeyFilter(Set mergeKeyFilter) { return this; } + public Builder withLogBlockTimestamps(Set validLogBlockTimestamps) { + withInstantRange(Option.of(new ExplicitMatchRange(validLogBlockTimestamps))); + return this; + } + @Override public HoodieMetadataMergedLogRecordScanner build() { return new HoodieMetadataMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter, - diskMapType, isBitCaskDiskMapCompressionEnabled); + diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange); + } + } + + /** + * Class to assist in checking if an instant is part of a set of instants. + */ + private static class ExplicitMatchRange extends InstantRange { + Set instants; + + public ExplicitMatchRange(Set instants) { + super(Collections.min(instants), Collections.max(instants)); + this.instants = instants; + } + + @Override + public boolean isInRange(String instant) { + return this.instants.contains(instant); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java index 5924d8989e6b0..2efc96c6f3dee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -41,13 +41,9 @@ 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"; public static final String REBOOTSTRAP_STR = "rebootstrap"; public static final String BOOTSTRAP_ERR_STR = "bootstrap_error"; @@ -57,7 +53,6 @@ public class HoodieMetadataMetrics implements Serializable { 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); @@ -82,32 +77,35 @@ private Map getStats(HoodieTableFileSystemView fsView, boolean d 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; + for (String metadataPartition : MetadataPartitionType.all()) { + List latestSlices = fsView.getLatestFileSlices(metadataPartition).collect(Collectors.toList()); + + // Total size of the metadata and count of base/log files + long totalBaseFileSizeInBytes = 0; + long totalLogFileSizeInBytes = 0; + int baseFileCount = 0; + int logFileCount = 0; + + 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().getFileSize(); + ++logFileCount; + } } - 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)); + stats.put(metadataPartition + "." + STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes)); + stats.put(metadataPartition + "." + STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes)); + stats.put(metadataPartition + "." + STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount)); + stats.put(metadataPartition + "." + 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; @@ -121,26 +119,20 @@ protected void updateMetrics(String action, long durationInMs) { // 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)); + incrementMetric(countKey, 1); + incrementMetric(durationKey, durationInMs); } - public void updateMetrics(long totalBaseFileSizeInBytes, long totalLogFileSizeInBytes, int baseFileCount, - int logFileCount) { - if (metricsRegistry == null) { - return; + public void updateSizeMetrics(HoodieTableMetaClient metaClient, HoodieBackedTableMetadata metadata) { + Map stats = getStats(false, metaClient, metadata); + for (Map.Entry e : stats.entrySet()) { + incrementMetric(e.getKey(), Long.parseLong(e.getValue())); } + } - // Set new size and count for metadata table's data files - metricsRegistry.set("basefile.size", totalBaseFileSizeInBytes); - metricsRegistry.set("logfile.size", totalLogFileSizeInBytes); - metricsRegistry.set("basefile.count", baseFileCount); - metricsRegistry.set("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)); + protected void incrementMetric(String action, long value) { + LOG.info(String.format("Updating metadata metrics (%s=%d) in %s", action, value, metricsRegistry)); + metricsRegistry.add(action, value); } public Registry registry() { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index 3964cd1006b75..68273b009fa41 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -105,11 +104,12 @@ static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetad Map getAllFilesInPartitions(List partitionPaths) throws IOException; /** - * Get the instant time at which Metadata Table was last updated. - * - * This is the timestamp of the Instant on the dataset which was last synced to the Metadata Table. + * Get the instant time to which the metadata is synced w.r.t data timeline. */ - Option getUpdateTime(); + Option getSyncedInstantTime(); - boolean isInSync(); + /** + * Returns the timestamp of the latest compaction. + */ + Option getLatestCompactionTime(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 0ee3ba2e04c62..b028056bb70d9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -19,28 +19,30 @@ 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.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; 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.timeline.TimelineMetadataUtils; -import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; + +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.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -59,56 +61,20 @@ public class HoodieTableMetadataUtil { private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class); /** - * Converts a timeline instant to metadata table records. + * Delete the metadata table for the dataset. This will be invoked during upgrade/downgrade operation during which no other + * process should be running. * - * @param datasetMetaClient The meta client associated with the timeline instant - * @param metadataTableTimeline Current timeline of the Metadata Table - * @param instant to fetch and convert to metadata table records - * @return a list of metadata table records - * @throws IOException + * @param basePath base path of the dataset + * @param context instance of {@link HoodieEngineContext}. */ - public static Option> convertInstantToMetaRecords(HoodieTableMetaClient datasetMetaClient, - HoodieActiveTimeline metadataTableTimeline, 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(metadataTableTimeline, rollbackMetadata, instant.getTimestamp(), lastSyncTs)); - break; - case HoodieTimeline.RESTORE_ACTION: - HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( - timeline.getInstantDetails(instant).get()); - records = Option.of(convertMetadataToRecords(metadataTableTimeline, restoreMetadata, instant.getTimestamp(), lastSyncTs)); - break; - case HoodieTimeline.SAVEPOINT_ACTION: - // Nothing to be done here - break; - case HoodieTimeline.REPLACE_COMMIT_ACTION: - HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes( - timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); - // Note: we only add new files created here. Replaced files are removed from metadata later by cleaner. - records = Option.of(convertMetadataToRecords(replaceMetadata, instant.getTimestamp())); - break; - default: - throw new HoodieException("Unknown type of action " + instant.getAction()); + public static void deleteMetadataTable(String basePath, HoodieEngineContext context) { + final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get()); + try { + fs.delete(new Path(metadataTablePath), true); + } catch (Exception e) { + throw new HoodieMetadataException("Failed to remove metadata table from path " + metadataTablePath, e); } - - return records; } /** @@ -134,12 +100,11 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c return; } - int offset = partition.equals(NON_PARTITIONED_NAME) ? 0 : partition.length() + 1; + int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 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()); @@ -155,33 +120,6 @@ public static List convertMetadataToRecords(HoodieCommitMetadata c 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. * @@ -192,7 +130,6 @@ public static List convertMetadataToRecords(HoodieCleanerPlan clea 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.getDeletePathPatterns(); @@ -228,10 +165,17 @@ public static List convertMetadataToRecords(HoodieActiveTimeline m } public static List convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline, - HoodieRollbackMetadata rollbackMetadata, String instantTime, Option lastSyncTs) { + HoodieRollbackMetadata rollbackMetadata, String instantTime, + Option lastSyncTs, boolean wasSynced) { + Map> partitionToAppendedFiles = new HashMap<>(); Map> partitionToDeletedFiles = new HashMap<>(); processRollbackMetadata(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs); + if (!wasSynced) { + // Since the instant-being-rolled-back was never committed to the metadata table, the files added there + // need not be deleted. For MOR Table, the rollback appends logBlocks so we need to keep the appended files. + partitionToDeletedFiles.clear(); + } return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback"); } @@ -249,7 +193,6 @@ private static void processRollbackMetadata(HoodieActiveTimeline metadataTableTi Map> partitionToDeletedFiles, Map> partitionToAppendedFiles, Option lastSyncTs) { - rollbackMetadata.getPartitionMetadata().values().forEach(pm -> { final String instantToRollback = rollbackMetadata.getCommitsRollback().get(0); // Has this rollback produced new files? @@ -368,4 +311,46 @@ private static List convertFilesToRecords(Map return records; } + + /** + * Map a record key to a file group in partition of interest. + * + * Note: For hashing, the algorithm is same as String.hashCode() but is being defined here as hashCode() + * implementation is not guaranteed by the JVM to be consistent across JVM versions and implementations. + * + * @param recordKey record key for which the file group index is looked up for. + * @return An integer hash of the given string + */ + public static int mapRecordKeyToFileGroupIndex(String recordKey, int numFileGroups) { + int h = 0; + for (int i = 0; i < recordKey.length(); ++i) { + h = 31 * h + recordKey.charAt(i); + } + + return Math.abs(Math.abs(h) % numFileGroups); + } + + /** + * Loads the list of file groups for a partition of the Metadata Table with latest file slices. + * + * The list of file slices returned is sorted in the correct order of file group name. + * @param metaClient instance of {@link HoodieTableMetaClient}. + * @param partition The name of the partition whose file groups are to be loaded. + * @return List of latest file slices for all file groups in a given partition. + */ + public static List loadPartitionFileGroupsWithLatestFileSlices(HoodieTableMetaClient metaClient, String partition) { + LOG.info("Loading file groups for metadata table partition " + partition); + + // If there are no commits on the metadata table then the table's default FileSystemView will not return any file + // slices even though we may have initialized them. + HoodieTimeline timeline = metaClient.getActiveTimeline(); + if (timeline.empty()) { + final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.createNewInstantTime()); + timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails); + } + + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline); + return fsView.getLatestFileSlices(partition).sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())) + .collect(Collectors.toList()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java index 0436de707d2fd..380f4d04d34a6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -18,16 +18,31 @@ package org.apache.hudi.metadata; +import java.util.Arrays; +import java.util.List; + public enum MetadataPartitionType { - FILES("files"); + FILES("files", "files-"); + // refers to partition path in metadata table. private final String partitionPath; + // refers to fileId prefix used for all file groups in this partition. + private final String fileIdPrefix; - MetadataPartitionType(String partitionPath) { + MetadataPartitionType(String partitionPath, String fileIdPrefix) { this.partitionPath = partitionPath; + this.fileIdPrefix = fileIdPrefix; } public String partitionPath() { return partitionPath; } + + public String getFileIdPrefix() { + return fileIdPrefix; + } + + public static List all() { + return Arrays.asList(MetadataPartitionType.FILES.partitionPath()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java deleted file mode 100644 index b2aca1f11cc8b..0000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * 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.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.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieException; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -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 TimelineMergedTableMetadata implements Serializable { - - private static final Logger LOG = LogManager.getLogger(TimelineMergedTableMetadata.class); - - HoodieTableMetaClient metaClient; - private List instants; - private Option lastSyncTs; - private Set mergeKeyFilter; - private HoodieActiveTimeline metadataTableTimeline; - - // keep it a simple hash map, so it can be easily passed onto the executors, once merged. - protected final Map> timelineMergedRecords; - - public TimelineMergedTableMetadata(HoodieTableMetaClient metaClient, HoodieActiveTimeline metadataTableTimeline, - List instants, Option lastSyncTs, Set mergeKeyFilter) { - this.metaClient = metaClient; - this.instants = instants; - this.lastSyncTs = lastSyncTs; - this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet(); - this.metadataTableTimeline = metadataTableTimeline; - this.timelineMergedRecords = new HashMap<>(); - - 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, - metadataTableTimeline, 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 (timelineMergedRecords.containsKey(key)) { - // Merge and store the merged record - HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(timelineMergedRecords.get(key).getData(), new Properties()); - timelineMergedRecords.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); - } else { - // Put the record as is - timelineMergedRecords.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) timelineMergedRecords.get(key)); - } - - /** - * Returns the timestamp of the latest synced instant. - */ - public Option getSyncedInstantTime() { - if (instants.isEmpty()) { - return Option.empty(); - } - - return Option.of(instants.get(instants.size() - 1).getTimestamp()); - } -} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 600ee1673d6cf..33a695ed70fd0 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -23,6 +23,7 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -60,6 +61,7 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanerPlan; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCompactionPlan; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRequestedReplaceMetadata; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRestoreMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata; public class FileCreateUtils { @@ -130,6 +132,14 @@ private static void deleteMetaFile(String basePath, String instantTime, String s } } + private static void deleteMetaFile(String basePath, String instantTime, String suffix) throws IOException { + Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + Path metaFilePath = parentPath.resolve(instantTime + suffix); + if (Files.exists(metaFilePath)) { + Files.delete(metaFilePath); + } + } + public static void createCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } @@ -150,6 +160,10 @@ public static void createInflightCommit(String basePath, String instantTime) thr createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); } + public static void createDeltaCommit(String basePath, String instantTime, HoodieCommitMetadata metadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + } + public static void createDeltaCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION); } @@ -166,6 +180,10 @@ public static void createInflightDeltaCommit(String basePath, String instantTime createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION); } + public static void createInflightReplaceCommit(String basePath, String instantTime) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); + } + public static void createReplaceCommit(String basePath, String instantTime, HoodieReplaceCommitMetadata metadata) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); } @@ -210,6 +228,10 @@ public static void createRollbackFile(String basePath, String instantTime, Hoodi createMetaFile(basePath, instantTime, HoodieTimeline.ROLLBACK_EXTENSION, serializeRollbackMetadata(hoodieRollbackMetadata).get()); } + public static void createRestoreFile(String basePath, String instantTime, HoodieRestoreMetadata hoodieRestoreMetadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.RESTORE_ACTION, serializeRestoreMetadata(hoodieRestoreMetadata).get()); + } + private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); Files.createDirectories(parentPath); @@ -224,7 +246,7 @@ public static void createRequestedCompaction(String basePath, String instantTime } public static void createInflightCompaction(String basePath, String instantTime) throws IOException { - createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); + createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); } public static void createPartitionMetaFile(String basePath, String partitionPath) throws IOException { @@ -309,6 +331,10 @@ public static void deleteDeltaCommit(String basePath, String instantTime) throws removeMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION); } + public static void deleteReplaceCommit(String basePath, String instantTime) throws IOException { + removeMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION); + } + public static long getTotalMarkerFileCount(String basePath, String partitionPath, String instantTime, IOType ioType) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime, partitionPath); if (Files.notExists(parentPath)) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 099fec2874a7c..2a829b596a685 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -19,15 +19,13 @@ package org.apache.hudi.common.testutils; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; @@ -55,6 +53,10 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; + +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; @@ -102,6 +104,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createRestoreFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createRollbackFile; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; @@ -114,7 +117,7 @@ public class HoodieTestTable { private static final Logger LOG = LogManager.getLogger(HoodieTestTable.class); private static final Random RANDOM = new Random(); - private static HoodieTestTableState testTableState; + protected static HoodieTestTableState testTableState; private final List inflightCommits = new ArrayList<>(); protected final String basePath; @@ -152,16 +155,19 @@ public static List makeIncrementalCommitTimes(int num) { } public static List makeIncrementalCommitTimes(int num, int firstOffsetSeconds) { + return makeIncrementalCommitTimes(num, firstOffsetSeconds, 0); + } + + public static List makeIncrementalCommitTimes(int num, int firstOffsetSeconds, int deltaSecs) { final Instant now = Instant.now(); return IntStream.range(0, num) - .mapToObj(i -> makeNewCommitTime(now.plus(firstOffsetSeconds + i, SECONDS))) + .mapToObj(i -> makeNewCommitTime(now.plus(deltaSecs == 0 ? (firstOffsetSeconds + i) : (i == 0 ? (firstOffsetSeconds) : (i * deltaSecs) + i), SECONDS))) .collect(Collectors.toList()); } public HoodieTestTable addRequestedCommit(String instantTime) throws Exception { createRequestedCommit(basePath, instantTime); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -170,7 +176,14 @@ public HoodieTestTable addInflightCommit(String instantTime) throws Exception { createInflightCommit(basePath, instantTime); inflightCommits.add(instantTime); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + + public HoodieTestTable addInflightDeltaCommit(String instantTime) throws Exception { + createRequestedDeltaCommit(basePath, instantTime); + createInflightDeltaCommit(basePath, instantTime); + inflightCommits.add(instantTime); + currentInstantTime = instantTime; return this; } @@ -179,7 +192,6 @@ public HoodieTestTable addCommit(String instantTime) throws Exception { createInflightCommit(basePath, instantTime); createCommit(basePath, instantTime); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -210,15 +222,17 @@ public HoodieTestTable addCommit(String instantTime, HoodieCommitMetadata metada createInflightCommit(basePath, instantTime); createCommit(basePath, instantTime, metadata); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { - createCommit(basePath, instantTime, metadata); + if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { + createCommit(basePath, instantTime, metadata); + } else { + createDeltaCommit(basePath, instantTime, metadata); + } inflightCommits.remove(instantTime); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -227,7 +241,14 @@ public HoodieTestTable addDeltaCommit(String instantTime) throws Exception { createInflightDeltaCommit(basePath, instantTime); createDeltaCommit(basePath, instantTime); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + + public HoodieTestTable addDeltaCommit(String instantTime, HoodieCommitMetadata metadata) throws Exception { + createRequestedDeltaCommit(basePath, instantTime); + createInflightDeltaCommit(basePath, instantTime); + createDeltaCommit(basePath, instantTime, metadata); + currentInstantTime = instantTime; return this; } @@ -240,14 +261,12 @@ public HoodieTestTable addReplaceCommit( createInflightReplaceCommit(basePath, instantTime, inflightReplaceMetadata); createReplaceCommit(basePath, instantTime, completeReplaceMetadata); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } public HoodieTestTable addRequestedReplace(String instantTime, Option requestedReplaceMetadata) throws Exception { createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -255,7 +274,6 @@ public HoodieTestTable addInflightClean(String instantTime, HoodieCleanerPlan cl createRequestedCleanFile(basePath, instantTime, cleanerPlan); createInflightCleanFile(basePath, instantTime, cleanerPlan); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -264,7 +282,6 @@ public HoodieTestTable addClean(String instantTime, HoodieCleanerPlan cleanerPla createInflightCleanFile(basePath, instantTime, cleanerPlan); createCleanFile(basePath, instantTime, metadata); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -296,7 +313,6 @@ public Pair getHoodieCleanMetadata(Strin public HoodieTestTable addInflightRollback(String instantTime) throws IOException { createInflightRollbackFile(basePath, instantTime); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); return this; } @@ -304,7 +320,12 @@ public HoodieTestTable addRollback(String instantTime, HoodieRollbackMetadata ro createInflightRollbackFile(basePath, instantTime); createRollbackFile(basePath, instantTime, rollbackMetadata); currentInstantTime = instantTime; - metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + + public HoodieTestTable addRestore(String instantTime, HoodieRestoreMetadata restoreMetadata) throws IOException { + createRestoreFile(basePath, instantTime, restoreMetadata); + currentInstantTime = instantTime; return this; } @@ -319,7 +340,11 @@ public HoodieRollbackMetadata getRollbackMetadata(String instantTimeToDelete, Ma rollbackPartitionMetadata.setSuccessDeleteFiles(entry.getValue()); rollbackPartitionMetadata.setFailedDeleteFiles(new ArrayList<>()); rollbackPartitionMetadata.setWrittenLogFiles(getWrittenLogFiles(instantTimeToDelete, entry)); - rollbackPartitionMetadata.setRollbackLogFiles(createImmutableMap(logFileName(instantTimeToDelete, UUID.randomUUID().toString(), 0), (long) (100 + RANDOM.nextInt(500)))); + long rollbackLogFileSize = 50 + RANDOM.nextInt(500); + String fileId = UUID.randomUUID().toString(); + String logFileName = logFileName(instantTimeToDelete, fileId, 0); + FileCreateUtils.createLogFile(basePath, entry.getKey(), instantTimeToDelete, fileId, 0, (int) rollbackLogFileSize); + rollbackPartitionMetadata.setRollbackLogFiles(createImmutableMap(logFileName, rollbackLogFileSize)); partitionMetadataMap.put(entry.getKey(), rollbackPartitionMetadata); } rollbackMetadata.setPartitionMetadata(partitionMetadataMap); @@ -335,7 +360,7 @@ private Map getWrittenLogFiles(String instant, Map.Entry> fileInfos = testTableState.getPartitionToLogFileInfoMap(instant).get(entry.getKey()); for (Pair fileInfo : fileInfos) { if (fileName.equals(logFileName(instant, fileInfo.getLeft(), fileInfo.getRight()[0]))) { @@ -366,7 +391,6 @@ public HoodieSavepointMetadata getSavepointMetadata(String instant, Map fileSlices = new ArrayList<>(); + for (Map.Entry> entry : commitMetadata.getPartitionToWriteStats().entrySet()) { + for (HoodieWriteStat stat: entry.getValue()) { + fileSlices.add(new FileSlice(entry.getKey(), instantTime, stat.getPath())); + } + } + this.addRequestedCompaction(instantTime, fileSlices.toArray(new FileSlice[0])); + createInflightCompaction(basePath, instantTime); + inflightCommits.add(instantTime); + currentInstantTime = instantTime; + return this; + } + public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception { createRequestedCompaction(basePath, instantTime); createInflightCompaction(basePath, instantTime); - return HoodieTestTable.of(metaClient) - .addCommit(instantTime, commitMetadata); + return addCommit(instantTime, commitMetadata); + } + + public HoodieTestTable moveInflightCompactionToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { + createCommit(basePath, instantTime, metadata); + inflightCommits.remove(instantTime); + currentInstantTime = instantTime; + return this; } public HoodieTestTable forCommit(String instantTime) { @@ -648,6 +692,7 @@ public void deleteFilesInPartition(String partitionPath, List filesToDel } public HoodieTestTable doRollback(String commitTimeToRollback, String commitTime) throws Exception { + metaClient = HoodieTableMetaClient.reload(metaClient); Option commitMetadata = getMetadataForInstant(commitTimeToRollback); if (!commitMetadata.isPresent()) { throw new IllegalArgumentException("Instant to rollback not present in timeline: " + commitTimeToRollback); @@ -660,7 +705,32 @@ public HoodieTestTable doRollback(String commitTimeToRollback, String commitTime return addRollback(commitTime, rollbackMetadata); } - public HoodieTestTable doCluster(String commitTime, Map> partitionToReplaceFileIds) throws Exception { + public HoodieTestTable doRestore(String commitToRestoreTo, String restoreTime) throws Exception { + metaClient = HoodieTableMetaClient.reload(metaClient); + List commitsToRollback = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().findInstantsAfter(commitToRestoreTo).getReverseOrderedInstants().collect(Collectors.toList()); + Map> rollbackMetadataMap = new HashMap<>(); + for (HoodieInstant commitInstantToRollback: commitsToRollback) { + Option commitMetadata = getCommitMeta(commitInstantToRollback); + if (!commitMetadata.isPresent()) { + throw new IllegalArgumentException("Instant to rollback not present in timeline: " + commitInstantToRollback.getTimestamp()); + } + Map> partitionFiles = getPartitionFiles(commitMetadata.get()); + rollbackMetadataMap.put(commitInstantToRollback.getTimestamp(), + Collections.singletonList(getRollbackMetadata(commitInstantToRollback.getTimestamp(), partitionFiles))); + for (Map.Entry> entry : partitionFiles.entrySet()) { + deleteFilesInPartition(entry.getKey(), entry.getValue()); + } + } + + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.convertRestoreMetadata(restoreTime,1000L, + commitsToRollback, rollbackMetadataMap); + return addRestore(restoreTime, restoreMetadata); + } + + public HoodieReplaceCommitMetadata doCluster(String commitTime, Map> partitionToReplaceFileIds, List partitions, int filesPerPartition) throws Exception { + HoodieTestTableState testTableState = getTestTableStateWithPartitionFileInfo(CLUSTER, metaClient.getTableType(), commitTime, partitions, filesPerPartition); + this.currentInstantTime = commitTime; Map>> partitionToReplaceFileIdsWithLength = new HashMap<>(); for (Map.Entry> entry : partitionToReplaceFileIds.entrySet()) { String partition = entry.getKey(); @@ -670,10 +740,15 @@ public HoodieTestTable doCluster(String commitTime, Map> pa partitionToReplaceFileIdsWithLength.get(partition).add(Pair.of(fileId, length)); } } - List writeStats = generateHoodieWriteStatForPartition(partitionToReplaceFileIdsWithLength, commitTime, false); + List writeStats = generateHoodieWriteStatForPartition(testTableState.getPartitionToBaseFileInfoMap(commitTime), commitTime, false); + for (String partition : testTableState.getPartitionToBaseFileInfoMap(commitTime).keySet()) { + this.withBaseFilesInPartition(partition, testTableState.getPartitionToBaseFileInfoMap(commitTime).get(partition)); + } HoodieReplaceCommitMetadata replaceMetadata = - (HoodieReplaceCommitMetadata) buildMetadata(writeStats, partitionToReplaceFileIds, Option.empty(), CLUSTER, EMPTY_STRING, REPLACE_COMMIT_ACTION); - return addReplaceCommit(commitTime, Option.empty(), Option.empty(), replaceMetadata); + (HoodieReplaceCommitMetadata) buildMetadata(writeStats, partitionToReplaceFileIds, Option.empty(), CLUSTER, EMPTY_STRING, + REPLACE_COMMIT_ACTION); + addReplaceCommit(commitTime, Option.empty(), Option.empty(), replaceMetadata); + return replaceMetadata; } public HoodieCleanMetadata doClean(String commitTime, Map partitionFileCountsToDelete) throws IOException { @@ -718,7 +793,11 @@ public HoodieSavepointMetadata doSavepoint(String commitTime) throws IOException return savepointMetadata; } - public HoodieTestTable doCompaction(String commitTime, List partitions) throws Exception { + public HoodieCommitMetadata doCompaction(String commitTime, List partitions) throws Exception { + return doCompaction(commitTime, partitions, false); + } + + public HoodieCommitMetadata doCompaction(String commitTime, List partitions, boolean inflight) throws Exception { this.currentInstantTime = commitTime; if (partitions.isEmpty()) { partitions = Collections.singletonList(EMPTY_STRING); @@ -728,7 +807,12 @@ public HoodieTestTable doCompaction(String commitTime, List partitions) for (String partition : partitions) { this.withBaseFilesInPartition(partition, testTableState.getPartitionToBaseFileInfoMap(commitTime).get(partition)); } - return addCompaction(commitTime, commitMetadata); + if (inflight) { + this.addInflightCompaction(commitTime, commitMetadata); + } else { + this.addCompaction(commitTime, commitMetadata); + } + return commitMetadata; } public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationType operationType, @@ -765,9 +849,17 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy this.withPartitionMetaFiles(str); } if (createInflightCommit) { - this.addInflightCommit(commitTime); + if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { + this.addInflightCommit(commitTime); + } else { + this.addInflightDeltaCommit(commitTime); + } } else { - this.addCommit(commitTime, commitMetadata); + if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { + this.addCommit(commitTime, commitMetadata); + } else { + this.addDeltaCommit(commitTime, commitMetadata); + } } for (String partition : partitions) { this.withBaseFilesInPartition(partition, testTableState.getPartitionToBaseFileInfoMap(commitTime).get(partition)); @@ -779,23 +871,12 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy } private Option getMetadataForInstant(String instantTime) { + metaClient = HoodieTableMetaClient.reload(metaClient); Option hoodieInstant = metaClient.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); try { if (hoodieInstant.isPresent()) { - switch (hoodieInstant.get().getAction()) { - case HoodieTimeline.REPLACE_COMMIT_ACTION: - HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant.get()).get(), HoodieReplaceCommitMetadata.class); - return Option.of(replaceCommitMetadata); - case HoodieTimeline.DELTA_COMMIT_ACTION: - case HoodieTimeline.COMMIT_ACTION: - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant.get()).get(), HoodieCommitMetadata.class); - return Option.of(commitMetadata); - default: - throw new IllegalArgumentException("Unknown instant action" + hoodieInstant.get().getAction()); - } + return getCommitMeta(hoodieInstant.get()); } else { return Option.empty(); } @@ -804,6 +885,22 @@ private Option getMetadataForInstant(String instantTime) { } } + private Option getCommitMeta(HoodieInstant hoodieInstant) throws IOException { + switch (hoodieInstant.getAction()) { + case HoodieTimeline.REPLACE_COMMIT_ACTION: + HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); + return Option.of(replaceCommitMetadata); + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.COMMIT_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); + return Option.of(commitMetadata); + default: + throw new IllegalArgumentException("Unknown instant action" + hoodieInstant.getAction()); + } + } + private static Map> getPartitionFiles(HoodieCommitMetadata commitMetadata) { Map> partitionFilesToDelete = new HashMap<>(); Map> partitionToWriteStats = commitMetadata.getPartitionToWriteStats(); @@ -815,7 +912,7 @@ private static Map> getPartitionFiles(HoodieCommitMetadata } private static HoodieTestTableState getTestTableStateWithPartitionFileInfo(WriteOperationType operationType, HoodieTableType tableType, String commitTime, - List partitions, int filesPerPartition) { + List partitions, int filesPerPartition) { for (String partition : partitions) { Stream fileLengths = IntStream.range(0, filesPerPartition).map(i -> 100 + RANDOM.nextInt(500)).boxed(); if (MERGE_ON_READ.equals(tableType) && UPSERT.equals(operationType)) { @@ -861,7 +958,7 @@ private static List generateHoodieWriteStatForPartitionLogFiles for (Pair fileIdInfo : entry.getValue()) { HoodieWriteStat writeStat = new HoodieWriteStat(); String fileName = bootstrap ? fileIdInfo.getKey() : - FileCreateUtils.logFileName(commitTime, fileIdInfo.getKey(), fileIdInfo.getValue()[0]); + FileCreateUtils.logFileName(commitTime, fileIdInfo.getKey(), fileIdInfo.getValue()[0]); writeStat.setFileId(fileName); writeStat.setPartitionPath(partition); writeStat.setPath(partition + "/" + fileName); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 3049947f8a244..26c159533bad7 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -225,8 +225,6 @@ public void notifyCheckpointComplete(long checkpointId) { startInstant(); // sync Hive if is enabled syncHiveIfEnabled(); - // sync metadata if is enabled - syncMetadataIfEnabled(); } }, "commits the instant %s", this.instant ); @@ -299,23 +297,6 @@ private void initMetadataSync() { this.metadataSyncExecutor = new NonThrownExecutor(LOG, true); } - /** - * Sync the write metadata to the metadata table. - */ - private void syncMetadataIfEnabled() { - if (tableState.syncMetadata) { - this.metadataSyncExecutor.execute(this::syncMetadata, - "sync metadata table for instant %s", this.instant); - } - } - - /** - * Sync the write metadata to the metadata table. - */ - private void syncMetadata() { - this.writeClient.syncTableMetadata(); - } - private void reset() { this.eventBuffer = new WriteMetadataEvent[this.parallelism]; } @@ -366,11 +347,6 @@ private void initInstant(String instant) { LOG.info("Recommit instant {}", instant); commitInstant(instant); } - if (tableState.syncMetadata) { - // initialize metadata table first if enabled - // condition: the data set timeline has committed instants - syncMetadata(); - } // starts a new instant startInstant(); }, "initialize instant %s", instant); @@ -391,8 +367,6 @@ private void handleEndInputEvent(WriteMetadataEvent event) { commitInstant(this.instant); // sync Hive if is enabled in batch mode. syncHiveIfEnabled(); - // sync metadata if is enabled in batch mode. - syncMetadataIfEnabled(); } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index d374882e9e6ed..1c051c8cd2300 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -79,7 +79,6 @@ private static HiveSyncConfig buildSyncConfig(Configuration conf) { hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME); hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC); hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED); - hiveSyncConfig.verifyMetadataFileListing = false; hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS); hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP); hiveSyncConfig.autoCreateDatabase = conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB); diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java b/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java index fc8ce6ac6abd6..54085ebc128ef 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java +++ b/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java @@ -175,8 +175,6 @@ private static HoodieMetadataConfig metadataConfig(org.apache.flink.configuratio // set up metadata.enabled=true in table DDL to enable metadata listing properties.put(HoodieMetadataConfig.ENABLE, conf.getBoolean(FlinkOptions.METADATA_ENABLED)); - properties.put(HoodieMetadataConfig.SYNC_ENABLE, conf.getBoolean(FlinkOptions.METADATA_ENABLED)); - properties.put(HoodieMetadataConfig.VALIDATE_ENABLE, false); return HoodieMetadataConfig.newBuilder().fromProperties(properties).build(); } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index f379893229db8..05387141630ee 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -180,7 +181,7 @@ public void testHiveSyncInvoked() throws Exception { assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1)); } - @Test + @Disabled void testSyncMetadataTable() throws Exception { // reset reset(); diff --git a/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java b/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java index 334df5961314d..060974df73327 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java +++ b/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java @@ -27,6 +27,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -89,8 +90,7 @@ void testFileListingUsingMetadataNonPartitionedTable() throws Exception { assertTrue(fileStatuses[0].getPath().toString().endsWith(HoodieFileFormat.PARQUET.getFileExtension())); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) + @Disabled void testFileListingEmptyTable(boolean enableMetadata) { Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); conf.setBoolean(FlinkOptions.METADATA_ENABLED, enableMetadata); 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 70c43011b0fba..73573043c6162 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 @@ -72,7 +72,6 @@ import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE; -import static org.apache.hudi.common.config.HoodieMetadataConfig.VALIDATE_ENABLE; public class HoodieInputFormatUtils { @@ -419,7 +418,6 @@ public static Map> groupSnapshotPathsByMetaCli public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) { return HoodieMetadataConfig.newBuilder() .enable(conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS)) - .validate(conf.getBoolean(VALIDATE_ENABLE.key(), VALIDATE_ENABLE.defaultValue())) .build(); } 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 1fd3d2f966cd7..2648740f54e0f 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 @@ -88,7 +88,7 @@ protected List getPartitions(Option partitionsLimit) throws IOE // calls in metrics as they are not part of normal HUDI operation. HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient.getBasePath(), - HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false); + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, 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/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index f771bc3d3a533..2618d92affe9c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -125,9 +125,6 @@ case class HoodieFileIndex( properties.put(HoodieMetadataConfig.ENABLE, sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(), HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString)) - properties.put(HoodieMetadataConfig.VALIDATE_ENABLE, - sqlConf.getConfString(HoodieMetadataConfig.VALIDATE_ENABLE.key(), - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue().toString)) properties.putAll(options.asJava) properties } 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 83b37acd5dbef..9726be894be2a 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 @@ -539,7 +539,6 @@ object HoodieSparkSqlWriter { hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS) hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC) hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE) - hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.VALIDATE_ENABLE) hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).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 51b632b50370b..bdb2afb4b48be 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 @@ -18,13 +18,14 @@ package org.apache.hudi import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.common.config.HoodieMetadataConfig.{ENABLE, VALIDATE_ENABLE} +import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE import org.apache.hudi.common.config.{HoodieConfig, TypedProperties} -import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import java.util.Properties import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters.{mapAsScalaMapConverter, _} +import scala.collection.JavaConverters.mapAsScalaMapConverter +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory /** * WriterUtils to assist in write path in Datasource and tests. @@ -50,7 +51,6 @@ object HoodieWriterUtils { PARTITIONPATH_FIELD.key -> PARTITIONPATH_FIELD.defaultValue, KEYGENERATOR_CLASS_NAME.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, ENABLE.key -> ENABLE.defaultValue.toString, - VALIDATE_ENABLE.key -> VALIDATE_ENABLE.defaultValue.toString, COMMIT_METADATA_KEYPREFIX.key -> COMMIT_METADATA_KEYPREFIX.defaultValue, INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue, STREAMING_RETRY_CNT.key -> STREAMING_RETRY_CNT.defaultValue, diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index 59671322b0652..2e89baa70b8bf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -251,6 +252,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec .withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName()) .withBootstrapParallelism(3) .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); client.bootstrap(Option.empty()); @@ -258,8 +260,9 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); // Rollback Bootstrap - FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, + HoodieActiveTimeline.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs)); + metaClient.reloadActiveTimeline(); client.rollbackFailedBootstrap(); metaClient.reloadActiveTimeline(); assertEquals(0, metaClient.getCommitsTimeline().countInstants()); @@ -374,8 +377,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -393,8 +395,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -410,8 +411,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -428,8 +428,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -444,8 +443,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -462,8 +460,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -550,6 +547,7 @@ public Map> select(List "partition", HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName) val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier) + initializeMetaClientForBootstrap(fooTableParams, tableType, true) - val client = spy(DataSourceUtils.createHoodieClient( + val client = spy(DataSourceUtils.createHoodieClient( new JavaSparkContext(sc), null, tempBasePath, @@ -522,6 +524,27 @@ class HoodieSparkSqlWriterSuite { } } + def initializeMetaClientForBootstrap(fooTableParams : Map[String, String], tableType: String, addBootstrapPath : Boolean) : Unit = { + // when metadata is enabled, directly instantiating write client using DataSourceUtils.createHoodieClient + // will hit a code which tries to instantiate meta client for data table. if table does not exist, it fails. + // hence doing an explicit instantiation here. + val tableMetaClientBuilder = HoodieTableMetaClient.withPropertyBuilder() + .setTableType(tableType) + .setTableName(hoodieFooTableName) + .setRecordKeyFields(fooTableParams(DataSourceWriteOptions.RECORDKEY_FIELD.key)) + .setBaseFileFormat(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().name()) + .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) + .setPayloadClassName(fooTableParams(PAYLOAD_CLASS_NAME.key)) + .setPreCombineField(fooTableParams(PRECOMBINE_FIELD.key)) + .setPartitionFields(fooTableParams(DataSourceWriteOptions.PARTITIONPATH_FIELD.key)) + .setKeyGeneratorClassProp(fooTableParams(KEYGENERATOR_CLASS_NAME.key)) + if(addBootstrapPath) { + tableMetaClientBuilder + .setBootstrapBasePath(fooTableParams(HoodieBootstrapConfig.BASE_PATH.key)) + } + tableMetaClientBuilder.initTable(sc.hadoopConfiguration, tempBasePath) + } + /** * Test cases for schema evolution in different types of tables. * 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 bfd4423db16a1..ffe2b4e058f5f 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 @@ -17,6 +17,7 @@ package org.apache.hudi.functional +import org.apache.hadoop.fs.FileSystem import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} @@ -74,6 +75,8 @@ class TestCOWDataSource extends HoodieClientTestBase { cleanupSparkContexts() cleanupTestDataGenerator() cleanupFileSystem() + FileSystem.closeAll() + System.gc() } @Test def testShortNameStorage() { @@ -368,7 +371,7 @@ class TestCOWDataSource extends HoodieClientTestBase { val recordsDF = spark.createDataFrame(rdd, schema) recordsDF.write.format("org.apache.hudi") .options(commonOpts) - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") @@ -401,29 +404,35 @@ class TestCOWDataSource extends HoodieClientTestBase { } private def getDataFrameWriter(keyGenerator: String): DataFrameWriter[Row] = { + getDataFrameWriter(keyGenerator, true) + } + + private def getDataFrameWriter(keyGenerator: String, enableMetadata: Boolean): DataFrameWriter[Row] = { val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2)) - + val opts = commonOpts ++ Map(HoodieMetadataConfig.ENABLE.key() -> String.valueOf(enableMetadata)) inputDF.write.format("hudi") - .options(commonOpts) + .options(opts) .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator) .mode(SaveMode.Overwrite) } @Test def testSparkPartitonByWithCustomKeyGenerator(): Unit = { // Without fieldType, the default is SIMPLE - var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + var writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false) writer.partitionBy("current_ts") + .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") .load(basePath + "/*/*") assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("current_ts").cast("string")).count() == 0) // Specify fieldType as TIMESTAMP - writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false) writer.partitionBy("current_ts:TIMESTAMP") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") + .mode(SaveMode.Overwrite) .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") .load(basePath + "/*/*") @@ -431,10 +440,11 @@ class TestCOWDataSource extends HoodieClientTestBase { assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= udf_date_format(col("current_ts"))).count() == 0) // Mixed fieldType - writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false) writer.partitionBy("driver", "rider:SIMPLE", "current_ts:TIMESTAMP") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") + .mode(SaveMode.Overwrite) .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") .load(basePath + "/*/*/*") @@ -442,7 +452,7 @@ class TestCOWDataSource extends HoodieClientTestBase { concat(col("driver"), lit("/"), col("rider"), lit("/"), udf_date_format(col("current_ts")))).count() == 0) // Test invalid partitionKeyType - writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName) + writer = getDataFrameWriter(classOf[CustomKeyGenerator].getName, false) writer = writer.partitionBy("current_ts:DUMMY") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") @@ -459,9 +469,10 @@ class TestCOWDataSource extends HoodieClientTestBase { // Use the `driver` field as the partition key var writer = getDataFrameWriter(classOf[SimpleKeyGenerator].getName) writer.partitionBy("driver") + .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0) // Use the `driver,rider` field as the partition key, If no such field exists, the default value `default` is used @@ -469,7 +480,7 @@ class TestCOWDataSource extends HoodieClientTestBase { writer.partitionBy("driver", "rider") .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("default")).count() == 0) } @@ -477,9 +488,10 @@ class TestCOWDataSource extends HoodieClientTestBase { // Use the `driver` field as the partition key var writer = getDataFrameWriter(classOf[ComplexKeyGenerator].getName) writer.partitionBy("driver") + .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= col("driver")).count() == 0) // Use the `driver`,`rider` field as the partition key @@ -487,15 +499,16 @@ class TestCOWDataSource extends HoodieClientTestBase { writer.partitionBy("driver", "rider") .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= concat(col("driver"), lit("/"), col("rider"))).count() == 0) } @Test def testSparkPartitonByWithTimestampBasedKeyGenerator() { - val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName) + val writer = getDataFrameWriter(classOf[TimestampBasedKeyGenerator].getName, false) writer.partitionBy("current_ts") .option(Config.TIMESTAMP_TYPE_FIELD_PROP, "EPOCHMILLISECONDS") .option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyyMMdd") + .mode(SaveMode.Overwrite) .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") @@ -507,10 +520,11 @@ class TestCOWDataSource extends HoodieClientTestBase { @Test def testSparkPartitonByWithGlobalDeleteKeyGenerator() { val writer = getDataFrameWriter(classOf[GlobalDeleteKeyGenerator].getName) writer.partitionBy("driver") + .mode(SaveMode.Overwrite) .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0) } @@ -518,17 +532,19 @@ class TestCOWDataSource extends HoodieClientTestBase { // Empty string column var writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName) writer.partitionBy("") + .mode(SaveMode.Overwrite) .save(basePath) var recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0) // Non-existent column writer = getDataFrameWriter(classOf[NonpartitionedKeyGenerator].getName) writer.partitionBy("abc") + .mode(SaveMode.Overwrite) .save(basePath) recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*") + .load(basePath) assertTrue(recordsReadDF.filter(col("_hoodie_partition_path") =!= lit("")).count() == 0) } @@ -597,7 +613,7 @@ class TestCOWDataSource extends HoodieClientTestBase { val recordsDF = spark.createDataFrame(rdd, schema1) recordsDF.write.format("org.apache.hudi") .options(opts) - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .save(basePath) // 2. write records with schema2 add column age @@ -651,10 +667,10 @@ class TestCOWDataSource extends HoodieClientTestBase { val inputDF = spark.read.schema(schema1.toDDL).json(spark.sparkContext.parallelize(records, 2)) inputDF.write.format("org.apache.hudi") .options(opts) - .mode(SaveMode.Append) + .mode(SaveMode.Overwrite) .save(basePath) val recordsReadDF = spark.read.format("org.apache.hudi") - .load(basePath + "/*/*") + .load(basePath) val resultSchema = new StructType(recordsReadDF.schema.filter(p=> !p.name.startsWith("_hoodie")).toArray) assertEquals(resultSchema, schema1) } 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 06408bf61f06e..d4d580fe276af 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 @@ -73,9 +73,6 @@ public class DLASyncConfig implements Serializable { @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.VALIDATE_ENABLE.defaultValue(); - @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -97,7 +94,6 @@ public static DLASyncConfig copy(DLASyncConfig cfg) { newConfig.skipRTSync = cfg.skipRTSync; newConfig.useDLASyncHiveStylePartitioning = cfg.useDLASyncHiveStylePartitioning; newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; return newConfig; } @@ -110,7 +106,6 @@ public String toString() { + 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 6af01194539e9..20f94f01ef0b3 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 @@ -71,7 +71,7 @@ public class HoodieDLAClient extends AbstractSyncHoodieClient { public HoodieDLAClient(DLASyncConfig syncConfig, FileSystem fs) { super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, - syncConfig.verifyMetadataFileListing, false, fs); + false, 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 58953012b07f3..0c2abdbf571ac 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 @@ -89,9 +89,6 @@ public class HiveSyncConfig implements Serializable { @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.VALIDATE_ENABLE.defaultValue(); - @Parameter(names = {"--table-properties"}, description = "Table properties to hive table") public String tableProperties; @@ -137,7 +134,6 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.tableName = cfg.tableName; newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; newConfig.decodePartition = cfg.decodePartition; newConfig.tableProperties = cfg.tableProperties; @@ -169,7 +165,6 @@ public String toString() { + ", ignoreExceptions=" + ignoreExceptions + ", skipROSuffix=" + skipROSuffix + ", useFileListingFromMetadata=" + useFileListingFromMetadata - + ", verifyMetadataFileListing=" + verifyMetadataFileListing + ", tableProperties='" + tableProperties + '\'' + ", serdeProperties='" + serdeProperties + '\'' + ", help=" + help 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 2e9b86e967e37..265ab750d5aee 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 @@ -62,7 +62,7 @@ public class HoodieHiveClient extends AbstractSyncHoodieClient { private final HiveSyncConfig syncConfig; public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { - super(cfg.basePath, cfg.assumeDatePartitioning, cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, cfg.withOperationField, fs); + super(cfg.basePath, cfg.assumeDatePartitioning, cfg.useFileListingFromMetadata, cfg.withOperationField, fs); this.syncConfig = cfg; // Support JDBC, HiveQL and metastore based implementations for backwards compatibility. Future users should diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java index fd7dbd81b1e3b..4bafd854ae318 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java @@ -38,7 +38,6 @@ public static GlobalHiveSyncConfig copy(GlobalHiveSyncConfig cfg) { newConfig.tableName = cfg.tableName; newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; newConfig.decodePartition = cfg.decodePartition; newConfig.batchSyncNum = cfg.batchSyncNum; 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 11ff74528dbe1..cdda18da6c3bc 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 @@ -54,17 +54,15 @@ public abstract class AbstractSyncHoodieClient { private final String basePath; private final boolean assumeDatePartitioning; private final boolean useFileListingFromMetadata; - private final boolean verifyMetadataFileListing; private final boolean withOperationField; public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, boolean useFileListingFromMetadata, - boolean verifyMetadataFileListing, boolean withOperationField, FileSystem fs) { + boolean withOperationField, FileSystem fs) { this.metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); this.tableType = metaClient.getTableType(); this.basePath = basePath; this.assumeDatePartitioning = assumeDatePartitioning; this.useFileListingFromMetadata = useFileListingFromMetadata; - this.verifyMetadataFileListing = verifyMetadataFileListing; this.withOperationField = withOperationField; this.fs = fs; } @@ -156,8 +154,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); HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); - return FSUtils.getAllPartitionPaths(engineContext, basePath, useFileListingFromMetadata, verifyMetadataFileListing, - assumeDatePartitioning); + return FSUtils.getAllPartitionPaths(engineContext, basePath, useFileListingFromMetadata, assumeDatePartitioning); } else { LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); return TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().getCommitsTimeline() 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 860e0ade71552..43e58d531ed0f 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 @@ -74,15 +74,11 @@ static class Config implements Serializable { @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.VALIDATE_ENABLE.defaultValue(); } public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, final boolean shouldAssumeDatePartitioning, - final boolean useFileListingFromMetadata, - final boolean verifyMetadataFileListing) throws IOException { + final boolean useFileListingFromMetadata) throws IOException { FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration()); final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration()); final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(baseDir).build(); @@ -100,7 +96,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(context, baseDir, useFileListingFromMetadata, verifyMetadataFileListing, shouldAssumeDatePartitioning); + List partitions = FSUtils.getAllPartitionPaths(context, baseDir, useFileListingFromMetadata, shouldAssumeDatePartitioning); if (partitions.size() > 0) { LOG.info(String.format("The job needs to copy %d partitions.", partitions.size())); @@ -194,8 +190,7 @@ public static void main(String[] args) throws IOException { // Copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning, cfg.useFileListingFromMetadata, - cfg.verifyMetadataFileListing); + copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning, cfg.useFileListingFromMetadata); // 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 85f3d2d189b50..c2cfa390d08e8 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 @@ -153,7 +153,7 @@ private Option getLatestCommitTimestamp(FileSystem fs, Config cfg) { } private List getPartitions(HoodieEngineContext engineContext, Config cfg) { - return FSUtils.getAllPartitionPaths(engineContext, cfg.sourceBasePath, true, false, false); + return FSUtils.getAllPartitionPaths(engineContext, cfg.sourceBasePath, true, 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 53b2febccaa82..ac15897f5785c 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 @@ -95,8 +95,7 @@ private void setHostAddrFromSparkConf(SparkConf sparkConf) { public void run() throws IOException { JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster); HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); - List allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, cfg.basePath, - cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, true); + List allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, cfg.basePath, cfg.useFileListingFromMetadata, true); Collections.shuffle(allPartitionPaths); List selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions) .collect(Collectors.toList()); @@ -308,9 +307,6 @@ public static class Config implements Serializable { @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.VALIDATE_ENABLE.defaultValue(); - @Parameter(names = {"--help", "-h"}) public Boolean help = false; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 6313ab7b012bd..58b665cc83780 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -24,6 +24,8 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -739,6 +741,8 @@ private void testUpsertsContinuousModeWithMultipleWriters(HoodieTableType tableT TypedProperties props = prepareMultiWriterProps(PROPS_FILENAME_TEST_MULTI_WRITER); props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass"); props.setProperty("hoodie.write.lock.filesystem.path", tableBasePath); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY,"3"); + props.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY,"5000"); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_MULTI_WRITER); // Keep it higher than batch-size to test continuous mode int totalRecords = 3000; @@ -992,6 +996,7 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "true", "2", "", "")); cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add(HoodieMetadataConfig.ENABLE.key() + "=false"); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); deltaStreamerTestRunner(ds, cfg, (r) -> { TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs); @@ -1043,6 +1048,7 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws configs.add(String.format("%s=%s", HoodieCompactionConfig.MIN_COMMITS_TO_KEEP.key(), "2")); configs.add(String.format("%s=%s", HoodieCompactionConfig.MAX_COMMITS_TO_KEEP.key(), "3")); configs.add(String.format("%s=%s", HoodieCompactionConfig.ASYNC_CLEAN, asyncClean)); + configs.add(HoodieMetadataConfig.ENABLE.key() + "=false"); cfg.configs = configs; cfg.continuousMode = false; ds = new HoodieDeltaStreamer(cfg, jsc); 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 15f702a8d4034..f192ede73a159 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 @@ -71,8 +71,7 @@ public void testEmptySnapshotCopy() throws IOException { // Do the snapshot HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); copier.snapshot(jsc(), basePath, outputPath, true, - HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue()); + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS); // Nothing changed; we just bail out assertEquals(fs.listStatus(new Path(basePath)).length, 1); @@ -125,8 +124,7 @@ public void testSnapshotCopy() throws Exception { // Do a snapshot copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue()); + copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS); // Check results assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName()))); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java index c977b79cb2c25..541da0a554fa4 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java @@ -203,6 +203,8 @@ public void testExportDatasetWithNoCommit() throws IOException { public void testExportDatasetWithNoPartition() throws IOException { // delete all source data lfs.delete(new Path(sourcePath + "/" + PARTITION_PATH), true); + // delete hudi metadata table too. + lfs.delete(new Path(cfg.sourceBasePath + "/" + ".hoodie/metadata"), true); // export final Throwable thrown = assertThrows(HoodieSnapshotExporterException.class, () -> {