zhangyue19921010 commented on code in PR #6133:
URL: https://github.com/apache/hudi/pull/6133#discussion_r1028259347


##########
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java:
##########
@@ -150,7 +168,50 @@ public boolean doesMarkerDirExist(String markerDir) {
    * @param markerName marker name
    * @return the {@code CompletableFuture} instance for the request
    */
-  public CompletableFuture<String> createMarker(Context context, String 
markerDir, String markerName) {
+  public CompletableFuture<String> createMarker(Context context, String 
markerDir, String markerName,
+                                                String batchInterval, String 
period, String maxAllowableHeartbeatIntervalInMs,
+                                                String basePath, String 
earlyConflictDetectionEnable,
+                                                String 
earlyConflictDetectionClassName) {
+    // Step1 do early conflict detection if enable
+    if (Boolean.parseBoolean(earlyConflictDetectionEnable)) {
+      try {
+        synchronized (earlyConflictDetectionLock) {
+          if (earlyConflictDetectionStrategy == null) {
+            earlyConflictDetectionStrategy = 
ReflectionUtils.loadClass(earlyConflictDetectionClassName);
+          }
+
+          if (!markerDir.equalsIgnoreCase(currentMarkerDir)) {
+            this.currentMarkerDir = markerDir;
+            Set<String> actions = CollectionUtils.createSet(COMMIT_ACTION, 
DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION);
+            Set<HoodieInstant> oldInstants = 
viewManager.getFileSystemView(basePath)
+                .getTimeline()
+                .filterCompletedInstants()
+                .filter(instant -> actions.contains(instant.getAction()))
+                .getInstants()
+                .collect(Collectors.toSet());
+
+            earlyConflictDetectionStrategy.fresh(batchInterval, period, 
markerDir, basePath, maxAllowableHeartbeatIntervalInMs, fileSystem,
+                this, oldInstants);
+          }
+        }
+
+        if (earlyConflictDetectionStrategy.hasMarkerConflict()) {
+          earlyConflictDetectionStrategy.resolveMarkerConflict(basePath, 
markerDir, markerName);

Review Comment:
   Same reason, this check is batch and async. For specific request get false 
result. It means maker checker find a conflict but maybe it is not current 
request related marker conflict.
   
   So is it possible to let current executor to handle others' conflict based 
on timeline sever in async and batch mode. :)



##########
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCheckerRunnable.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.MarkerUtils;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.timeline.service.handlers.MarkerHandler;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class MarkerCheckerRunnable implements Runnable {
+  private static final Logger LOG = 
LogManager.getLogger(MarkerCheckerRunnable.class);
+
+  private MarkerHandler markerHandler;
+  private String markerDir;
+  private String basePath;
+  private FileSystem fs;
+  private AtomicBoolean hasConflict;
+  private long maxAllowableHeartbeatIntervalInMs;
+  private Set<HoodieInstant> oldInstants;
+
+  public MarkerCheckerRunnable(AtomicBoolean hasConflict, MarkerHandler 
markerHandler, String markerDir,
+                               String basePath, FileSystem fileSystem, long 
maxAllowableHeartbeatIntervalInMs,
+                               Set<HoodieInstant> oldInstants) {
+    this.markerHandler = markerHandler;
+    this.markerDir = markerDir;
+    this.basePath = basePath;
+    this.fs = fileSystem;
+    this.hasConflict = hasConflict;
+    this.maxAllowableHeartbeatIntervalInMs = maxAllowableHeartbeatIntervalInMs;
+    this.oldInstants = oldInstants;
+  }
+
+  @Override
+  public void run() {
+    try {
+      if (!fs.exists(new Path(markerDir))) {
+        return;
+      }
+
+      HoodieTimer timer = new HoodieTimer().startTimer();

Review Comment:
   changed!



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to