nsivabalan commented on code in PR #8832: URL: https://github.com/apache/hudi/pull/8832#discussion_r1214772268
########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/IngestionPrimaryWriterBasedConflictResolutionStrategy.java: ########## @@ -0,0 +1,111 @@ +/* + * 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 + .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) + .filterCompletedInstants() + .findInstantsModifiedAfter(currentInstant.getTimestamp()) + .getInstantsOrderedByStateTransitionTime() Review Comment: 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. ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java: ########## @@ -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); Review Comment: 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. ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java: ########## @@ -234,6 +235,11 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); finalizeWrite(table, clusteringCommitTime, writeStats); + // Do conflict resolution checks for clustering if SparkAllowUpdateStrategy is used. + // By using this UpdateStrategy implementation, Ingestion writers are given preference over clustering re-writers. + if (this.config.getWriteConflictResolutionStrategy() instanceof IngestionPrimaryWriterBasedConflictResolutionStrategy) { Review Comment: this is very tight coupling. lets try to abstract this out. may be we can introduce new api like isPreCommitRequired() within ConflictResolutionStrategy interface. and call preCommit when it returns true. ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java: ########## @@ -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); Review Comment: Infact resolveWriteConflictIfAny already had arguments whether to reload the active timeline or not. So, can't we leverage that? ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConflictResolutionStrategy.java: ########## @@ -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); Review Comment: 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 ? ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/IngestionPrimaryWriterBasedConflictResolutionStrategy.java: ########## @@ -0,0 +1,111 @@ +/* + * 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 Review Comment: lets avoid code duplication. we can move code from within SimpleConcurrentFileWritesConflictResolutionStrategy and make them protected and re-use them here. ########## hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java: ########## @@ -268,6 +267,30 @@ public String getStateTransitionTime() { return stateTransitionTime; } + /* Review Comment: why commented out? if not required, we can remove ########## hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java: ########## @@ -204,6 +204,13 @@ public HoodieDefaultTimeline findInstantsInRangeByStateTransitionTime(String sta details); } + @Override + public HoodieDefaultTimeline findInstantsModifiedAfter(String instantTime) { + return new HoodieDefaultTimeline(instants.stream() + .filter(s -> HoodieTimeline.compareTimestamps(s.getStateTransitionTime(), Review Comment: lets be wary on using state transition time -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org