[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588933140



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -359,10 +388,21 @@ public abstract O bulkInsertPreppedRecords(I 
preppedRecords, final String instan
* Common method containing steps to be performed before write 
(upsert/insert/...
* @param instantTime
* @param writeOperationType
+   * @param metaClient
*/
-  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType) {
+  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType,
+  HoodieTableMetaClient metaClient) {
 setOperationType(writeOperationType);
-syncTableMetadata();
+
this.txnManager.setLastCompletedTransaction(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
+.lastInstant());
+LOG.info("Last Instant Cached by writer with instant " + instantTime + " 
is " + this.txnManager.getLastCompletedTransactionOwner());
+this.txnManager.setTransactionOwner(Option.of(new 
HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)));
+this.txnManager.beginTransaction();

Review comment:
   Made changes accordingly





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588922630



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/lock/LockManager.java
##
@@ -0,0 +1,125 @@
+/*
+ * 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.lock;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * This class wraps implementations of {@link LockProvider} and provides an 
easy way to manage the lifecycle of a lock.
+ */
+public class LockManager implements Serializable {
+
+  private static final Logger LOG = LogManager.getLogger(LockManager.class);
+  private final HoodieWriteConfig writeConfig;
+  private final LockConfiguration lockConfiguration;
+  private final SerializableConfiguration hadoopConf;
+  private volatile LockProvider lockProvider;
+  // Holds the latest completed write instant to know which ones to check 
conflict against
+  private final AtomicReference> 
latestCompletedWriteInstant;
+
+  public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) {
+this.latestCompletedWriteInstant = new AtomicReference<>(Option.empty());
+this.writeConfig = writeConfig;
+this.hadoopConf = new SerializableConfiguration(fs.getConf());
+this.lockConfiguration = new LockConfiguration(writeConfig.getProps());
+  }
+
+  public void lock() {
+if 
(writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
+  LockProvider lockProvider = getLockProvider();
+  boolean acquired = false;
+  try {
+int retries = 
lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP);
+long waitTimeInMs = 
lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP);
+int retryCount = 0;
+while (retryCount <= retries) {
+  acquired = 
lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), 
TimeUnit.MILLISECONDS);
+  if (acquired) {
+break;
+  }
+  LOG.info("Retrying...");
+  Thread.sleep(waitTimeInMs);
+  retryCount++;
+}
+  } catch (Exception e) {

Review comment:
   added





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588918517



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/lock/SimpleConcurrentFileWritesConflictResolutionStrategy.java
##
@@ -0,0 +1,102 @@
+/*
+ * 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.lock;
+
+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.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieWriteConflictException;
+import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ConcurrentModificationException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+
+/**
+ * This class is a basic implementation of a conflict resolution strategy for 
concurrent writes {@link ConflictResolutionStrategy}.
+ */
+public class SimpleConcurrentFileWritesConflictResolutionStrategy
+implements ConflictResolutionStrategy {
+
+  private static final Logger LOG = 
LogManager.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class);
+
+  @Override
+  public Stream getInstantsStream(HoodieActiveTimeline 
activeTimeline, HoodieInstant currentInstant,
+ Option 
lastSuccessfulInstant) {
+
+// 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()
+// TODO : getWriteTimeline to ensure we include replace commits as well
+.filterCompletedInstants()
+.findInstantsAfter(lastSuccessfulInstant.isPresent() ? 
lastSuccessfulInstant.get().getTimestamp() : "0")
+.getInstants();
+
+Stream compactionAndClusteringTimeline = activeTimeline
+.getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, 
COMPACTION_ACTION))
+.findInstantsAfter(currentInstant.getTimestamp())
+.getInstants();
+return Stream.concat(completedCommitsInstantStream, 
compactionAndClusteringTimeline);
+  }
+
+  @Override
+  public boolean hasConflict(HoodieCommitOperation thisOperation, 
HoodieCommitOperation otherOperation) {
+// TODO : UUID's can clash even for insert/insert, handle that case.
+Set fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds();
+Set fileIdsSetForSecondInstant = 
otherOperation.getMutatedFileIds();
+Set intersection = new HashSet<>(fileIdsSetForFirstInstant);
+intersection.retainAll(fileIdsSetForSecondInstant);
+if (!intersection.isEmpty()) {
+  LOG.error("Found conflicting writes between first operation = " + 
thisOperation

Review comment:
   Okay, made it INFO





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588918353



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java
##
@@ -0,0 +1,191 @@
+/*
+ * 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.config;
+
+import 
org.apache.hudi.client.lock.SimpleConcurrentFileWritesConflictResolutionStrategy;
+import org.apache.hudi.client.lock.ConflictResolutionStrategy;
+import org.apache.hudi.client.lock.ZookeeperBasedLockProvider;
+import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.lock.LockProvider;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS;

Review comment:
   `HiveMetastoreLockProvider` has always been in `hudi-hive-sync` package. 
Only the LockConfiguration is in `hudi-common` since that is shared across 
`hudi-hive-sync` and `hudi-client-common`





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588918353



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java
##
@@ -0,0 +1,191 @@
+/*
+ * 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.config;
+
+import 
org.apache.hudi.client.lock.SimpleConcurrentFileWritesConflictResolutionStrategy;
+import org.apache.hudi.client.lock.ConflictResolutionStrategy;
+import org.apache.hudi.client.lock.ZookeeperBasedLockProvider;
+import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.lock.LockProvider;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS;

Review comment:
   `HiveMetastoreLockProvider` has always been in `hudi-hive-sync` package. 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588918281



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java
##
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.lock;
+
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+
+/**
+ * Pluggable lock implementations using this provider class.
+ */
+public abstract class LockProvider implements Lock, AutoCloseable {

Review comment:
   I want to make some methods final which we cannot do in interfaces. But 
it's fine, I have changed it to interface, I don't have a very strong 
preference. 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588918149



##
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
##
@@ -222,6 +225,17 @@ protected void commit(Option> 
extraMetadata, HoodieWriteMeta
 LOG.info("Committing metadata bootstrap !!");
   }
 
+  @Override
+  protected void syncTableMetadata() {

Review comment:
   https://issues.apache.org/jira/browse/HUDI-1665





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588917850



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java
##
@@ -0,0 +1,105 @@
+/*
+ * 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.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieWriteConflictException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ConcurrentModificationException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+
+/**
+ * This class is a basic implementation of a conflict resolution strategy for 
concurrent writes {@link ConflictResolutionStrategy}.
+ */
+public class SimpleConcurrentFileWritesConflictResolutionStrategy
+implements ConflictResolutionStrategy {
+
+  private static final Logger LOG = 
LogManager.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class);
+
+  @Override
+  public Stream getCandidateInstants(HoodieActiveTimeline 
activeTimeline, HoodieInstant currentInstant,
+ Option 
lastSuccessfulInstant) {
+
+// 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.
+// NOTE that any commits from table services such as compaction, 
clustering or cleaning since the
+// overlapping of files is handled using MVCC.
+Stream completedCommitsInstantStream = activeTimeline
+.getCommitsTimeline()
+.filterCompletedInstants()
+.findInstantsAfter(lastSuccessfulInstant.isPresent() ? 
lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS)
+.getInstants();
+
+Stream compactionAndClusteringTimeline = activeTimeline
+.getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, 
COMPACTION_ACTION))
+.findInstantsAfter(currentInstant.getTimestamp())
+.getInstants();
+return Stream.concat(completedCommitsInstantStream, 
compactionAndClusteringTimeline);
+  }
+
+  @Override
+  public boolean hasConflict(HoodieCommitOperation thisOperation, 
HoodieCommitOperation otherOperation) {
+// TODO : UUID's can clash even for insert/insert, handle that case.
+Set fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds();
+Set fileIdsSetForSecondInstant = 
otherOperation.getMutatedFileIds();
+Set intersection = new HashSet<>(fileIdsSetForFirstInstant);
+intersection.retainAll(fileIdsSetForSecondInstant);
+if (!intersection.isEmpty()) {
+  LOG.warn("Found conflicting writes between first operation = " + 
thisOperation
+  + ", second operation = " + otherOperation + " , intersecting file 
ids " + intersection);
+  return true;
+}
+return false;
+  }
+
+  @Override
+  public Option resolveConflict(HoodieTable table,
+  HoodieCommitOperation thisOperation, HoodieCommitOperation 
otherOperation) {
+// Since compaction is eventually written as commit, we need to ensure
+// we handle this during conflict resolution and not treat the commit from 
co

[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588917643



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java
##
@@ -0,0 +1,105 @@
+/*
+ * 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.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieWriteConflictException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ConcurrentModificationException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+
+/**
+ * This class is a basic implementation of a conflict resolution strategy for 
concurrent writes {@link ConflictResolutionStrategy}.
+ */
+public class SimpleConcurrentFileWritesConflictResolutionStrategy
+implements ConflictResolutionStrategy {
+
+  private static final Logger LOG = 
LogManager.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class);
+
+  @Override
+  public Stream getCandidateInstants(HoodieActiveTimeline 
activeTimeline, HoodieInstant currentInstant,
+ Option 
lastSuccessfulInstant) {
+
+// 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.
+// NOTE that any commits from table services such as compaction, 
clustering or cleaning since the
+// overlapping of files is handled using MVCC.
+Stream completedCommitsInstantStream = activeTimeline
+.getCommitsTimeline()
+.filterCompletedInstants()
+.findInstantsAfter(lastSuccessfulInstant.isPresent() ? 
lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS)
+.getInstants();
+
+Stream compactionAndClusteringTimeline = activeTimeline
+.getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, 
COMPACTION_ACTION))
+.findInstantsAfter(currentInstant.getTimestamp())
+.getInstants();
+return Stream.concat(completedCommitsInstantStream, 
compactionAndClusteringTimeline);
+  }
+
+  @Override
+  public boolean hasConflict(HoodieCommitOperation thisOperation, 
HoodieCommitOperation otherOperation) {
+// TODO : UUID's can clash even for insert/insert, handle that case.
+Set fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds();
+Set fileIdsSetForSecondInstant = 
otherOperation.getMutatedFileIds();
+Set intersection = new HashSet<>(fileIdsSetForFirstInstant);
+intersection.retainAll(fileIdsSetForSecondInstant);
+if (!intersection.isEmpty()) {
+  LOG.warn("Found conflicting writes between first operation = " + 
thisOperation
+  + ", second operation = " + otherOperation + " , intersecting file 
ids " + intersection);
+  return true;
+}
+return false;
+  }
+
+  @Override
+  public Option resolveConflict(HoodieTable table,
+  HoodieCommitOperation thisOperation, HoodieCommitOperation 
otherOperation) {
+// Since compaction is eventually written as commit, we need to ensure
+// we handle this during conflict resolution and not treat the commit from 
co

[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588857743



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -359,10 +388,21 @@ public abstract O bulkInsertPreppedRecords(I 
preppedRecords, final String instan
* Common method containing steps to be performed before write 
(upsert/insert/...
* @param instantTime
* @param writeOperationType
+   * @param metaClient
*/
-  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType) {
+  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType,
+  HoodieTableMetaClient metaClient) {
 setOperationType(writeOperationType);
-syncTableMetadata();
+
this.txnManager.setLastCompletedTransaction(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
+.lastInstant());
+LOG.info("Last Instant Cached by writer with instant " + instantTime + " 
is " + this.txnManager.getLastCompletedTransactionOwner());
+this.txnManager.setTransactionOwner(Option.of(new 
HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)));
+this.txnManager.beginTransaction();
+try {
+  syncTableMetadata();
+} finally {
+  this.txnManager.endTransaction();

Review comment:
   Yes, underlying unlock implementation should throw error. 

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/lock/ConflictResolutionStrategy.java
##
@@ -0,0 +1,64 @@
+/*
+ * 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.lock;
+
+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.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.stream.Stream;
+
+/**
+ * Strategy interface for conflict resolution with multiple writers.
+ * Users can provide pluggable implementations for different kinds of 
strategies to resolve conflicts when multiple
+ * writers are mutating the hoodie table.
+ */
+public interface ConflictResolutionStrategy {
+
+  /**
+   * Stream of instants to check conflicts against.
+   * @return
+   */
+  Stream getInstantsStream(HoodieActiveTimeline activeTimeline, 
HoodieInstant currentInstant, Option lastSuccessfulInstant);
+
+  /**
+   * Implementations of this method will determine whether a conflict exists 
between 2 commits.
+   * @param thisOperation
+   * @param otherOperation
+   * @return
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+  boolean hasConflict(HoodieCommitOperation thisOperation, 
HoodieCommitOperation otherOperation);
+
+  /**
+   * Implementations of this method will determine how to resolve a conflict 
between 2 commits.
+   * @param thisOperation
+   * @param otherOperation
+   * @return
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+  Option 
resolveConflict(Option metadataWriter, 
HoodieTable table,

Review comment:
   Removed





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588857743



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -359,10 +388,21 @@ public abstract O bulkInsertPreppedRecords(I 
preppedRecords, final String instan
* Common method containing steps to be performed before write 
(upsert/insert/...
* @param instantTime
* @param writeOperationType
+   * @param metaClient
*/
-  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType) {
+  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType,
+  HoodieTableMetaClient metaClient) {
 setOperationType(writeOperationType);
-syncTableMetadata();
+
this.txnManager.setLastCompletedTransaction(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
+.lastInstant());
+LOG.info("Last Instant Cached by writer with instant " + instantTime + " 
is " + this.txnManager.getLastCompletedTransactionOwner());
+this.txnManager.setTransactionOwner(Option.of(new 
HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)));
+this.txnManager.beginTransaction();
+try {
+  syncTableMetadata();
+} finally {
+  this.txnManager.endTransaction();

Review comment:
   Yes





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-06 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588857493



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java
##
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.lock;
+
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+
+/**
+ * Pluggable lock implementations using this provider class.
+ */
+public abstract class LockProvider implements Lock, AutoCloseable {
+
+  private static final Logger LOG = LogManager.getLogger(LockProvider.class);
+
+  protected LockConfiguration lockConfiguration;
+
+  @Override
+  public final void lockInterruptibly() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final void lock() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final boolean tryLock() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) {
+try {
+  return tryLock(time, unit);
+} catch (Exception e) {
+  throw new HoodieLockException(e);
+}
+  }
+
+  public T getLock() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final Condition newCondition() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() {
+try {
+  close();

Review comment:
   Intended to super.close()

##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/lock/LockProvider.java
##
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.lock;
+
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+
+/**
+ * Pluggable lock implementations using this provider class.
+ */
+public abstract class LockProvider implements Lock, AutoCloseable {
+
+  private static final Logger LOG = LogManager.getLogger(LockProvider.class);
+
+  protected LockConfiguration lockConfiguration;
+
+  @Override
+  public final void lockInterruptibly() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final void lock() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final boolean tryLock() {
+throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) {
+try {
+  return tryLock(time, unit);

Review comment:
   same as above





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-05 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r588086851



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/lock/SimpleConcurrentFileWritesConflictResolutionStrategy.java
##
@@ -0,0 +1,102 @@
+/*
+ * 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.lock;
+
+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.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieWriteConflictException;
+import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ConcurrentModificationException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+
+/**
+ * This class is a basic implementation of a conflict resolution strategy for 
concurrent writes {@link ConflictResolutionStrategy}.
+ */
+public class SimpleConcurrentFileWritesConflictResolutionStrategy
+implements ConflictResolutionStrategy {
+
+  private static final Logger LOG = 
LogManager.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class);
+
+  @Override
+  public Stream getInstantsStream(HoodieActiveTimeline 
activeTimeline, HoodieInstant currentInstant,
+ Option 
lastSuccessfulInstant) {
+
+// 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()
+// TODO : getWriteTimeline to ensure we include replace commits as well
+.filterCompletedInstants()
+.findInstantsAfter(lastSuccessfulInstant.isPresent() ? 
lastSuccessfulInstant.get().getTimestamp() : "0")
+.getInstants();
+
+Stream compactionAndClusteringTimeline = activeTimeline
+.getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, 
COMPACTION_ACTION))
+.findInstantsAfter(currentInstant.getTimestamp())
+.getInstants();
+return Stream.concat(completedCommitsInstantStream, 
compactionAndClusteringTimeline);
+  }
+
+  @Override
+  public boolean hasConflict(HoodieCommitOperation thisOperation, 
HoodieCommitOperation otherOperation) {
+// TODO : UUID's can clash even for insert/insert, handle that case.
+Set fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds();
+Set fileIdsSetForSecondInstant = 
otherOperation.getMutatedFileIds();
+Set intersection = new HashSet<>(fileIdsSetForFirstInstant);
+intersection.retainAll(fileIdsSetForSecondInstant);
+if (!intersection.isEmpty()) {
+  LOG.error("Found conflicting writes between first operation = " + 
thisOperation
+  + ", second operation = " + otherOperation + " , intersecting file 
ids " + intersection);
+  return true;
+}
+return false;
+  }
+
+  @Override
+  public Option 
resolveConflict(Option metadataWriter, 
HoodieTable table,
+  HoodieCommitOperation 
thisOperation, HoodieCommitOperation otherOperation) {
+// NOTE that any commits from table services such as compaction, 
clustering or cleaning since the
+// overlapping of files is handled us

[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-03-05 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r58089



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -30,16 +31,19 @@
 import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
 import org.apache.hudi.client.embedded.EmbeddedTimelineService;
 import org.apache.hudi.client.heartbeat.HeartbeatUtils;
+import org.apache.hudi.client.lock.TransactionManager;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.TableService;

Review comment:
   I kept it here because WriteOperationType is also here.

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -359,10 +388,21 @@ public abstract O bulkInsertPreppedRecords(I 
preppedRecords, final String instan
* Common method containing steps to be performed before write 
(upsert/insert/...
* @param instantTime
* @param writeOperationType
+   * @param metaClient
*/
-  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType) {
+  protected void preWrite(String instantTime, WriteOperationType 
writeOperationType,
+  HoodieTableMetaClient metaClient) {
 setOperationType(writeOperationType);
-syncTableMetadata();
+
this.txnManager.setLastCompletedTransaction(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
+.lastInstant());
+LOG.info("Last Instant Cached by writer with instant " + instantTime + " 
is " + this.txnManager.getLastCompletedTransactionOwner());
+this.txnManager.setTransactionOwner(Option.of(new 
HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)));
+this.txnManager.beginTransaction();

Review comment:
   I also realized this during implementation but wanted to keep 
`beginTransaction(..)` API simple. I've added a overridden method now

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/lock/ConflictResolutionStrategy.java
##
@@ -0,0 +1,64 @@
+/*
+ * 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.lock;
+
+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.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.stream.Stream;
+
+/**
+ * Strategy interface for conflict resolution with multiple writers.
+ * Users can provide pluggable implementations for different kinds of 
strategies to resolve conflicts when multiple
+ * writers are mutating the hoodie table.
+ */
+public interface ConflictResolutionStrategy {
+
+  /**
+   * Stream of instants to check conflicts against.
+   * @return
+   */
+  Stream getInstantsStream(HoodieActiveTimeline activeTimeline, 
HoodieInstant currentInstant, Option lastSuccessfulInstant);
+
+  /**
+   * Implementations of this method will determine whether a conflict exists 
between 2 commits.
+   * @param thisOperation
+   * @param otherOperation
+   * @return
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+  boolean hasConflict(HoodieCommitOperation thisOperation, 
HoodieCommitOperation otherOperation);
+
+  /**
+   * Implementations of this method will determine how to resolve a conflict 
between 2 commits.
+   * @param thisOperation
+   * @param otherOperation
+   * @return
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+  Option 
resolveConflict(Option metadataWriter, 
HoodieTable table,

Review comment:
   So this is 

[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-05 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569977625



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##
@@ -41,7 +41,8 @@
   public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
   public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
   public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
-
+  // Turn on inline cleaning
+  public static final String INLINE_CLEAN_PROP = "hoodie.clean.inline";

Review comment:
   Right now, this config is harmless, it just assumes the role of 
autoClean.

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -239,6 +272,9 @@ public void bootstrap(Option> 
extraMetadata) {
 if (rollbackPending) {
   rollBackInflightBootstrap();
 }
+if (config.getWriteConcurrencyMode().isMultiWriter()) {

Review comment:
   Right now, we don't have multi-writer so people might remain cautious. 
Once we support multiwriter, it's easy to set one config and run 
bootstrap/incremental, guarding against such behaviors. 
   Right now, do we have a way to enforce users to downgrade to a single writer 
?





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-03 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569978474



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -239,6 +272,9 @@ public void bootstrap(Option> 
extraMetadata) {
 if (rollbackPending) {
   rollBackInflightBootstrap();
 }
+if (config.getWriteConcurrencyMode().isMultiWriter()) {

Review comment:
   Right now, we don't have multi-writer so people might remain cautious. 
Once we support multiwriter, it's easy to set one config and run 
bootstrap/incremental, guarding against such behaviors. 
   Right now, do we have a way to enforce users to downgrade to a single writer 
?





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-03 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569977625



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##
@@ -41,7 +41,8 @@
   public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
   public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
   public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
-
+  // Turn on inline cleaning
+  public static final String INLINE_CLEAN_PROP = "hoodie.clean.inline";

Review comment:
   Right now, this config is harmless, it just assumes the role of 
autoClean.





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-03 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569201388



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##
@@ -389,6 +397,10 @@ public boolean isAsyncClean() {
 return 
Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.ASYNC_CLEAN_PROP));
   }
 
+  public boolean isInlineCleaning() {

Review comment:
   I kept the same naming convention as before for `isInlineCompaction` 
etc. I refactored all of them to `inline<..>Enabled`





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569200054



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##
@@ -923,6 +935,39 @@ public int getMetadataCleanerCommitsRetained() {
 return 
Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP));
   }
 
+  /**
+   * Hoodie Client Lock Configs.
+   * @return
+   */
+
+  public String getLockProviderClass() {
+return props.getProperty(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP);
+  }
+
+  public String getLockHiveDatabaseName() {
+return props.getProperty(HIVE_DATABASE_NAME_PROP);
+  }
+
+  public String getLockHiveTableName() {
+return props.getProperty(HIVE_TABLE_NAME_PROP);
+  }
+
+  public ConflictResolutionStrategy getWriteConflictResolutionStrategy() {
+return 
ReflectionUtils.loadClass(props.getProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP));
+  }
+
+  public Long getLockAcquireWaitTimeoutInMs() {
+return 
Long.valueOf(props.getProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP));
+  }
+
+  public WriteConcurrencyMode getWriteConcurrencyMode() {
+return 
WriteConcurrencyMode.fromValue(props.getProperty(WRITE_CONCURRENCY_MODE_PROP));
+  }
+
+  public Boolean isInlineTableServiceEnabled() {

Review comment:
   done





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569199577



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##
@@ -230,13 +235,18 @@ protected void initTableMetadata() {
 }
   }
 
-  protected void bootstrapIfNeeded(HoodieEngineContext engineContext, 
HoodieTableMetaClient datasetMetaClient) throws IOException {
-HoodieTimer timer = new HoodieTimer().startTimer();
-boolean exists = datasetMetaClient.getFs().exists(new 
Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME));
-if (!exists) {
-  // Initialize for the first time by listing partitions and files 
directly from the file system
-  bootstrapFromFilesystem(engineContext, datasetMetaClient);
-  metrics.ifPresent(m -> 
m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer()));
+  protected void bootstrapIfNeeded(HoodieEngineContext engineContext, 
HoodieTableMetaClient datasetMetaClient) {

Review comment:
   refactored to `bootstrapMetadata`

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##
@@ -88,7 +88,7 @@
   protected SerializableConfiguration hadoopConf;
   protected final transient HoodieEngineContext engineContext;
 
-  protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, 
HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) {
+  protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, 
HoodieWriteConfig writeConfig, HoodieEngineContext engineContext, boolean 
bootstrapIfNeeded) {

Review comment:
   refactored to `bootstrapMetadata` 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569198671



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -98,6 +112,7 @@
   private transient HoodieWriteCommitCallback commitCallback;
   private transient AsyncCleanerService asyncCleanerService;
   protected final boolean rollbackPending;
+  protected AtomicReference> latestCompletedWriteInstant 
= new AtomicReference<>();

Review comment:
   Created issue -> https://issues.apache.org/jira/browse/HUDI-1577





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569196481



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -403,30 +439,32 @@ protected void postCommit(HoodieTable table, 
HoodieCommitMetadata me
   // Delete the marker directory for the instant.
   new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, 
config.getMarkersDeleteParallelism());
 
-  // Do an inline compaction if enabled
-  if (config.isInlineCompaction()) {
-runAnyPendingCompactions(table);
-metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, 
"true");
-inlineCompact(extraMetadata);
-  } else {
-metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, 
"false");
-  }
+  if (config.isInlineTableServiceEnabled()) {

Review comment:
   I just wrapped all table services together as inline or async. The 
scheduling is protected via lock in inline as well as async. 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569191295



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/TableService.java
##
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.exception.HoodieException;
+
+import java.util.Locale;
+
+/**
+ * Supported runtime table services.
+ */
+public enum TableService {
+  COMPACT("compact"),

Review comment:
   It should be, but for each of these table services, we now follow the 
steps of `schedule` -> `inflight` -> `complete`. Archival doesn't do that right 
now and we never run archival async (unlike clean), so I haven't added it here. 
I have refactored the `clean` actions to do this. Filed a ticket for async 
archival -> https://issues.apache.org/jira/browse/HUDI-1576





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569191295



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/TableService.java
##
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.exception.HoodieException;
+
+import java.util.Locale;
+
+/**
+ * Supported runtime table services.
+ */
+public enum TableService {
+  COMPACT("compact"),

Review comment:
   It should be, but for each of these table services, we now follow the 
steps of `schedule` -> `inflight` -> `complete`. Archival doesn't do that right 
now and we never run archival async (unlike clean), so I haven't added it here. 
I have refactored the `clean` actions to do this. 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569194229



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -599,6 +637,7 @@ public HoodieRestoreMetadata restoreToInstant(final String 
instantTime) throws H
   public HoodieCleanMetadata clean(String cleanInstantTime) throws 
HoodieIOException {
 LOG.info("Cleaner started");
 final Timer.Context timerContext = metrics.getCleanCtx();
+scheduleCleaningAtInstant(cleanInstantTime, Option.empty());

Review comment:
   I refactored the code to ensure we follow `schedule` -> `inflight` -> 
`complete` for clean actions as well. 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569193815



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -403,30 +439,32 @@ protected void postCommit(HoodieTable table, 
HoodieCommitMetadata me
   // Delete the marker directory for the instant.
   new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, 
config.getMarkersDeleteParallelism());
 
-  // Do an inline compaction if enabled
-  if (config.isInlineCompaction()) {
-runAnyPendingCompactions(table);
-metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, 
"true");
-inlineCompact(extraMetadata);
-  } else {
-metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, 
"false");
-  }
+  if (config.isInlineTableServiceEnabled()) {
+// Do an inline compaction if enabled
+if (config.isInlineCompaction()) {
+  runAnyPendingCompactions(table);
+  metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, 
"true");
+  inlineCompact(extraMetadata);
+} else {
+  metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, 
"false");
+}
 
-  // Do an inline clustering if enabled
-  if (config.isInlineClustering()) {
-runAnyPendingClustering(table);
-metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, 
"true");
-inlineCluster(extraMetadata);
-  } else {
-metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, 
"false");
-  }
+// Do an inline clustering if enabled

Review comment:
   Intellij says ok, not sure why it's appearing this way here





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569192781



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -220,7 +253,7 @@ void emitCommitMetrics(String instantTime, 
HoodieCommitMetadata metadata, String
 }
   }
 
-  protected void syncTableMetadata() {
+  protected void syncTableMetadata(boolean bootstrapIfNeeded) {

Review comment:
   `init` also has a different meaning inside the 
`HoodieBackedTableMetadataWriter` where init just means initializing the table 
metadata. How about just `bootstrapMetadata` ?





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569191813



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/TableService.java
##
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.exception.HoodieException;
+
+import java.util.Locale;
+
+/**
+ * Supported runtime table services.
+ */
+public enum TableService {
+  COMPACT("compact"),
+  CLUSTER("cluster"),
+  CLEAN("clean");
+  private final String value;

Review comment:
   removed





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569191295



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/TableService.java
##
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.exception.HoodieException;
+
+import java.util.Locale;
+
+/**
+ * Supported runtime table services.
+ */
+public enum TableService {
+  COMPACT("compact"),

Review comment:
   It should be, but for each of these table services, we now follow the 
steps of `schedule` -> `inflight` -> `complete`. Archival doesn't do that right 
now, so I haven't added it here. I have refactored the `clean` actions to do 
this. 





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hudi] n3nash commented on a change in pull request #2374: [HUDI-845] Added locking capability to allow multiple writers

2021-02-02 Thread GitBox


n3nash commented on a change in pull request #2374:
URL: https://github.com/apache/hudi/pull/2374#discussion_r569189393



##
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/WriteConcurrencyMode.java
##
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.exception.HoodieException;
+
+import java.util.Locale;
+
+/**
+ * Different concurrency modes for write operations.
+ */
+public enum WriteConcurrencyMode {
+  NO_WRITER("no_writer"),

Review comment:
   Old code, removed.





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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org