Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -860,6 +860,16 @@ 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<String> delegateToTableServiceManager(TableServiceType tableServiceType, HoodieTable table) {
if (!config.getTableServiceManagerConfig().isEnabledAndActionSupported(ActionType.compaction)) {
return Option.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.hudi.client.utils.MetadataConversionUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieMetadataWrapper;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
Expand All @@ -31,6 +32,8 @@

import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -118,6 +121,8 @@ private void init(HoodieInstant instant) {
if (instant.isCompleted()) {
this.mutatedPartitionAndFileIds = getPartitionAndFileIdWithoutSuffixFromSpecificRecord(
this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToWriteStats());
Map<String, List<String>> partitionToReplaceFileIds = this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToReplaceFileIds();
this.mutatedPartitionAndFileIds.addAll(CommitUtils.flattenPartitionToReplaceFileIds(partitionToReplaceFileIds));
this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getOperationType());
} else {
// we need to have different handling for requested and inflight replacecommit because
Expand Down Expand Up @@ -158,6 +163,10 @@ private void init(HoodieInstant instant) {
case LOG_COMPACTION_ACTION:
this.mutatedPartitionAndFileIds = CommitUtils.getPartitionAndFileIdWithoutSuffix(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats());
this.operationType = this.metadataWrapper.getCommitMetadata().getOperationType();
if (this.operationType.equals(WriteOperationType.CLUSTER) || WriteOperationType.isOverwrite(this.operationType)) {
HoodieReplaceCommitMetadata replaceCommitMetadata = (HoodieReplaceCommitMetadata) this.metadataWrapper.getCommitMetadata();
mutatedPartitionAndFileIds.addAll(CommitUtils.flattenPartitionToReplaceFileIds(replaceCommitMetadata.getPartitionToReplaceFileIds()));
}
break;
default:
throw new IllegalArgumentException("Unsupported Action Type " + getInstantActionType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
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;
Expand All @@ -40,7 +40,7 @@ public interface ConflictResolutionStrategy {
* Stream of instants to check conflicts against.
* @return
*/
Stream<HoodieInstant> getCandidateInstants(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant, Option<HoodieInstant> lastSuccessfulInstant);
Stream<HoodieInstant> getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, Option<HoodieInstant> lastSuccessfulInstant);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is a public interface. there are chances that someone outside could have implemented their own resolution strategy. so, lets deprecate this and introduce a new one.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but curious to know why this change though ?
from the impl, I see that we are reloading the active timeline. so, why can't we do that at the caller before calling this method ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Infact resolveWriteConflictIfAny already had arguments whether to reload the active timeline or not. So, can't we leverage that?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is hard code to false now, kind of think it is hard to maintain for the invoker to know whether to refresh these timelines.

Copy link
Contributor Author

@suryaprasanna suryaprasanna Jun 11, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Caller does not now which implementation to use, so whether to refresh the timeline or not something should be up to the implementation classes. Like IngestionPrimaryWriterBasedConflictResolutionStrategy.java requires reload other may not require reload.
  • Also, if others have custom implementation of the base class and they are tied to using HoodieTimeline, they can still easily migrate to the new API with HoodieTableMetaClient whereas vice versa breaks the usability. In this case by replacing timeline with metaClient object we are providing more functionality to users.


/**
* Implementations of this method will determine whether a conflict exists between 2 commits.
Expand All @@ -61,4 +61,11 @@ public interface ConflictResolutionStrategy {
Option<HoodieCommitMetadata> resolveConflict(HoodieTable table,
ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) throws HoodieWriteConflictException;

/**
* Write clients uses their preCommit API to run conflict resolution.
* This method determines whether to execute preCommit for table services like clustering.
* @return boolean
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
boolean isPreCommitRequired();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* 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<HoodieInstant> getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant,
Option<HoodieInstant> 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<HoodieInstant> 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<HoodieInstant> completedCommitsInstants = activeTimeline
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets avoid code duplication. we can move code from within SimpleConcurrentFileWritesConflictResolutionStrategy and make them protected and re-use them here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

.getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION))
.filterCompletedInstants()
.findInstantsModifiedAfterByStateTransitionTime(currentInstant.getTimestamp())
.getInstantsOrderedByStateTransitionTime()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using state transition is driving by a config if I am not wrong.
@danny0405 : have we started using state transition everywhere? if not, we should avoid these apis.

Copy link
Contributor

@danny0405 danny0405 Jun 3, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's okay we add some apis on the timeline, the timeline api is kind of internal, users should not be exposed to/aware of these changes. Anyway we need some completion time filtering on the timeline, these apis can be reused when we migrate to more elegant impl for the real time solution.

Copy link
Contributor Author

@suryaprasanna suryaprasanna Jun 11, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, completion time ordering API will be ideal to use. For now rewording the findInstantsModifiedAfter method to findInstantsModifiedAfterByStateTransitionTime.

.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<HoodieInstant> getCandidateInstantsForTableServicesCommits(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant) {
// Fetch list of completed commits.
Stream<HoodieInstant> 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<HoodieInstant> inflightIngestionCommitsStream =
Copy link
Contributor

@Zouxxyy Zouxxyy Jul 12, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@suryaprasanna @danny0405 One reminder, not all inflight commits have metadata, for example, bulk insert's inflight metadata is empty,
in this case, IngestionPrimaryWriterBasedConflictResolutionStrategy will be invalid.

image

Flink even not save metadata in inflight

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Zouxxyy I idea behind creating IngestionPrimaryWriterBasedConflictResolutionStrategy, is to give preference for Ingestion writers over clustering writers.
For example, you have clustering plans as mutable plans which can be created and removed by rollback, this is achievable by using SparkAllowUpdateStrategy. Clustering writers and Ingestion writers can run simultaneously on the same partition. If clustering writer were to complete writing first and about to create a completed replacecommit, in that stage it will check if there are any pending inflight ingestion instants in the timeline and fail if it finds an overlap.
For bulk insert operations, they create new files so there wont be ay conflict with clustering writers, so in that case replacecommit is allowed to commit. There could be a case where if users are using insert API, but it can still write to existing files, if small file handling is enabled. Even then possibility of overlap between clustering and ingestion can be reduced and even if there is overlap you can fail clustering commit and allow Ingestion to commit even if it is arriving late.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in that stage it will check if there are any pending inflight ingestion instants in the timeline and fail if it finds an overlap.

Thanks for the reply, what I mean is now we use the metadata of pending ingestion instant for conflict detection here, but not all inflight commits have metadata, e.g. the .inflght file written by flink is empty.

And can you help review this #9220 which improve the current stragies, thanks

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. I am not much familiar with Flink, but on the Spark engine the inflight commits that are doing upsert operations should have partition to fileIds(that needs to be modified) present in the .inflight file, this is actually done after the WorkloadProfile object is loaded. So, conflict resolution will only see those file ids that are getting modified not new ones. During conflict resolution that is all we care, since there won't be any overlap between new files. Sure, I will review the RFC.

activeTimeline
.getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION))
.filterInflights()
.getInstantsAsStream();

// Merge and sort the instants and return.
List<HoodieInstant> 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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

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;
Expand Down Expand Up @@ -49,9 +50,9 @@ public class SimpleConcurrentFileWritesConflictResolutionStrategy
private static final Logger LOG = LoggerFactory.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class);

@Override
public Stream<HoodieInstant> getCandidateInstants(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant,
Option<HoodieInstant> lastSuccessfulInstant) {

public Stream<HoodieInstant> getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant,
Option<HoodieInstant> 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
Expand Down Expand Up @@ -110,4 +111,9 @@ public Option<HoodieCommitMetadata> resolveConflict(HoodieTable table,
throw new HoodieWriteConflictException(new ConcurrentModificationException("Cannot resolve conflicts for overlapping writes"));
}

@Override
public boolean isPreCommitRequired() {
return false;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,13 @@ public static Option<HoodieCommitMetadata> resolveWriteConflictIfAny(
Stream<HoodieInstant> completedInstantsDuringCurrentWriteOperation = getCompletedInstantsDuringCurrentWriteOperation(table.getMetaClient(), pendingInstants);

ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy();
Stream<HoodieInstant> instantStream = Stream.concat(resolutionStrategy.getCandidateInstants(reloadActiveTimeline
? table.getMetaClient().reloadActiveTimeline() : table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant),
if (reloadActiveTimeline) {
table.getMetaClient().reloadActiveTimeline();
}
Stream<HoodieInstant> instantStream = Stream.concat(resolutionStrategy.getCandidateInstants(
table.getMetaClient(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant),
completedInstantsDuringCurrentWriteOperation);

final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElse(new HoodieCommitMetadata()));
instantStream.forEach(instant -> {
try {
Expand Down Expand Up @@ -105,6 +109,10 @@ public static Option<Pair<HoodieInstant, Map<String, String>>> getLastCompletedT
HoodieTableMetaClient metaClient) {
Option<HoodieInstant> hoodieInstantOption = metaClient.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants().lastInstant();
return getHoodieInstantAndMetaDataPair(metaClient, hoodieInstantOption);
}

private static Option<Pair<HoodieInstant, Map<String, String>>> getHoodieInstantAndMetaDataPair(HoodieTableMetaClient metaClient, Option<HoodieInstant> hoodieInstantOption) {
try {
if (hoodieInstantOption.isPresent()) {
HoodieCommitMetadata commitMetadata = TimelineUtils.getCommitMetadata(hoodieInstantOption.get(), metaClient.getActiveTimeline());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public void testNoConcurrentWrites() throws Exception {
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime));

SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy();
Stream<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant);
Stream<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant);
Assertions.assertEquals(0, candidateInstants.count());
}

Expand All @@ -75,7 +75,7 @@ public void testConcurrentWrites() throws Exception {
newInstantTime = HoodieTestTable.makeNewCommitTime();
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime));
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy();
Stream<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant);
Stream<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant);
Assertions.assertEquals(0, candidateInstants.count());
}

Expand All @@ -95,8 +95,8 @@ public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Except
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy();
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
timeline = timeline.reload();
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
metaClient.reloadActiveTimeline();
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect(
Collectors.toList());
// writer 1 conflicts with writer 2
Assertions.assertEquals(1, candidateInstants.size());
Expand Down Expand Up @@ -127,8 +127,8 @@ public void testConcurrentWritesWithDifferentPartition() throws Exception {
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy();
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH);
timeline = timeline.reload();
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect(
metaClient.reloadActiveTimeline();
List<HoodieInstant> candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect(
Collectors.toList());

// there should be 1 candidate instant
Expand Down
Loading