diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java index 26b10c1c1bfe5..8f2abccc9adfc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java @@ -193,14 +193,15 @@ public HoodieHeartbeatClient getHeartbeatClient() { * @param metadata Current committing instant's metadata * @param pendingInflightAndRequestedInstants Pending instants on the timeline * - * @see {@link BaseHoodieWriteClient#preCommit} - * @see {@link BaseHoodieTableServiceClient#preCommit} + * @see BaseHoodieWriteClient#preCommit + * @see BaseHoodieTableServiceClient#preCommit */ protected void resolveWriteConflict(HoodieTable table, HoodieCommitMetadata metadata, Set pendingInflightAndRequestedInstants) { Timer.Context conflictResolutionTimer = metrics.getConflictResolutionCtx(); try { + // Because HoodieTable is newly initialized, no need to reload active timeline here TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), - Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner(), false, pendingInflightAndRequestedInstants); + Option.of(metadata), config, pendingInflightAndRequestedInstants); metrics.emitConflictResolutionSuccessful(); } catch (HoodieWriteConflictException e) { metrics.emitConflictResolutionFailed(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 5eb2a8f9ff342..5d4a6813455b4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -393,7 +393,7 @@ public Option scheduleTableService(String instantTime, Option inflightInstant = Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, tableServiceType.getAction(), instantTime)); try { - this.txnManager.beginTransaction(inflightInstant, Option.empty()); + this.txnManager.beginTransaction(inflightInstant); LOG.info("Scheduling table service " + tableServiceType); return scheduleTableServiceInternal(instantTime, extraMetadata, tableServiceType); } finally { @@ -809,7 +809,7 @@ protected void removeInflightFilesAlreadyRolledBack(List instantsToRollb .collect(Collectors.toList()); LOG.info("Rollback completed instants {}", rollbackCompletedInstants); try { - this.txnManager.beginTransaction(Option.empty(), Option.empty()); + this.txnManager.beginTransaction(Option.empty()); rollbackCompletedInstants.forEach(instant -> { // remove pending commit files. HoodieInstant hoodieInstant = activeTimeline @@ -922,17 +922,6 @@ public void rollbackFailedBootstrap() { } } - /** - * Some writers use SparkAllowUpdateStrategy and treat replacecommit plan as revocable plan. - * In those cases, their ConflictResolutionStrategy implementation should run conflict resolution - * even for clustering operations. - * - * @return boolean - */ - protected boolean isPreCommitRequired() { - return this.config.getWriteConflictResolutionStrategy().isPreCommitRequired(); - } - private Option delegateToTableServiceManager(TableServiceType tableServiceType, HoodieTable table) { if (!config.getTableServiceManagerConfig().isEnabledAndActionSupported(ActionType.compaction)) { return Option.empty(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 48aae4ff135af..d28588fa0b26e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -128,12 +128,8 @@ public abstract class BaseHoodieWriteClient extends BaseHoodieClient private final SupportsUpgradeDowngrade upgradeDowngradeHelper; private transient WriteOperationType operationType; private transient HoodieWriteCommitCallback commitCallback; - protected transient Timer.Context writeTimer = null; - - protected Option>> lastCompletedTxnAndMetadata = Option.empty(); protected Set pendingInflightAndRequestedInstants = Collections.emptySet(); - protected BaseHoodieTableServiceClient tableServiceClient; /** @@ -227,8 +223,7 @@ public boolean commitStats(String instantTime, HoodieData writeStat extraMetadata, operationType, config.getWriteSchema(), commitActionType); HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, commitActionType, instantTime); HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config); - this.txnManager.beginTransaction(Option.of(inflightInstant), - lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); + this.txnManager.beginTransaction(Option.of(inflightInstant)); try { preCommit(inflightInstant, metadata); if (extraPreCommitFunc.isPresent()) { @@ -507,11 +502,10 @@ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instan public void preWrite(String instantTime, WriteOperationType writeOperationType, HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); - this.lastCompletedTxnAndMetadata = txnManager.isLockRequired() - ? TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient) : Option.empty(); - this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(metaClient); - this.pendingInflightAndRequestedInstants.remove(instantTime); - tableServiceClient.setPendingInflightAndRequestedInstants(this.pendingInflightAndRequestedInstants); + if (txnManager.isLockRequired() && config.getWriteConflictResolutionStrategy().isPendingInstantsBeforeWriteRequired()) { + this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, instantTime); + tableServiceClient.setPendingInflightAndRequestedInstants(this.pendingInflightAndRequestedInstants); + } tableServiceClient.startAsyncCleanerService(this); tableServiceClient.startAsyncArchiveService(this); } @@ -970,7 +964,7 @@ public void dropIndex(List partitionTypes) { HoodieTable table = createTable(config, hadoopConf); String dropInstant = HoodieActiveTimeline.createNewInstantTime(); HoodieInstant ownerInstant = new HoodieInstant(true, HoodieTimeline.INDEXING_ACTION, dropInstant); - this.txnManager.beginTransaction(Option.of(ownerInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(ownerInstant)); try { context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table: " + config.getTableName()); table.getMetadataWriter(dropInstant).ifPresent(w -> { @@ -1181,7 +1175,7 @@ protected void doInitTable(WriteOperationType operationType, HoodieTableMetaClie if (instantTime.isPresent()) { ownerInstant = Option.of(new HoodieInstant(true, CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), instantTime.get())); } - this.txnManager.beginTransaction(ownerInstant, Option.empty()); + this.txnManager.beginTransaction(ownerInstant); try { tryUpgrade(metaClient, instantTime); initMetadataTable(instantTime); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index b1a82c1ed0369..17a7d1d5234e3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -236,7 +236,7 @@ public boolean archiveIfRequired(HoodieEngineContext context, boolean acquireLoc try { if (acquireLock) { // there is no owner or instant time per se for archival. - txnManager.beginTransaction(Option.empty(), Option.empty()); + txnManager.beginTransaction(Option.empty()); } List instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); verifyLastMergeArchiveFilesIfNecessary(context); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/BucketIndexConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/BucketIndexConcurrentFileWritesConflictResolutionStrategy.java deleted file mode 100644 index a15a4cc533c82..0000000000000 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/BucketIndexConcurrentFileWritesConflictResolutionStrategy.java +++ /dev/null @@ -1,61 +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.client.transaction; - -import org.apache.hudi.index.bucket.BucketIdentifier; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashSet; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy} using bucket index. - */ -public class BucketIndexConcurrentFileWritesConflictResolutionStrategy - extends SimpleConcurrentFileWritesConflictResolutionStrategy { - private static final Logger LOG = LoggerFactory.getLogger(BucketIndexConcurrentFileWritesConflictResolutionStrategy.class); - - @Override - public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { - // TODO : UUID's can clash even for insert/insert, handle that case. - Set partitionBucketIdSetForFirstInstant = thisOperation - .getMutatedPartitionAndFileIds() - .stream() - .map(partitionAndFileId -> - BucketIdentifier.partitionBucketIdStr(partitionAndFileId.getLeft(), BucketIdentifier.bucketIdFromFileId(partitionAndFileId.getRight())) - ).collect(Collectors.toSet()); - Set partitionBucketIdSetForSecondInstant = otherOperation - .getMutatedPartitionAndFileIds() - .stream() - .map(partitionAndFileId -> - BucketIdentifier.partitionBucketIdStr(partitionAndFileId.getLeft(), BucketIdentifier.bucketIdFromFileId(partitionAndFileId.getRight())) - ).collect(Collectors.toSet()); - Set intersection = new HashSet<>(partitionBucketIdSetForFirstInstant); - intersection.retainAll(partitionBucketIdSetForSecondInstant); - if (!intersection.isEmpty()) { - LOG.info("Found conflicting writes between first operation = " + thisOperation - + ", second operation = " + otherOperation + " , intersecting bucket ids " + intersection); - return true; - } - return false; - } -} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java index 2a393bc75c707..b9d5227e2375c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java @@ -121,6 +121,7 @@ private void init(HoodieInstant instant) { if (instant.isCompleted()) { this.mutatedPartitionAndFileIds = getPartitionAndFileIdWithoutSuffixFromSpecificRecord( this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToWriteStats()); + // for replacecommit, FG before and after replace need to be considered Map> partitionToReplaceFileIds = this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToReplaceFileIds(); this.mutatedPartitionAndFileIds.addAll(CommitUtils.flattenPartitionToReplaceFileIds(partitionToReplaceFileIds)); this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getOperationType()); @@ -163,6 +164,7 @@ private void init(HoodieInstant instant) { case LOG_COMPACTION_ACTION: this.mutatedPartitionAndFileIds = CommitUtils.getPartitionAndFileIdWithoutSuffix(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats()); this.operationType = this.metadataWrapper.getCommitMetadata().getOperationType(); + // for replacecommit, FG before and after replace need to be considered if (this.operationType.equals(WriteOperationType.CLUSTER) || WriteOperationType.isOverwrite(this.operationType)) { HoodieReplaceCommitMetadata replaceCommitMetadata = (HoodieReplaceCommitMetadata) this.metadataWrapper.getCommitMetadata(); mutatedPartitionAndFileIds.addAll(CommitUtils.flattenPartitionToReplaceFileIds(replaceCommitMetadata.getPartitionToReplaceFileIds())); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java index 1deba86fe1df4..5929c9d2c0b16 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java @@ -20,13 +20,13 @@ import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIMethod; -import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieWriteConflictException; -import org.apache.hudi.table.HoodieTable; +import java.util.Set; import java.util.stream.Stream; /** @@ -40,7 +40,9 @@ public interface ConflictResolutionStrategy { * Stream of instants to check conflicts against. * @return */ - Stream getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, Option lastSuccessfulInstant); + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + Stream getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, + Option> pendingInstantsBeforeWritten); /** * Implementations of this method will determine whether a conflict exists between 2 commits. @@ -58,14 +60,17 @@ public interface ConflictResolutionStrategy { * @return */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - Option resolveConflict(HoodieTable table, - ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) throws HoodieWriteConflictException; + void resolveConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) throws HoodieWriteConflictException; + + /** + * Whether to record the pending instants before write. + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + boolean isPendingInstantsBeforeWriteRequired(); /** - * Write clients uses their preCommit API to run conflict resolution. - * This method determines whether to execute preCommit for table services like clustering. - * @return boolean + * Whether the write operation needs to do conflict detection. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - boolean isPreCommitRequired(); + boolean isConflictResolveRequired(WriteOperationType operationType); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java index 7ed6d51038c08..d7a1373a75fa9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -51,7 +51,7 @@ public void beginTransaction(String newTxnOwnerInstantTime) { LOG.info("Transaction starting for " + newTxnOwnerInstantTime + " and " + filePath); lockManager.lock(); - reset(currentTxnOwnerInstant, Option.of(getInstant(newTxnOwnerInstantTime)), Option.empty()); + reset(currentTxnOwnerInstant, Option.of(getInstant(newTxnOwnerInstantTime))); LOG.info("Transaction started for " + newTxnOwnerInstantTime + " and " + filePath); } } @@ -60,7 +60,7 @@ public void endTransaction(String currentTxnOwnerInstantTime) { if (isLockRequired) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstantTime + " for " + filePath); - if (reset(Option.of(getInstant(currentTxnOwnerInstantTime)), Option.empty(), Option.empty())) { + if (reset(Option.of(getInstant(currentTxnOwnerInstantTime)), Option.empty())) { lockManager.unlock(); LOG.info("Transaction ended with transaction owner " + currentTxnOwnerInstantTime + " for " + filePath); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/IngestionPrimaryWriterBasedConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/IngestionPrimaryWriterBasedConflictResolutionStrategy.java deleted file mode 100644 index 1c71be9f70bda..0000000000000 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/IngestionPrimaryWriterBasedConflictResolutionStrategy.java +++ /dev/null @@ -1,116 +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.client.transaction; - -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.ClusteringUtils; -import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.Option; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Comparator; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; - -/** - * This class extends the base implementation of conflict resolution strategy. - * It gives preference to ingestion writers compared to table services. - */ -public class IngestionPrimaryWriterBasedConflictResolutionStrategy - extends SimpleConcurrentFileWritesConflictResolutionStrategy { - - private static final Logger LOG = LoggerFactory.getLogger(IngestionPrimaryWriterBasedConflictResolutionStrategy.class); - - /** - * For tableservices like replacecommit and compaction commits this method also returns ingestion inflight commits. - */ - @Override - public Stream getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, - Option lastSuccessfulInstant) { - HoodieActiveTimeline activeTimeline = metaClient.reloadActiveTimeline(); - if ((REPLACE_COMMIT_ACTION.equals(currentInstant.getAction()) - && ClusteringUtils.isClusteringCommit(metaClient, currentInstant)) - || COMPACTION_ACTION.equals(currentInstant.getAction())) { - return getCandidateInstantsForTableServicesCommits(activeTimeline, currentInstant); - } else { - return getCandidateInstantsForNonTableServicesCommits(activeTimeline, currentInstant); - } - } - - private Stream getCandidateInstantsForNonTableServicesCommits(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant) { - - // To findout which instants are conflicting, we apply the following logic - // Get all the completed instants timeline only for commits that have happened - // since the last successful write based on the transition times. - // We need to check for write conflicts since they may have mutated the same files - // that are being newly created by the current write. - List completedCommitsInstants = activeTimeline - .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) - .filterCompletedInstants() - .findInstantsModifiedAfterByStateTransitionTime(currentInstant.getTimestamp()) - .getInstantsOrderedByStateTransitionTime() - .collect(Collectors.toList()); - LOG.info(String.format("Instants that may have conflict with %s are %s", currentInstant, completedCommitsInstants)); - return completedCommitsInstants.stream(); - } - - /** - * To find which instants are conflicting, we apply the following logic - * Get both completed instants and ingestion inflight commits that have happened since the last successful write. - * We need to check for write conflicts since they may have mutated the same files - * that are being newly created by the current write. - */ - private Stream getCandidateInstantsForTableServicesCommits(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant) { - // Fetch list of completed commits. - Stream completedCommitsStream = - activeTimeline - .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) - .filterCompletedInstants() - .findInstantsModifiedAfterByStateTransitionTime(currentInstant.getTimestamp()) - .getInstantsAsStream(); - - // Fetch list of ingestion inflight commits. - Stream inflightIngestionCommitsStream = - activeTimeline - .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION)) - .filterInflights() - .getInstantsAsStream(); - - // Merge and sort the instants and return. - List instantsToConsider = Stream.concat(completedCommitsStream, inflightIngestionCommitsStream) - .sorted(Comparator.comparing(o -> o.getStateTransitionTime())) - .collect(Collectors.toList()); - LOG.info(String.format("Instants that may have conflict with %s are %s", currentInstant, instantsToConsider)); - return instantsToConsider.stream(); - } - - @Override - public boolean isPreCommitRequired() { - return true; - } -} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java index ce16e14af22b4..00b3027fae5c3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -18,17 +18,18 @@ package org.apache.hudi.client.transaction; -import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieWriteConflictException; -import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.bucket.BucketIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,40 +37,106 @@ import java.util.ConcurrentModificationException; import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.model.WriteOperationType.CLUSTER; +import static org.apache.hudi.common.model.WriteOperationType.LOG_COMPACT; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps; /** - * This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy}. + * This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy} + * based on start_timestamp, and need to record the pending instants before write. */ public class SimpleConcurrentFileWritesConflictResolutionStrategy implements ConflictResolutionStrategy { private static final Logger LOG = LoggerFactory.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class); + private final HoodieWriteConfig writeConfig; + + public SimpleConcurrentFileWritesConflictResolutionStrategy(HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + } + @Override public Stream getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, - Option lastSuccessfulInstant) { - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - // To find which instants are conflicting, we apply the following logic - // 1. Get completed instants timeline only for commits that have happened since the last successful write. - // 2. Get any scheduled or completed compaction or clustering operations that have started and/or finished - // after the current instant. We need to check for write conflicts since they may have mutated the same files - // that are being newly created by the current write. - Stream completedCommitsInstantStream = activeTimeline - .getCommitsTimeline() + Option> pendingInstantsBeforeWritten) { + // Divide the process of obtaining candidate instants into two steps: + // 1) get completed instants after current instant starts + // 2) pick from instants that are still pending + return Stream.concat(getCompletedInstantsAfterCurrent(metaClient, currentInstant, pendingInstantsBeforeWritten), + pickFromCurrentPendingInstants(metaClient, currentInstant)); + } + + /** + * Get completed instants after current instant starts based on start_timestamp: + * 1) pick start_timestamp > current start_timestamp && have completed instants + * 2) pending instants that were recorded before write, and have been converted to completed + * + * @param metaClient meta client + * @param currentInstant current instant + * @param pendingInstantsBeforeWrite pending instant recorded before write + * @return selected {@link HoodieInstant} stream + */ + public Stream getCompletedInstantsAfterCurrent(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, + Option> pendingInstantsBeforeWrite) { + return metaClient.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) .filterCompletedInstants() - .findInstantsAfter(lastSuccessfulInstant.isPresent() ? lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS) + .filter(i -> compareTimestamps(i.getTimestamp(), GREATER_THAN, currentInstant.getTimestamp()) + || (pendingInstantsBeforeWrite.isPresent() && pendingInstantsBeforeWrite.get().contains(i.getTimestamp()))) .getInstantsAsStream(); + } + + /** + * Pick from instants that are still pending, need to compare the priority of the two operations. + * + * @param metaClient meta client + * @param currentInstant current instant + * @return selected {@link HoodieInstant} stream + */ + public Stream pickFromCurrentPendingInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant) { + + // Whether to pick the pending instant to candidate instants: + // +-----------------+----------------------------------------+---------------------------------------+------------+ + // | current\pending | ingestion | clustering | compaction | + // |-----------------+----------------------------------------+---------------------------------------+------------+ + // | ingestion | no | no if #isIngestionPrimaryClustering() | yes | + // |-----------------+----------------------------------------+---------------------------------------+------------+ + // | clustering | yes if #isIngestionPrimaryClustering() | no | no | + // +-----------------+----------------------------------------+---------------------------------------+------------+ + + Set actionsToPick = new HashSet<>(); + + if (REPLACE_COMMIT_ACTION.equals(currentInstant.getAction()) + && ClusteringUtils.isPendingClusteringInstant(metaClient, currentInstant)) { + if (isIngestionPrimaryClustering()) { + actionsToPick.add(COMMIT_ACTION); + actionsToPick.add(DELTA_COMMIT_ACTION); + } + } else { + if (!isIngestionPrimaryClustering()) { + actionsToPick.add(REPLACE_COMMIT_ACTION); + } + actionsToPick.add(COMPACTION_ACTION); + } - Stream compactionAndClusteringPendingTimeline = activeTimeline - .getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, COMPACTION_ACTION)) - .findInstantsAfter(currentInstant.getTimestamp()) + Set tableServices = Stream.of(REPLACE_COMMIT_ACTION, COMPACTION_ACTION).collect(Collectors.toSet()); + + return metaClient.getActiveTimeline() + .getTimelineOfActions(actionsToPick) .filterInflightsAndRequested() + // If target is table service, we can exclude the plan scheduled before current start, + // because conflict has been resolved at writing + .filter(i -> !tableServices.contains(i.getAction()) + || compareTimestamps(i.getTimestamp(), GREATER_THAN, currentInstant.getTimestamp())) .getInstantsAsStream(); - return Stream.concat(completedCommitsInstantStream, compactionAndClusteringPendingTimeline); } @Override @@ -77,6 +144,18 @@ public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperatio // TODO : UUID's can clash even for insert/insert, handle that case. Set> partitionAndFileIdsSetForFirstInstant = thisOperation.getMutatedPartitionAndFileIds(); Set> partitionAndFileIdsSetForSecondInstant = otherOperation.getMutatedPartitionAndFileIds(); + + if (HoodieIndex.IndexType.BUCKET.name().equalsIgnoreCase(writeConfig.getIndexType().name())) { + partitionAndFileIdsSetForFirstInstant = partitionAndFileIdsSetForFirstInstant + .stream() + .map(x -> Pair.of(x.getLeft(), BucketIdentifier.bucketIdStrFromFileId(x.getRight()))) + .collect(Collectors.toSet()); + partitionAndFileIdsSetForSecondInstant = partitionAndFileIdsSetForSecondInstant + .stream() + .map(x -> Pair.of(x.getLeft(), BucketIdentifier.bucketIdStrFromFileId(x.getRight()))) + .collect(Collectors.toSet()); + } + Set> intersection = new HashSet<>(partitionAndFileIdsSetForFirstInstant); intersection.retainAll(partitionAndFileIdsSetForSecondInstant); if (!intersection.isEmpty()) { @@ -88,8 +167,7 @@ public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperatio } @Override - public Option resolveConflict(HoodieTable table, - ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { + public void resolveConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { // A completed COMPACTION action eventually shows up as a COMMIT action on the timeline. // We need to ensure we handle this during conflict resolution and not treat the commit from a // compaction operation as a regular commit. Regular commits & deltacommits are candidates for conflict. @@ -99,21 +177,42 @@ public Option resolveConflict(HoodieTable table, // add that to the below check so that concurrent updates do not conflict. if (otherOperation.getOperationType() == WriteOperationType.COMPACT) { if (HoodieTimeline.compareTimestamps(otherOperation.getInstantTimestamp(), HoodieTimeline.LESSER_THAN, thisOperation.getInstantTimestamp())) { - return thisOperation.getCommitMetadataOption(); + return; } } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(thisOperation.getInstantActionType())) { // Since log compaction is a rewrite operation, it can be committed along with other delta commits. // The ordering of the commits is taken care by AbstractHoodieLogRecordReader scan method. // Conflict arises only if the log compaction commit has a lesser timestamp compared to compaction commit. - return thisOperation.getCommitMetadataOption(); + return; } // just abort the current write if conflicts are found throw new HoodieWriteConflictException(new ConcurrentModificationException("Cannot resolve conflicts for overlapping writes")); } @Override - public boolean isPreCommitRequired() { - return false; + public boolean isPendingInstantsBeforeWriteRequired() { + return true; + } + + public boolean isConflictResolveRequired(WriteOperationType operationType) { + if (operationType == null) { + return false; + } + + if (WriteOperationType.isDataChange(operationType)) { + return true; + } else if (operationType.equals(CLUSTER)) { + return isIngestionPrimaryClustering(); + } else { + return operationType.equals(LOG_COMPACT); + } } + /** + * Whether ingestion takes precedence over clustering + */ + public boolean isIngestionPrimaryClustering() { + return writeConfig.getClusteringUpdatesStrategyClass() + .equals("org.apache.hudi.client.clustering.update.strategy.SparkAllowUpdateStrategy"); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/StateTransitionTimeBasedConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/StateTransitionTimeBasedConflictResolutionStrategy.java new file mode 100644 index 0000000000000..354dc9b7a92a1 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/StateTransitionTimeBasedConflictResolutionStrategy.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; + +import java.util.Set; +import java.util.stream.Stream; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; + +/** + * ConflictResolutionStrategy base on StateTransitionTime, it uses StateTransitionTime (end_timestamp) to get completed + * instants after current instant, and no need to record the pending instants before write. + */ +public class StateTransitionTimeBasedConflictResolutionStrategy + extends SimpleConcurrentFileWritesConflictResolutionStrategy { + + public StateTransitionTimeBasedConflictResolutionStrategy(HoodieWriteConfig writeConfig) { + super(writeConfig); + } + + /** + * Get completed instants after current instant starts based on StateTransitionTime (end_timestamp): + * 1) pick end_timestamp > current start_timestamp && have completed instants + * + * @param metaClient meta client + * @param currentInstant current instant + * @param pendingInstantsBeforeWrite pending instant recorded before write (no use) + * @return selected {@link HoodieInstant} stream + */ + @Override + public Stream getCompletedInstantsAfterCurrent(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, + Option> pendingInstantsBeforeWrite) { + return metaClient.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) + .filterCompletedInstants() + .findInstantsModifiedAfterByStateTransitionTime(currentInstant.getTimestamp()) + .getInstantsAsStream(); + } + + @Override + public boolean isPendingInstantsBeforeWriteRequired() { + return false; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java index b3e9abc7a3a13..efaeadfbe5e04 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java @@ -39,7 +39,6 @@ public class TransactionManager implements Serializable { protected final LockManager lockManager; protected final boolean isLockRequired; protected Option currentTxnOwnerInstant = Option.empty(); - private Option lastCompletedTxnOwnerInstant = Option.empty(); public TransactionManager(HoodieWriteConfig config, FileSystem fs) { this(new LockManager(config, fs), config.isLockRequired()); @@ -50,22 +49,19 @@ protected TransactionManager(LockManager lockManager, boolean isLockRequired) { this.isLockRequired = isLockRequired; } - public void beginTransaction(Option newTxnOwnerInstant, - Option lastCompletedTxnOwnerInstant) { + public void beginTransaction(Option newTxnOwnerInstant) { if (isLockRequired) { - LOG.info("Transaction starting for " + newTxnOwnerInstant - + " with latest completed transaction instant " + lastCompletedTxnOwnerInstant); + LOG.info("Transaction starting for " + newTxnOwnerInstant); lockManager.lock(); - reset(currentTxnOwnerInstant, newTxnOwnerInstant, lastCompletedTxnOwnerInstant); - LOG.info("Transaction started for " + newTxnOwnerInstant - + " with latest completed transaction instant " + lastCompletedTxnOwnerInstant); + reset(currentTxnOwnerInstant, newTxnOwnerInstant); + LOG.info("Transaction started for " + newTxnOwnerInstant); } } public void endTransaction(Option currentTxnOwnerInstant) { if (isLockRequired) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant); - if (reset(currentTxnOwnerInstant, Option.empty(), Option.empty())) { + if (reset(currentTxnOwnerInstant, Option.empty())) { lockManager.unlock(); LOG.info("Transaction ended with transaction owner " + currentTxnOwnerInstant); } @@ -73,11 +69,9 @@ public void endTransaction(Option currentTxnOwnerInstant) { } protected synchronized boolean reset(Option callerInstant, - Option newTxnOwnerInstant, - Option lastCompletedTxnOwnerInstant) { + Option newTxnOwnerInstant) { if (!this.currentTxnOwnerInstant.isPresent() || this.currentTxnOwnerInstant.get().equals(callerInstant.get())) { this.currentTxnOwnerInstant = newTxnOwnerInstant; - this.lastCompletedTxnOwnerInstant = lastCompletedTxnOwnerInstant; return true; } return false; @@ -94,10 +88,6 @@ public LockManager getLockManager() { return lockManager; } - public Option getLastCompletedTransactionOwner() { - return lastCompletedTxnOwnerInstant; - } - public Option getCurrentTransactionOwner() { return currentTxnOwnerInstant; } 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 d162fe28a62b0..e29cc1f67d5bd 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 @@ -24,12 +24,9 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.HoodieTable; @@ -37,7 +34,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Map; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -47,91 +43,55 @@ public class TransactionUtils { private static final Logger LOG = LoggerFactory.getLogger(TransactionUtils.class); /** - * Resolve any write conflicts when committing data. + * Resolve any write conflicts when committing data, need reload active timeline before calling * * @param table * @param currentTxnOwnerInstant * @param thisCommitMetadata * @param config - * @param lastCompletedTxnOwnerInstant * @param pendingInstants - * - * @return - * @throws HoodieWriteConflictException */ - public static Option resolveWriteConflictIfAny( + public static void resolveWriteConflictIfAny( final HoodieTable table, final Option currentTxnOwnerInstant, final Option thisCommitMetadata, final HoodieWriteConfig config, - Option lastCompletedTxnOwnerInstant, - boolean reloadActiveTimeline, - Set pendingInstants) throws HoodieWriteConflictException { + final Set pendingInstants + ) throws HoodieWriteConflictException { if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { - // deal with pendingInstants - Stream completedInstantsDuringCurrentWriteOperation = getCompletedInstantsDuringCurrentWriteOperation(table.getMetaClient(), pendingInstants); ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy(); - if (reloadActiveTimeline) { - table.getMetaClient().reloadActiveTimeline(); - } - Stream instantStream = Stream.concat(resolutionStrategy.getCandidateInstants( - table.getMetaClient(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant), - completedInstantsDuringCurrentWriteOperation); - final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElse(new HoodieCommitMetadata())); + Stream instantStream = resolutionStrategy.getCandidateInstants( + table.getMetaClient(), currentTxnOwnerInstant.get(), Option.of(pendingInstants)); + + final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), + thisCommitMetadata.orElse(new HoodieCommitMetadata())); + instantStream.forEach(instant -> { try { ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient()); if (resolutionStrategy.hasConflict(thisOperation, otherOperation)) { LOG.info("Conflict encountered between current instant = " + thisOperation + " and instant = " + otherOperation + ", attempting to resolve it..."); - resolutionStrategy.resolveConflict(table, thisOperation, otherOperation); + resolutionStrategy.resolveConflict(thisOperation, otherOperation); } } catch (IOException io) { throw new HoodieWriteConflictException("Unable to resolve conflict, if present", io); } }); LOG.info("Successfully resolved conflicts, if any"); - - return thisOperation.getCommitMetadataOption(); - } - return thisCommitMetadata; - } - - /** - * Get the last completed transaction hoodie instant and {@link HoodieCommitMetadata#getExtraMetadata()}. - * - * @param metaClient - * @return - */ - public static Option>> getLastCompletedTxnInstantAndMetadata( - HoodieTableMetaClient metaClient) { - Option hoodieInstantOption = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().lastInstant(); - return getHoodieInstantAndMetaDataPair(metaClient, hoodieInstantOption); - } - - private static Option>> getHoodieInstantAndMetaDataPair(HoodieTableMetaClient metaClient, Option hoodieInstantOption) { - try { - if (hoodieInstantOption.isPresent()) { - HoodieCommitMetadata commitMetadata = TimelineUtils.getCommitMetadata(hoodieInstantOption.get(), metaClient.getActiveTimeline()); - return Option.of(Pair.of(hoodieInstantOption.get(), commitMetadata.getExtraMetadata())); - } else { - return Option.empty(); - } - } catch (IOException io) { - throw new HoodieIOException("Unable to read metadata for instant " + hoodieInstantOption.get(), io); } } /** - * Get InflightAndRequest instants. + * Get InflightAndRequest instants and without current * - * @param metaClient - * @return + * @param metaClient meta client + * @param currentTimestamp current instant's timestamp + * @return set of picked instants' timestamp */ - public static Set getInflightAndRequestedInstants(HoodieTableMetaClient metaClient) { + public static Set getInflightAndRequestedInstantsWithoutCurrent(HoodieTableMetaClient metaClient, String currentTimestamp) { // collect InflightAndRequest instants for deltaCommit/commit/compaction/clustering Set timelineActions = CollectionUtils .createImmutableSet(HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMMIT_ACTION); @@ -139,6 +99,7 @@ public static Set getInflightAndRequestedInstants(HoodieTableMetaClient .getActiveTimeline() .getTimelineOfActions(timelineActions) .filterInflightsAndRequested() + .filter(i -> !i.getTimestamp().equals(currentTimestamp)) .getInstantsAsStream() .map(HoodieInstant::getTimestamp) .collect(Collectors.toSet()); @@ -149,7 +110,6 @@ public static Stream getCompletedInstantsDuringCurrentWriteOperat // some pending instants maybe finished during current write operation, // we should check the conflict of those pending operation return metaClient - .reloadActiveTimeline() .getCommitsTimeline() .filterCompletedInstants() .getInstantsAsStream() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java index 1d5b09629e4c5..0dd63b4943526 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java @@ -17,8 +17,6 @@ package org.apache.hudi.config; -import org.apache.hudi.client.transaction.BucketIndexConcurrentFileWritesConflictResolutionStrategy; -import org.apache.hudi.client.transaction.ConflictResolutionStrategy; import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; import org.apache.hudi.common.config.ConfigClassProperty; @@ -27,7 +25,6 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.Option; -import org.apache.hudi.index.HoodieIndex; import java.io.File; import java.io.FileReader; @@ -205,13 +202,6 @@ public class HoodieLockConfig extends HoodieConfig { public static final ConfigProperty WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME = ConfigProperty .key(LOCK_PREFIX + "conflict.resolution.strategy") .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) - .withInferFunction(hoodieConfig -> { - if (HoodieIndex.IndexType.BUCKET.name().equalsIgnoreCase(hoodieConfig.getStringOrDefault(HoodieIndexConfig.INDEX_TYPE, null))) { - return Option.of(BucketIndexConcurrentFileWritesConflictResolutionStrategy.class.getName()); - } else { - return Option.of(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()); - } - }) .markAdvanced() .sinceVersion("0.8.0") .withDocumentation("Lock provider class name, this should be subclass of " @@ -334,8 +324,8 @@ public HoodieLockConfig.Builder withLockWaitTimeInMillis(Long waitTimeInMillis) return this; } - public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutionStrategy conflictResolutionStrategy) { - lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME, conflictResolutionStrategy.getClass().getName()); + public HoodieLockConfig.Builder withConflictResolutionStrategy(String conflictResolutionStrategy) { + lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME, conflictResolutionStrategy); return this; } 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 5f594e553af0f..be0a0d03e9297 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 @@ -2421,7 +2421,8 @@ public String getLockHiveTableName() { } public ConflictResolutionStrategy getWriteConflictResolutionStrategy() { - return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME)); + return (ConflictResolutionStrategy) ReflectionUtils.loadClass( + getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME), this); } public Long getAsyncConflictDetectorInitialDelayMs() { 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 c04f1ba8f2147..56696c84afee5 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 @@ -216,7 +216,7 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan cleanStats ); if (!skipLocking) { - this.txnManager.beginTransaction(Option.of(inflightInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(inflightInstant)); } writeTableMetadata(metadata, inflightInstant.getTimestamp()); table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant, 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 55d8e4e47af54..4d5c1c7633d57 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 @@ -63,7 +63,6 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -81,8 +80,7 @@ public abstract class BaseCommitActionExecutor protected final Option> extraMetadata; protected final WriteOperationType operationType; protected final TaskContextSupplier taskContextSupplier; - protected final Option txnManagerOption; - protected final Option>> lastCompletedTxn; + protected final TransactionManager txnManager; protected final Set pendingInflightAndRequestedInstants; public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, @@ -92,17 +90,20 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c this.operationType = operationType; this.extraMetadata = extraMetadata; this.taskContextSupplier = context.getTaskContextSupplier(); - // TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link BaseHoodieWriteClient}. - this.txnManagerOption = config.shouldAutoCommit() ? Option.of(new TransactionManager(config, table.getMetaClient().getFs())) : Option.empty(); - if (this.txnManagerOption.isPresent() && this.txnManagerOption.get().isLockRequired()) { - // these txn metadata are only needed for auto commit when optimistic concurrent control is also enabled - this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); - this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(table.getMetaClient()); - this.pendingInflightAndRequestedInstants.remove(instantTime); + // TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager + // is held in {@link BaseHoodieWriteClient}, https://issues.apache.org/jira/browse/HUDI-6566 + if (config.shouldAutoCommit()) { + this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); + if (txnManager.isLockRequired() && config.getWriteConflictResolutionStrategy().isPendingInstantsBeforeWriteRequired()) { + this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(table.getMetaClient(), instantTime); + } else { + this.pendingInflightAndRequestedInstants = null; + } } else { - this.lastCompletedTxn = Option.empty(); - this.pendingInflightAndRequestedInstants = Collections.emptySet(); + this.txnManager = null; + this.pendingInflightAndRequestedInstants = null; } + if (!table.getStorageLayout().writeOperationSupported(operationType)) { throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName() + " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName()); @@ -189,15 +190,16 @@ protected void commitOnAutoCommit(HoodieWriteMetadata result) { protected void autoCommit(Option> extraMetadata, HoodieWriteMetadata result) { final Option inflightInstant = Option.of(new HoodieInstant(State.INFLIGHT, getCommitActionType(), instantTime)); - ValidationUtils.checkState(this.txnManagerOption.isPresent(), "The transaction manager has not been initialized"); - TransactionManager txnManager = this.txnManagerOption.get(); - txnManager.beginTransaction(inflightInstant, - lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); + ValidationUtils.checkState(this.txnManager != null, "The transaction manager has not been initialized"); + txnManager.beginTransaction(inflightInstant); try { setCommitMetadata(result); - // reload active timeline so as to get all updates after current transaction have started. hence setting last arg to true. - TransactionUtils.resolveWriteConflictIfAny(table, txnManager.getCurrentTransactionOwner(), - result.getCommitMetadata(), config, txnManager.getLastCompletedTransactionOwner(), true, pendingInflightAndRequestedInstants); + if (result.getCommitMetadata().isPresent() && config.getWriteConflictResolutionStrategy() + .isConflictResolveRequired(result.getCommitMetadata().get().getOperationType())) { + table.getMetaClient().reloadActiveTimeline(); + TransactionUtils.resolveWriteConflictIfAny(table, txnManager.getCurrentTransactionOwner(), + result.getCommitMetadata(), config, pendingInflightAndRequestedInstants); + } commit(extraMetadata, result); } finally { txnManager.endTransaction(inflightInstant); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 93e8a8f859f38..cb14d32d901f8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -251,7 +251,7 @@ private void updateTableConfigAndTimeline(HoodieInstant indexInstant, HoodieIndexCommitMetadata indexCommitMetadata) throws IOException { try { // update the table config and timeline in a lock as there could be another indexer running - txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); + txnManager.beginTransaction(Option.of(indexInstant)); updateMetadataPartitionsTableConfig(table.getMetaClient(), finalIndexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); table.getActiveTimeline().saveAsComplete( @@ -362,7 +362,7 @@ public void run() { } try { // we need take a lock here as inflight writer could also try to update the timeline - txnManager.beginTransaction(Option.of(instant), Option.empty()); + txnManager.beginTransaction(Option.of(instant)); LOG.info("Updating metadata table for instant: " + instant); switch (instant.getAction()) { // TODO: see if this can be moved to metadata writer itself diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index c8557cbbc4ccc..3c4898999fa3f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -95,7 +95,7 @@ public Option execute() { .filter(p -> requestedPartitions.contains(p.getPartitionPath())).collect(Collectors.toList()); final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); try { - this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(indexInstant)); // get last completed instant Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); if (indexUptoInstant.isPresent()) { 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 e5c7aa40385a6..ec48234712701 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 @@ -151,7 +151,7 @@ private HoodieRestoreMetadata finishRestore(Map additionalProps() { + return Stream.of( + Arguments.of(createProperties(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName())), + Arguments.of(createProperties(StateTransitionTimeBasedConflictResolutionStrategy.class.getName())) + ); + } + + public static Properties createProperties(String conflictResolutionStrategyClassName) { + Properties properties = new Properties(); + properties.setProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME.key(), conflictResolutionStrategyClassName); + properties.setProperty(HoodieIndexConfig.INDEX_TYPE.key(), HoodieIndex.IndexType.BUCKET.name()); + return properties; + } + + @ParameterizedTest + @MethodSource("additionalProps") + public void testNoConcurrentWrites(Properties props) throws Exception { String newInstantTime = HoodieTestTable.makeNewCommitTime(); createCommit(newInstantTime); - // consider commits before this are all successful - - Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); - Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()); Assertions.assertEquals(0, candidateInstants.count()); } - @Test - public void testConcurrentWrites() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWrites(Properties props) throws Exception { String newInstantTime = HoodieTestTable.makeNewCommitTime(); createCommit(newInstantTime); - // consider commits before this are all successful // writer 1 createInflightCommit(HoodieTestTable.makeNewCommitTime(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); // writer 2 createInflightCommit(HoodieTestTable.makeNewCommitTime(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); - Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); - Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()); Assertions.assertEquals(0, candidateInstants.count()); } - @Test - public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulCommit(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime()); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); @@ -93,10 +116,10 @@ public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Except createCommit(newInstantTime); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with writer 2 Assertions.assertEquals(1, candidateInstants.size()); @@ -104,19 +127,17 @@ public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Except ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, writer 1 and writer 2 should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected } } - @Test - public void testConcurrentWritesWithDifferentPartition() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithDifferentPartition(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime()); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH); @@ -125,10 +146,10 @@ public void testConcurrentWritesWithDifferentPartition() throws Exception { createCommit(newInstantTime); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // there should be 1 candidate instant @@ -141,7 +162,7 @@ public void testConcurrentWritesWithDifferentPartition() throws Exception { private void createCommit(String instantTime) throws Exception { String fileId1 = "00000001-file-" + instantTime + "-1"; - String fileId2 = "00000002-file-" + instantTime + "-2"; + String fileId2 = "00000002-file-" + instantTime + "-2"; HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); commitMetadata.addMetadata("test", "test"); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyWithTableService.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyWithTableService.java new file mode 100644 index 0000000000000..6d009c974ee8a --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyWithTableService.java @@ -0,0 +1,451 @@ +/* + * 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.transaction; + +import org.apache.hudi.client.utils.TransactionUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +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.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.exception.HoodieWriteConflictException; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCommit; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCommitMetadata; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCompaction; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCompactionRequested; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCompleteCommit; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createInflightCommit; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplace; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplaceInflight; +import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplaceRequested; + +public class TestConflictResolutionStrategyWithTableService extends HoodieCommonTestHarness { + @BeforeEach + public void init() throws IOException { + initMetaClient(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanMetaClient(); + } + + private static final String SPARK_ALLOW_UPDATE_STRATEGY = "org.apache.hudi.client.clustering.update.strategy.SparkAllowUpdateStrategy"; + private static final String SPARK_REJECT_UPDATE_STRATEGY = "org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy"; + + private static Stream additionalProps() { + return Stream.of( + Arguments.of(createProperties(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName(), SPARK_ALLOW_UPDATE_STRATEGY)), + Arguments.of(createProperties(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName(), SPARK_REJECT_UPDATE_STRATEGY)), + Arguments.of(createProperties(StateTransitionTimeBasedConflictResolutionStrategy.class.getName(), SPARK_ALLOW_UPDATE_STRATEGY)), + Arguments.of(createProperties(StateTransitionTimeBasedConflictResolutionStrategy.class.getName(), SPARK_REJECT_UPDATE_STRATEGY)) + ); + } + + public static Properties createProperties(String conflictResolutionStrategyClassName, String updatesStrategy) { + Properties properties = new Properties(); + properties.setProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME.key(), conflictResolutionStrategyClassName); + properties.setProperty(HoodieClusteringConfig.UPDATES_STRATEGY.key(), updatesStrategy); + return properties; + } + + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingScheduledCompaction(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + // compaction 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCompactionRequested(newInstantTime, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( + Collectors.toList()); + // Because scheduled compaction plan has the highest priority, writer 1 have + // a conflict with scheduled compaction plan 1 + Assertions.assertEquals(1, candidateInstants.size()); + } + + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulCompaction(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + // compaction 1 gets scheduled and finishes + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + // TODO: Remove sleep stmt once the modified times issue is fixed. + // Sleep thread for at least 1sec for consecutive commits that way they do not have two commits modified times falls on the same millisecond. + Thread.sleep(1000); + createCompaction(newInstantTime, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( + Collectors.toList()); + // writer 1 conflict with compaction 1 + Assertions.assertEquals(1, candidateInstants.size()); + } + + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingCompaction(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + // compaction 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCompactionRequested(newInstantTime, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newInstantTime)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + // TODO Create method to create compactCommitMetadata + // HoodieCommitMetadata currentMetadata = createCommitMetadata(newInstantTime); + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( + Collectors.toList()); + // writer 1 not conflicts with compaction 1 + Assertions.assertEquals(0, candidateInstants.size()); + } + + /** + * This method is verifying if a conflict exists for already commit compaction commit with current running ingestion commit. + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWriteAndCompactionScheduledEarlier(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // compaction 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCompactionRequested(newInstantTime, metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( + Collectors.toList()); + // writer 1 should not conflict with an earlier scheduled compaction 1 with the same file ids + Assertions.assertEquals(0, candidateInstants.size()); + } + + /** + * This method confirms that ingestion commit when completing only looks at the completed commits. + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingScheduledCluster(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + // clustering 1 gets scheduled + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createReplaceRequested(newInstantTime, metaClient); + createReplaceInflight(newInstantTime, WriteOperationType.CLUSTER, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( + Collectors.toList()); + if (props.get(HoodieClusteringConfig.UPDATES_STRATEGY.key()).equals(SPARK_ALLOW_UPDATE_STRATEGY)) { + Assertions.assertEquals(0, candidateInstants.size()); + } else { + Assertions.assertEquals(1, candidateInstants.size()); + } + } + + /** + * This method confirms ingestion commit failing due to already present replacecommit. + * Here the replacecommit is allowed to commit. Ideally replacecommit cannot be committed when there is a ingestion inflight. + * The following case can occur, during transition phase of ingestion commit from Requested to Inflight, + * during that time replacecommit can be completed. + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulCluster(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + // TODO: Remove sleep stmt once the modified times issue is fixed. + // Sleep thread for at least 1sec for consecutive commits that way they do not have two commits modified times falls on the same millisecond. + Thread.sleep(1000); + // clustering writer starts and complete before ingestion commit. + String replaceWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createReplace(replaceWriterInstant, WriteOperationType.CLUSTER, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy + .getCandidateInstants(metaClient, currentInstant.get(), Option.empty()) + .collect(Collectors.toList()); + Assertions.assertEquals(1, candidateInstants.size()); + Assertions.assertEquals(replaceWriterInstant, candidateInstants.get(0).getTimestamp()); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulReplace(Properties props) throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, metaClient); + // TODO: Remove sleep stmt once the modified times issue is fixed. + // Sleep thread for at least 1sec for consecutive commits that way they do not have two commits modified times falls on the same millisecond. + Thread.sleep(1000); + // replace 1 gets scheduled and finished + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE, metaClient); + + Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + + metaClient.reloadActiveTimeline(); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( + Collectors.toList()); + // writer 1 conflicts with replace 1 + Assertions.assertEquals(1, candidateInstants.size()); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + /** + * Test get candidate instants, all instants are ingestion + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testGetCandidateInstantsWithIngestion(Properties props) throws Exception { + // The whole process requires two reload active timeline, before write, pre commit + // + // start end + // current <----------------> + // instant1 <------> + // instant2 <------------> pick + // instant3 <---------------------------------> + // instant4 <---------> pick + // instant5 <----------------------> + + String instant1 = HoodieTestTable.makeNewCommitTime(); + createCommit(instant1, metaClient); + + String instant2 = HoodieTestTable.makeNewCommitTime(); + createInflightCommit(instant2, metaClient); + + String instant3 = HoodieTestTable.makeNewCommitTime(); + createInflightCommit(instant3, metaClient); + + String current = HoodieTestTable.makeNewCommitTime(); + // before write: reload active timeline, collect pending instants + metaClient.reloadActiveTimeline(); + Set pendingInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, current); + createInflightCommit(current, metaClient); + + createCompleteCommit(instant2, metaClient); + + String instant4 = HoodieTestTable.makeNewCommitTime(); + createInflightCommit(instant4, metaClient); + + String instant5 = HoodieTestTable.makeNewCommitTime(); + createInflightCommit(instant5, metaClient); + + createCompleteCommit(instant4, metaClient); + + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + + // pre commit: reload active timeline, getCandidateInstants + metaClient.reloadActiveTimeline(); + HoodieInstant currentInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, current); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant, Option.of(pendingInstants)) + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + + // Since the above instants are all ingestion, current's priority is the highest + // 2 and 4 (mandatory) are picked, 3 and 5 (optional) are excluded + Assertions.assertEquals(2, candidateInstants.size()); + Assertions.assertTrue(candidateInstants.contains(instant2)); + Assertions.assertTrue(candidateInstants.contains(instant4)); + } + + /** + * Test get candidate instants, current is ingestion, pending is clustering + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testGetCandidateInstantsWithIngestionToClustering(Properties props) throws Exception { + // start end + // current (ingestion) <----------------> + // instant1 (clustering, scheduled before current start) <-------------------------------> + // instant2 (clustering, scheduled after current start) <----------------------> pick if not with SparkAllowUpdateStrategy + + String instant1 = HoodieTestTable.makeNewCommitTime(); + createReplaceRequested(instant1, metaClient); + + String current = HoodieTestTable.makeNewCommitTime(); + // before write: reload active timeline, collect pending instants + metaClient.reloadActiveTimeline(); + Set pendingInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, current); + createInflightCommit(current, metaClient); + + String instant2 = HoodieTestTable.makeNewCommitTime(); + createReplaceRequested(instant2, metaClient); + + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + + // pre commit: reload active timeline, getCandidateInstants + metaClient.reloadActiveTimeline(); + HoodieInstant currentInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, current); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant, Option.of(pendingInstants)) + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + + if (props.get(HoodieClusteringConfig.UPDATES_STRATEGY.key()).equals(SPARK_ALLOW_UPDATE_STRATEGY)) { + Assertions.assertEquals(0, candidateInstants.size()); + } else { + Assertions.assertEquals(1, candidateInstants.size()); + Assertions.assertTrue(candidateInstants.contains(instant2)); + } + } + + /** + * Test get candidate instants, current is clustering, pending is ingestion + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testGetCandidateInstantsWithClusteringToIngestion(Properties props) throws Exception { + // scheduled end + // current (clustering) <----------------> + // instant1 (ingestion, start before clustering scheduled) <------------------------------> pick + // instant2 (ingestion, start after clustering scheduled) <-----------------------> pick + + String instant1 = HoodieTestTable.makeNewCommitTime(); + createInflightCommit(instant1, metaClient); + + String current = HoodieTestTable.makeNewCommitTime(); + // before write: reload active timeline, collect pending instants + metaClient.reloadActiveTimeline(); + Set pendingInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, current); + createReplaceRequested(current, metaClient); + + String instant2 = HoodieTestTable.makeNewCommitTime(); + createInflightCommit(instant2, metaClient); + + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + + // pre commit: reload active timeline, getCandidateInstants + metaClient.reloadActiveTimeline(); + HoodieInstant currentInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, current); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant, Option.of(pendingInstants)) + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + + if (props.get(HoodieClusteringConfig.UPDATES_STRATEGY.key()).equals(SPARK_ALLOW_UPDATE_STRATEGY)) { + Assertions.assertEquals(2, candidateInstants.size()); + Assertions.assertTrue(candidateInstants.contains(instant1)); + Assertions.assertTrue(candidateInstants.contains(instant2)); + } else { + Assertions.assertEquals(0, candidateInstants.size()); + } + } + + /** + * Test get candidate instants, current is ingestion, pending is compaction + */ + @ParameterizedTest + @MethodSource("additionalProps") + public void testGetCandidateInstantsWithIngestionToCompaction(Properties props) throws Exception { + // start end + // current (ingestion) <----------------> + // instant1 (compaction, scheduled before current start) <-------------------------------> + // instant2 (compaction, scheduled after current start) <----------------------> pick + + String instant1 = HoodieTestTable.makeNewCommitTime(); + createCompactionRequested(instant1, metaClient); + + String current = HoodieTestTable.makeNewCommitTime(); + // before write: reload active timeline, collect pending instants + metaClient.reloadActiveTimeline(); + Set pendingInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, current); + createInflightCommit(current, metaClient); + + String instant2 = HoodieTestTable.makeNewCommitTime(); + createCompactionRequested(instant2, metaClient); + + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + + // pre commit: reload active timeline, getCandidateInstants + metaClient.reloadActiveTimeline(); + HoodieInstant currentInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, current); + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant, Option.of(pendingInstants)) + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + + Assertions.assertEquals(1, candidateInstants.size()); + Assertions.assertTrue(candidateInstants.contains(instant2)); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestIngestionPrimaryWriterBasedConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestIngestionPrimaryWriterBasedConflictResolutionStrategy.java deleted file mode 100644 index 966da46690e2f..0000000000000 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestIngestionPrimaryWriterBasedConflictResolutionStrategy.java +++ /dev/null @@ -1,260 +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.client.transaction; - -import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.WriteOperationType; -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.testutils.HoodieCommonTestHarness; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieWriteConflictException; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCommit; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCommitMetadata; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCompaction; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCompactionRequested; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createInflightCommit; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplace; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplaceInflight; -import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplaceRequested; - -public class TestIngestionPrimaryWriterBasedConflictResolutionStrategy extends HoodieCommonTestHarness { - - @BeforeEach - public void init() throws IOException { - initMetaClient(); - } - - @Test - public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - // compaction 1 gets scheduled - String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - createCompactionRequested(newInstantTime, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( - Collectors.toList()); - // writer 1 does not have a conflict with scheduled compaction plan 1 - // Since, scheduled compaction plan is given lower priority compared ingestion commit. - Assertions.assertEquals(0, candidateInstants.size()); - } - - @Test - public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - // compaction 1 gets scheduled and finishes - String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - // TODO: Remove sleep stmt once the modified times issue is fixed. - // Sleep thread for atleast 1sec for consecutive commits that way they do not have two commits modified times falls on the same millisecond. - Thread.sleep(1000); - createCompaction(newInstantTime, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( - Collectors.toList()); - // writer 1 conflicts with compaction 1 - Assertions.assertEquals(1, candidateInstants.size()); - Assertions.assertEquals(newInstantTime, candidateInstants.get(0).getTimestamp()); - ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); - ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); - Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); - try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); - Assertions.fail("Cannot reach here, should have thrown a conflict"); - } catch (HoodieWriteConflictException e) { - // expected - } - } - - @Test - public void testConcurrentWritesWithInterleavingCompaction() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - // compaction 1 gets scheduled and finishes - String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - createCompactionRequested(newInstantTime, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newInstantTime)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - // TODO Create method to create compactCommitMetadata - // HoodieCommitMetadata currentMetadata = createCommitMetadata(newInstantTime); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( - Collectors.toList()); - // writer 1 conflicts with compaction 1 - Assertions.assertEquals(1, candidateInstants.size()); - Assertions.assertEquals(currentWriterInstant, candidateInstants.get(0).getTimestamp()); - // TODO: Once compactCommitMetadata is created use that to verify resolveConflict method. - } - - /** - * This method is verifying if a conflict exists for already commit compaction commit with current running ingestion commit. - */ - @Test - public void testConcurrentWriteAndCompactionScheduledEarlier() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - // consider commits before this are all successful - // compaction 1 gets scheduled - String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - createCompaction(newInstantTime, metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( - Collectors.toList()); - // writer 1 should not conflict with an earlier scheduled compaction 1 with the same file ids - Assertions.assertEquals(0, candidateInstants.size()); - } - - /** - * This method confirms that ingestion commit when completing only looks at the completed commits. - */ - @Test - public void testConcurrentWritesWithInterleavingScheduledCluster() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - // clustering 1 gets scheduled - String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - createReplaceRequested(newInstantTime, metaClient); - createReplaceInflight(newInstantTime, WriteOperationType.CLUSTER, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( - Collectors.toList()); - // Since we give preference to ingestion over clustering, there wont be a conflict with replacecommit. - Assertions.assertEquals(0, candidateInstants.size()); - } - - /** - * This method confirms ingestion commit failing due to already present replacecommit. - * Here the replacecommit is allowed to commit. Ideally replacecommit cannot be committed when there is a ingestion inflight. - * The following case can occur, during transition phase of ingestion commit from Requested to Inflight, - * during that time replacecommit can be completed. - */ - @Test - public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - // TODO: Remove sleep stmt once the modified times issue is fixed. - // Sleep thread for atleast 1sec for consecutive commits that way they do not have two commits modified times falls on the same millisecond. - Thread.sleep(1000); - // clustering writer starts and complete before ingestion commit. - String replaceWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createReplace(replaceWriterInstant, WriteOperationType.CLUSTER, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - metaClient.reloadActiveTimeline(); - List candidateInstants = strategy - .getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant) - .collect(Collectors.toList()); - Assertions.assertEquals(1, candidateInstants.size()); - Assertions.assertEquals(replaceWriterInstant, candidateInstants.get(0).getTimestamp()); - HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); - ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); - ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); - Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); - try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); - Assertions.fail("Cannot reach here, should have thrown a conflict"); - } catch (HoodieWriteConflictException e) { - // expected - } - } - - @Test - public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Exception { - createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); - // writer 1 starts - String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); - createInflightCommit(currentWriterInstant, metaClient); - // TODO: Remove sleep stmt once the modified times issue is fixed. - // Sleep thread for atleast 1sec for consecutive commits that way they do not have two commits modified times falls on the same millisecond. - Thread.sleep(1000); - // replace 1 gets scheduled and finished - String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); - createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE, metaClient); - - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - IngestionPrimaryWriterBasedConflictResolutionStrategy strategy = new IngestionPrimaryWriterBasedConflictResolutionStrategy(); - HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( - Collectors.toList()); - // writer 1 conflicts with replace 1 - Assertions.assertEquals(1, candidateInstants.size()); - ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); - ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); - Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); - try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); - Assertions.fail("Cannot reach here, should have thrown a conflict"); - } catch (HoodieWriteConflictException e) { - // expected - } - } - -} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java index cc18433b96336..47d2f92af688a 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -28,14 +28,19 @@ import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.exception.HoodieWriteConflictException; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; -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 java.io.IOException; import java.util.List; +import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -62,44 +67,59 @@ public void init() throws IOException { initMetaClient(); } - @Test - public void testNoConcurrentWrites() throws Exception { + @AfterEach + public void tearDown() throws IOException { + cleanMetaClient(); + } + + private static Stream additionalProps() { + return Stream.of( + Arguments.of(createProperties(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName())), + Arguments.of(createProperties(StateTransitionTimeBasedConflictResolutionStrategy.class.getName())) + ); + } + + public static Properties createProperties(String conflictResolutionStrategyClassName) { + Properties properties = new Properties(); + properties.setProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME.key(), conflictResolutionStrategyClassName); + return properties; + } + + @ParameterizedTest + @MethodSource("additionalProps") + public void testNoConcurrentWrites(Properties props) throws Exception { String newInstantTime = HoodieTestTable.makeNewCommitTime(); createCommit(newInstantTime, metaClient); - // consider commits before this are all successful - - Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); - Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); - Assertions.assertTrue(candidateInstants.count() == 0); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()); + Assertions.assertEquals(0, candidateInstants.count()); } - @Test - public void testConcurrentWrites() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWrites(Properties props) throws Exception { String newInstantTime = HoodieTestTable.makeNewCommitTime(); createCommit(newInstantTime, metaClient); - // consider commits before this are all successful // writer 1 createInflightCommit(HoodieTestTable.makeNewCommitTime(), metaClient); // writer 2 createInflightCommit(HoodieTestTable.makeNewCommitTime(), metaClient); - Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); - Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); - Assertions.assertTrue(candidateInstants.count() == 0); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); + metaClient.reloadActiveTimeline(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()); + Assertions.assertEquals(0, candidateInstants.count()); } - @Test - public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulCommit(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -108,30 +128,28 @@ public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Except createCommit(newInstantTime, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with writer 2 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, writer 1 and writer 2 should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected } } - @Test - public void testConcurrentWritesWithReplaceInflightCommit() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithReplaceInflightCommit(Properties props) throws Exception { createReplaceInflight(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - Option lastSuccessfulInstant = Option.empty(); - // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -141,32 +159,29 @@ public void testConcurrentWritesWithReplaceInflightCommit() throws Exception { String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); createReplaceInflight(newInstantTime, metaClient); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with writer 2 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, writer 1 and writer 2 should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected } } - @Test - public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingScheduledCompaction(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -175,30 +190,28 @@ public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exc createCompactionRequested(newInstantTime, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with scheduled compaction plan 1 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected } } - @Test - public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulCompaction(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -207,18 +220,18 @@ public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Ex createCompaction(newInstantTime, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with compaction 1 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected @@ -228,35 +241,30 @@ public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Ex /** * This method is verifying if a conflict exists for already commit compaction commit with current running ingestion commit. */ - @Test - public void testConcurrentWriteAndCompactionScheduledEarlier() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWriteAndCompactionScheduledEarlier(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); // compaction 1 gets scheduled String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); createCompaction(newInstantTime, metaClient); - // consider commits before this are all successful - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); - HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 should not conflict with an earlier scheduled compaction 1 with the same file ids - Assertions.assertTrue(candidateInstants.size() == 0); + Assertions.assertEquals(0, candidateInstants.size()); } - @Test - public void testConcurrentWritesWithInterleavingScheduledCluster() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingScheduledCluster(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -265,30 +273,28 @@ public void testConcurrentWritesWithInterleavingScheduledCluster() throws Except createReplaceRequested(newInstantTime, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with scheduled compaction plan 1 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected } } - @Test - public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulCluster(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -297,30 +303,28 @@ public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Excep createReplace(newInstantTime, WriteOperationType.CLUSTER, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with cluster 1 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected } } - @Test - public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Exception { + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithInterleavingSuccessfulReplace(Properties props) throws Exception { createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); - HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // writer 1 starts String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -329,18 +333,18 @@ public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Excep createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE, metaClient); Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); - List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( + List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), Option.empty()).collect( Collectors.toList()); // writer 1 conflicts with replace 1 - Assertions.assertTrue(candidateInstants.size() == 1); + Assertions.assertEquals(1, candidateInstants.size()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); Assertions.fail("Cannot reach here, should have thrown a conflict"); } catch (HoodieWriteConflictException e) { // expected @@ -348,9 +352,10 @@ public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Excep } // try to simulate HUDI-3355 - @Test - public void testConcurrentWritesWithPendingInstants() throws Exception { - // step1: create a pending replace/commit/compact instant: C1,C11,C12 + @ParameterizedTest + @MethodSource("additionalProps") + public void testConcurrentWritesWithPendingInstants(Properties props) throws Exception { + // step1: create a pending replace/compact/commit instant: C1,C11,C12 String newInstantTimeC1 = HoodieActiveTimeline.createNewInstantTime(); createPendingReplace(newInstantTimeC1, WriteOperationType.CLUSTER, metaClient); @@ -362,8 +367,6 @@ public void testConcurrentWritesWithPendingInstants() throws Exception { // step2: create a complete commit which has no conflict with C1,C11,C12, named it as C2 createCommit(HoodieActiveTimeline.createNewInstantTime(), metaClient); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - // consider commits before this are all successful - Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); // step3: write 1 starts, which has conflict with C1,C11,C12, named it as C3 String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); @@ -372,8 +375,7 @@ public void testConcurrentWritesWithPendingInstants() throws Exception { createRequestedCommit(commitC4, metaClient); // get PendingCommit during write 1 operation metaClient.reloadActiveTimeline(); - Set pendingInstant = TransactionUtils.getInflightAndRequestedInstants(metaClient); - pendingInstant.remove(currentWriterInstant); + Set pendingInstant = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, currentWriterInstant); // step5: finished pending cluster/compaction/commit operation createCompleteReplace(newInstantTimeC1, WriteOperationType.CLUSTER, metaClient); createCompleteCompaction(newCompactionInstantTimeC11, metaClient); @@ -382,14 +384,15 @@ public void testConcurrentWritesWithPendingInstants() throws Exception { // step6: do check Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); - SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); + ConflictResolutionStrategy strategy = TestConflictResolutionStrategyUtil.getConflictResolutionStrategy(metaClient, props); // make sure c3 has conflict with C1,C11,C12,C4; HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, "file-2"); + metaClient.reloadActiveTimeline(); timeline.reload(); List completedInstantsDuringCurrentWriteOperation = TransactionUtils - .getCompletedInstantsDuringCurrentWriteOperation(metaClient, pendingInstant).collect(Collectors.toList()); + .getCompletedInstantsDuringCurrentWriteOperation(metaClient, pendingInstant).collect(Collectors.toList()); // C1,C11,C12,C4 should be included - Assertions.assertTrue(completedInstantsDuringCurrentWriteOperation.size() == 4); + Assertions.assertEquals(4, completedInstantsDuringCurrentWriteOperation.size()); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); // check C3 has conflict with C1,C11,C12,C4 @@ -397,7 +400,11 @@ public void testConcurrentWritesWithPendingInstants() throws Exception { ConcurrentOperation thatCommitOperation = new ConcurrentOperation(instant, metaClient); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); try { - strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + strategy.resolveConflict(thisCommitOperation, thatCommitOperation); + // C11 is COMPACTION, and C3 is created after C11, so the resolve can pass + if (!instant.getTimestamp().equals(newCompactionInstantTimeC11)) { + Assertions.fail("Cannot reach here, should have thrown a conflict"); + } } catch (HoodieWriteConflictException e) { // expected } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java index 4222754a19499..ef6d12eb760f9 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java @@ -74,23 +74,20 @@ private HoodieWriteConfig getWriteConfig() { @Test public void testSingleWriterTransaction() { - Option lastCompletedInstant = getInstant("0000001"); Option newTxnOwnerInstant = getInstant("0000002"); - transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.beginTransaction(newTxnOwnerInstant); transactionManager.endTransaction(newTxnOwnerInstant); } @Test public void testSingleWriterNestedTransaction() { - Option lastCompletedInstant = getInstant("0000001"); Option newTxnOwnerInstant = getInstant("0000002"); - transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.beginTransaction(newTxnOwnerInstant); - Option lastCompletedInstant1 = getInstant("0000003"); Option newTxnOwnerInstant1 = getInstant("0000004"); assertThrows(HoodieLockException.class, () -> { - transactionManager.beginTransaction(newTxnOwnerInstant1, lastCompletedInstant1); + transactionManager.beginTransaction(newTxnOwnerInstant1); }); transactionManager.endTransaction(newTxnOwnerInstant); @@ -107,7 +104,6 @@ public void testMultiWriterTransactions() { final AtomicBoolean writer1Completed = new AtomicBoolean(false); final AtomicBoolean writer2Completed = new AtomicBoolean(false); - Option lastCompletedInstant1 = getInstant("0000001"); Option newTxnOwnerInstant1 = getInstant("0000002"); Option lastCompletedInstant2 = getInstant("0000003"); Option newTxnOwnerInstant2 = getInstant("0000004"); @@ -116,7 +112,7 @@ public void testMultiWriterTransactions() { // to join the sync up point. Thread writer1 = new Thread(() -> { assertDoesNotThrow(() -> { - transactionManager.beginTransaction(newTxnOwnerInstant1, lastCompletedInstant1); + transactionManager.beginTransaction(newTxnOwnerInstant1); }); latch.countDown(); try { @@ -145,7 +141,7 @@ public void testMultiWriterTransactions() { // } assertDoesNotThrow(() -> { - transactionManager.beginTransaction(newTxnOwnerInstant2, lastCompletedInstant2); + transactionManager.beginTransaction(newTxnOwnerInstant2); }); assertDoesNotThrow(() -> { transactionManager.endTransaction(newTxnOwnerInstant2); @@ -173,9 +169,8 @@ public void testMultiWriterTransactions() { @Test public void testEndTransactionByDiffOwner() throws InterruptedException { // 1. Begin and end by the same transaction owner - Option lastCompletedInstant = getInstant("0000001"); Option newTxnOwnerInstant = getInstant("0000002"); - transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.beginTransaction(newTxnOwnerInstant); CountDownLatch countDownLatch = new CountDownLatch(1); // Another writer thread @@ -189,60 +184,46 @@ public void testEndTransactionByDiffOwner() throws InterruptedException { countDownLatch.await(30, TimeUnit.SECONDS); // should not have reset the state within transaction manager since the owner is different. Assertions.assertTrue(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner().isPresent()); transactionManager.endTransaction(newTxnOwnerInstant); Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); } @Test public void testTransactionsWithInstantTime() { // 1. Begin and end by the same transaction owner - Option lastCompletedInstant = getInstant("0000001"); Option newTxnOwnerInstant = getInstant("0000002"); - transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.beginTransaction(newTxnOwnerInstant); Assertions.assertTrue(transactionManager.getCurrentTransactionOwner() == newTxnOwnerInstant); - Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner() == lastCompletedInstant); transactionManager.endTransaction(newTxnOwnerInstant); Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); // 2. Begin transaction with a new txn owner, but end transaction with wrong owner - lastCompletedInstant = getInstant("0000002"); newTxnOwnerInstant = getInstant("0000003"); - transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.beginTransaction(newTxnOwnerInstant); transactionManager.endTransaction(getInstant("0000004")); // Owner reset would not happen as the end txn was invoked with an incorrect current txn owner Assertions.assertTrue(transactionManager.getCurrentTransactionOwner() == newTxnOwnerInstant); - Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner() == lastCompletedInstant); transactionManager.endTransaction(newTxnOwnerInstant); // 3. But, we should be able to begin a new transaction for a new owner - lastCompletedInstant = getInstant("0000003"); newTxnOwnerInstant = getInstant("0000004"); - transactionManager.beginTransaction(newTxnOwnerInstant, lastCompletedInstant); + transactionManager.beginTransaction(newTxnOwnerInstant); Assertions.assertTrue(transactionManager.getCurrentTransactionOwner() == newTxnOwnerInstant); - Assertions.assertTrue(transactionManager.getLastCompletedTransactionOwner() == lastCompletedInstant); transactionManager.endTransaction(newTxnOwnerInstant); Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); // 4. Transactions with new instants but with same timestamps should properly reset owners - transactionManager.beginTransaction(getInstant("0000005"), Option.empty()); + transactionManager.beginTransaction(getInstant("0000005")); Assertions.assertTrue(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); transactionManager.endTransaction(getInstant("0000005")); Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); // 6. Transactions with no owners should also go through - transactionManager.beginTransaction(Option.empty(), Option.empty()); + transactionManager.beginTransaction(Option.empty()); Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); transactionManager.endTransaction(Option.empty()); Assertions.assertFalse(transactionManager.getCurrentTransactionOwner().isPresent()); - Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent()); } private Option getInstant(String timestamp) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java index 13105251acf7b..bdd0e3693352a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java @@ -92,7 +92,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable tab List writeStats = metadata.getWriteStats(); final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime); try { - this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(compactionInstant)); finalizeWrite(table, compactionCommitTime, writeStats); // commit to data table after committing to metadata table. // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a @@ -163,7 +163,7 @@ protected void completeLogCompaction(HoodieCommitMetadata metadata, List writeStats = metadata.getWriteStats(); final HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionCommitTime); try { - this.txnManager.beginTransaction(Option.of(logCompactionInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(logCompactionInstant)); preCommit(metadata); finalizeWrite(table, logCompactionCommitTime, writeStats); // commit to data table after committing to metadata table. @@ -199,13 +199,9 @@ protected void completeClustering( } try { - this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(clusteringInstant)); finalizeWrite(table, clusteringCommitTime, writeStats); - // Only in some cases conflict resolution needs to be performed. - // So, check if preCommit method that does conflict resolution needs to be triggered. - if (isPreCommitRequired()) { - preCommit(metadata); - } + preCommit(metadata); // commit to data table after committing to metadata table. // We take the lock here to ensure all writes to metadata table happens within a single lock (single writer). // Because more than one write to metadata table will result in conflicts since all of them updates the same partition. @@ -262,10 +258,12 @@ public void writeTableMetadata(HoodieTable table, String instantTime, String act @Override protected void preCommit(HoodieCommitMetadata metadata) { - // Create a Hoodie table after startTxn which encapsulated the commits and files visible. - // Important to create this after the lock to ensure the latest commits show up in the timeline without need for reload - HoodieTable table = createTable(config, hadoopConf); - resolveWriteConflict(table, metadata, this.pendingInflightAndRequestedInstants); + if (config.getWriteConflictResolutionStrategy().isConflictResolveRequired(metadata.getOperationType())) { + // Create a Hoodie table after startTxn which encapsulated the commits and files visible. + // Important to create this after the lock to ensure the latest commits show up in the timeline without need for reload + HoodieTable table = createTable(config, hadoopConf); + resolveWriteConflict(table, metadata, this.pendingInflightAndRequestedInstants); + } } /** 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 a320aad62fc06..c9c429320433e 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 @@ -304,11 +304,10 @@ public void preWrite(String instantTime, WriteOperationType writeOperationType, * should be called before the Driver starts a new transaction. */ public void preTxn(HoodieTableMetaClient metaClient) { - if (txnManager.isLockRequired()) { + if (txnManager.isLockRequired() && config.getWriteConflictResolutionStrategy().isPendingInstantsBeforeWriteRequired()) { // refresh the meta client which is reused metaClient.reloadActiveTimeline(); - this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient); - this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(metaClient); + this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstantsWithoutCurrent(metaClient, null); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java index d2fb78e08c696..b30bc68bfa54a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java @@ -142,7 +142,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable tab handleWriteErrors(writeStats, TableServiceType.COMPACT); final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime); try { - this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(compactionInstant)); finalizeWrite(table, compactionCommitTime, writeStats); // commit to data table after committing to metadata table. updateTableMetadata(table, metadata, compactionInstant, context.emptyHoodieData()); @@ -171,7 +171,7 @@ protected void completeLogCompaction(HoodieCommitMetadata metadata, handleWriteErrors(writeStats, TableServiceType.LOG_COMPACT); final HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionCommitTime); try { - this.txnManager.beginTransaction(Option.of(logCompactionInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(logCompactionInstant)); preCommit(metadata); finalizeWrite(table, logCompactionCommitTime, writeStats); // commit to data table after committing to metadata table. @@ -255,14 +255,10 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, handleWriteErrors(writeStats, TableServiceType.CLUSTER); final HoodieInstant clusteringInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime); try { - this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); + this.txnManager.beginTransaction(Option.of(clusteringInstant)); finalizeWrite(table, clusteringCommitTime, writeStats); - // Only in some cases conflict resolution needs to be performed. - // So, check if preCommit method that does conflict resolution needs to be triggered. - if (isPreCommitRequired()) { - preCommit(metadata); - } + preCommit(metadata); // Update table's metadata (table) updateTableMetadata(table, metadata, clusteringInstant, writeStatuses.orElse(context.emptyHoodieData())); @@ -323,9 +319,11 @@ private void updateTableMetadata(HoodieTable table, HoodieCommitMetadata commitM @Override protected void preCommit(HoodieCommitMetadata metadata) { - // Create a Hoodie table after startTxn which encapsulated the commits and files visible. - // Important to create this after the lock to ensure the latest commits show up in the timeline without need for reload - HoodieTable table = createTable(config, hadoopConf); - resolveWriteConflict(table, metadata, this.pendingInflightAndRequestedInstants); + if (config.getWriteConflictResolutionStrategy().isConflictResolveRequired(metadata.getOperationType())) { + // Create a Hoodie table after startTxn which encapsulated the commits and files visible. + // Important to create this after the lock to ensure the latest commits show up in the timeline without need for reload + HoodieTable table = createTable(config, hadoopConf); + resolveWriteConflict(table, metadata, this.pendingInflightAndRequestedInstants); + } } } 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 0984f9436bf47..b8ce604d1a602 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 @@ -18,8 +18,7 @@ package org.apache.hudi.client; -import org.apache.hudi.client.transaction.ConflictResolutionStrategy; -import org.apache.hudi.client.transaction.IngestionPrimaryWriterBasedConflictResolutionStrategy; +import org.apache.hudi.client.transaction.StateTransitionTimeBasedConflictResolutionStrategy; import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; @@ -141,14 +140,14 @@ public void clean() throws IOException { InProcessLockProvider.class, FileSystemBasedLockProvider.class); - private static final List CONFLICT_RESOLUTION_STRATEGY_CLASSES = Arrays.asList( - new SimpleConcurrentFileWritesConflictResolutionStrategy(), - new IngestionPrimaryWriterBasedConflictResolutionStrategy()); + private static final List CONFLICT_RESOLUTION_STRATEGY_CLASSES = Arrays.asList( + SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName(), + StateTransitionTimeBasedConflictResolutionStrategy.class.getName()); private static Iterable providerClassResolutionStrategyAndTableType() { List opts = new ArrayList<>(); for (Object providerClass : LOCK_PROVIDER_CLASSES) { - for (ConflictResolutionStrategy resolutionStrategy: CONFLICT_RESOLUTION_STRATEGY_CLASSES) { + for (String resolutionStrategy: CONFLICT_RESOLUTION_STRATEGY_CLASSES) { opts.add(new Object[] {HoodieTableType.COPY_ON_WRITE, providerClass, resolutionStrategy}); opts.add(new Object[] {HoodieTableType.MERGE_ON_READ, providerClass, resolutionStrategy}); } @@ -265,7 +264,7 @@ public void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String ta @ParameterizedTest @MethodSource("providerClassResolutionStrategyAndTableType") public void testHoodieClientBasicMultiWriter(HoodieTableType tableType, Class providerClass, - ConflictResolutionStrategy resolutionStrategy) throws Exception { + String resolutionStrategy) throws Exception { if (tableType == HoodieTableType.MERGE_ON_READ) { setUpMORTestTable(); } @@ -422,7 +421,7 @@ private void latchCountDownAndWait(CountDownLatch latch, long waitTimeMillis) { @ParameterizedTest @MethodSource("providerClassResolutionStrategyAndTableType") public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType, Class providerClass, - ConflictResolutionStrategy resolutionStrategy) throws Exception { + String resolutionStrategy) throws Exception { // create inserts X 1 if (tableType == HoodieTableType.MERGE_ON_READ) { setUpMORTestTable(); @@ -681,7 +680,7 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) // Timeline-server-based markers are not used for multi-writer tests .withMarkersType(MarkerType.DIRECT.name()) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) - .withConflictResolutionStrategy(new SimpleConcurrentFileWritesConflictResolutionStrategy()) + .withConflictResolutionStrategy(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .build()).withAutoCommit(false).withProperties(properties); HoodieWriteConfig cfg = writeConfigBuilder.build(); HoodieWriteConfig cfg2 = writeConfigBuilder.build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiwriterWithIngestionAsPrimaryWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiwriterWithIngestionAsPrimaryWriter.java index 6d2e21821064d..44334829d0655 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiwriterWithIngestionAsPrimaryWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiwriterWithIngestionAsPrimaryWriter.java @@ -18,9 +18,9 @@ package org.apache.hudi.client; -import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.clustering.update.strategy.SparkAllowUpdateStrategy; +import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; -import org.apache.hudi.client.transaction.IngestionPrimaryWriterBasedConflictResolutionStrategy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; @@ -41,9 +41,12 @@ import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.testutils.HoodieClientTestBase; + +import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -79,13 +82,10 @@ public void clean() throws IOException { cleanupResources(); } - @ParameterizedTest - @EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"}) - public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception { + @Test + public void testMultiWriterWithAsyncTableServicesWithCompaction() throws Exception { // create inserts X 1 - if (tableType == HoodieTableType.MERGE_ON_READ) { - setUpMORTestTable(); - } + setUpMORTestTable(); Properties properties = new Properties(); properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); // Disabling embedded timeline server, it doesn't work with multiwriter @@ -99,10 +99,12 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta .withEmbeddedTimelineServerEnabled(false) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType( FileSystemViewStorageType.MEMORY).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClusteringNumCommits(1).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withInlineClusteringNumCommits(1) + .withClusteringUpdatesStrategy(SparkAllowUpdateStrategy.class.getName()).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) - .withConflictResolutionStrategy(new IngestionPrimaryWriterBasedConflictResolutionStrategy()) + .withConflictResolutionStrategy(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .build()).withAutoCommit(false).withProperties(properties).build(); Set validInstants = new HashSet<>(); // Create the first commit with inserts @@ -126,11 +128,11 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta int numRecords = 100; String commitTimeBetweenPrevAndNew = instantTime2; try { - // For both COW and MOR table types the commit should not be blocked, since we are giving preference to ingestion. + // For MOR table types the commit should be blocked, since compaction has the highest priority createCommitWithUpserts(cfg, client1, instantTime3, commitTimeBetweenPrevAndNew, instant4, numRecords); validInstants.add(instant4); } catch (Exception e1) { - throw new RuntimeException(e1); + Assertions.assertTrue(e1 instanceof HoodieWriteConflictException); } }); String instant5 = HoodieActiveTimeline.createNewInstantTime(); @@ -138,9 +140,7 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta try { client2.scheduleTableService(instant5, Option.empty(), TableServiceType.COMPACT); } catch (Exception e2) { - if (tableType == HoodieTableType.MERGE_ON_READ) { - throw new RuntimeException(e2); - } + throw new RuntimeException(e2); } }); String instant6 = HoodieActiveTimeline.createNewInstantTime(); @@ -162,7 +162,7 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta createCommitWithInserts(cfg, client1, instantTime3, instant7, numRecords); validInstants.add(instant7); } catch (Exception e1) { - throw new RuntimeException(e1); + Assertions.assertTrue(e1 instanceof HoodieWriteConflictException); } }); future2 = executors.submit(() -> { @@ -171,9 +171,7 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta client2.commitCompaction(instant5, compactionMetadata.getCommitMetadata().get(), Option.empty()); validInstants.add(instant5); } catch (Exception e2) { - if (tableType == HoodieTableType.MERGE_ON_READ) { - Assertions.assertTrue(e2 instanceof HoodieWriteConflictException); - } + throw new RuntimeException(e2); } }); future3 = executors.submit(() -> { @@ -204,10 +202,12 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) HoodieWriteConfig cfg = getConfigBuilder() .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) .withAutoClean(false).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClusteringNumCommits(1).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withInlineClusteringNumCommits(1) + .withClusteringUpdatesStrategy(SparkAllowUpdateStrategy.class.getName()).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class) - .withConflictResolutionStrategy(new IngestionPrimaryWriterBasedConflictResolutionStrategy()) + .withConflictResolutionStrategy(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .build()).withAutoCommit(false).withProperties(properties).build(); // Create the first commit String instant1 = HoodieActiveTimeline.createNewInstantTime(); @@ -238,7 +238,7 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient client, String prevCommitTime, String newCommitTime, int numRecords) throws Exception { - // Finish first base commmit + // Finish first base commit JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::bulkInsert, false, false, numRecords); assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); 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 75023f98b5a26..c8eb697a4a70a 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 @@ -28,9 +28,10 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.clustering.plan.strategy.SparkSingleFileSortPlanStrategy; import org.apache.hudi.client.clustering.run.strategy.SparkSingleFileSortExecutionStrategy; +import org.apache.hudi.client.clustering.update.strategy.SparkAllowUpdateStrategy; import org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy; import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; -import org.apache.hudi.client.transaction.IngestionPrimaryWriterBasedConflictResolutionStrategy; +import org.apache.hudi.client.transaction.StateTransitionTimeBasedConflictResolutionStrategy; import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.client.validator.SparkPreCommitValidator; @@ -2686,7 +2687,7 @@ public void testClusteringCommitInPresenceOfInflightCommit() throws Exception { properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); HoodieLockConfig lockConfig = HoodieLockConfig.newBuilder() .withLockProvider(FileSystemBasedLockProviderTestClass.class) - .withConflictResolutionStrategy(new IngestionPrimaryWriterBasedConflictResolutionStrategy()) + .withConflictResolutionStrategy(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .build(); HoodieCleanConfig cleanConfig = HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build(); @@ -2713,9 +2714,11 @@ public void testClusteringCommitInPresenceOfInflightCommit() throws Exception { 100, dataGenerator::generateUniqueUpdates, SparkRDDWriteClient::upsert, false, 0, 200, 2, false); - // Schedule and execute a clustering plan on the same partition. During conflict resolution the commit should fail. + // Schedule and execute a clustering plan on the same partition with SparkAllowUpdateStrategy, during conflict + // resolution the commit should fail. HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1) + .withClusteringUpdatesStrategy(SparkAllowUpdateStrategy.class.getName()) .build(); HoodiePreCommitValidatorConfig preCommitValidatorConfig = HoodiePreCommitValidatorConfig.newBuilder() .withPreCommitValidator(StringUtils.nullToEmpty(SqlQuerySingleResultPreCommitValidator.class.getName())) @@ -2731,7 +2734,7 @@ public void testClusteringCommitInPresenceOfInflightCommit() throws Exception { .build(); // create client with new config. - String clusteringCommitTime = HoodieActiveTimeline.createNewInstantTime();// HoodieActiveTimeline.createNewInstantTime(); + String clusteringCommitTime = HoodieActiveTimeline.createNewInstantTime(); SparkRDDWriteClient clusteringWriteClient = getHoodieWriteClient(clusteringWriteConfig); // Schedule and execute clustering, this should fail since there is a conflict between ingestion inflight commit. @@ -2754,7 +2757,7 @@ public void testIngestionCommitInPresenceOfCompletedClusteringCommit() throws Ex properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); HoodieLockConfig lockConfig = HoodieLockConfig.newBuilder() .withLockProvider(FileSystemBasedLockProviderTestClass.class) - .withConflictResolutionStrategy(new IngestionPrimaryWriterBasedConflictResolutionStrategy()) + .withConflictResolutionStrategy(StateTransitionTimeBasedConflictResolutionStrategy.class.getName()) .build(); HoodieCleanConfig cleanConfig = HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build(); @@ -2792,7 +2795,7 @@ public void testIngestionCommitInPresenceOfCompletedClusteringCommit() throws Ex // To create a successful commit. HoodieLockConfig clusteringLockConfig = HoodieLockConfig.newBuilder() .withLockProvider(FileSystemBasedLockProviderTestClass.class) - .withConflictResolutionStrategy(new SimpleConcurrentFileWritesConflictResolutionStrategy()) + .withConflictResolutionStrategy(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .build(); HoodiePreCommitValidatorConfig preCommitValidatorConfig = HoodiePreCommitValidatorConfig.newBuilder() .withPreCommitValidator(StringUtils.nullToEmpty(SqlQuerySingleResultPreCommitValidator.class.getName())) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index e50431c7398b9..dc6de394bf32d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -73,13 +73,6 @@ public static Stream> getAllPendingClu .filter(Option::isPresent).map(Option::get); } - /** - * Checks if the replacecommit is clustering commit. - */ - public static boolean isClusteringCommit(HoodieTableMetaClient metaClient, HoodieInstant pendingReplaceInstant) { - return getClusteringPlan(metaClient, pendingReplaceInstant).isPresent(); - } - /** * Get requested replace metadata from timeline. * @param metaClient diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index b68dcbd0698eb..696774a4bfece 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -19,9 +19,6 @@ package org.apache.hudi.configuration; import org.apache.hudi.client.clustering.plan.strategy.FlinkConsistentBucketClusteringPlanStrategy; -import org.apache.hudi.client.transaction.BucketIndexConcurrentFileWritesConflictResolutionStrategy; -import org.apache.hudi.client.transaction.ConflictResolutionStrategy; -import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.WriteConcurrencyMode; @@ -330,15 +327,6 @@ public static String[] getIndexKeys(Configuration conf) { return getIndexKeyField(conf).split(","); } - /** - * Returns the conflict resolution strategy. - */ - public static ConflictResolutionStrategy getConflictResolutionStrategy(Configuration conf) { - return isBucketIndexType(conf) - ? new BucketIndexConcurrentFileWritesConflictResolutionStrategy() - : new SimpleConcurrentFileWritesConflictResolutionStrategy(); - } - /** * Returns whether to commit even when current batch has no data, for flink defaults false */ diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java index 960b85d95ab86..5d76edba4654a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.lock.FileSystemBasedLockProvider; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieStorageConfig; @@ -227,7 +228,7 @@ public static HoodieWriteConfig getHoodieClientConfig( if (OptionsResolver.isLockRequired(conf) && !conf.containsKey(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key())) { // configure the fs lock provider by default builder.withLockConfig(HoodieLockConfig.newBuilder() - .withConflictResolutionStrategy(OptionsResolver.getConflictResolutionStrategy(conf)) + .withConflictResolutionStrategy(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) .withLockProvider(FileSystemBasedLockProvider.class) .withLockWaitTimeInMillis(2000L) // 2s .withFileSystemLockExpire(1) // 1 minute