[ 
https://issues.apache.org/jira/browse/HUDI-1138?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17391063#comment-17391063
 ] 

ASF GitHub Bot commented on HUDI-1138:
--------------------------------------

nsivabalan commented on a change in pull request #3233:
URL: https://github.com/apache/hudi/pull/3233#discussion_r680407670



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectMarkerFiles.java
##########
@@ -175,70 +165,30 @@ public static String stripMarkerSuffix(String path) {
     return markerFiles;
   }
 
-  private String stripMarkerFolderPrefix(String fullMarkerPath) {
-    
ValidationUtils.checkArgument(fullMarkerPath.contains(HoodieTableMetaClient.MARKER_EXTN));
-    String markerRootPath = Path.getPathWithoutSchemeAndAuthority(
-        new Path(String.format("%s/%s/%s", basePath, 
HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime))).toString();
-    int begin = fullMarkerPath.indexOf(markerRootPath);
-    ValidationUtils.checkArgument(begin >= 0,
-        "Not in marker dir. Marker Path=" + fullMarkerPath + ", Expected 
Marker Root=" + markerRootPath);
-    return fullMarkerPath.substring(begin + markerRootPath.length() + 1);
-  }
-
-  /**
-   * The marker path will be 
<base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
-   */
-  public Path create(String partitionPath, String dataFileName, IOType type) {
+  @Override
+  protected Option<Path> create(String partitionPath, String dataFileName, 
IOType type, boolean checkIfExists) {
+    HoodieTimer timer = new HoodieTimer().startTimer();
     Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
+    Path dirPath = markerPath.getParent();
     try {
-      LOG.info("Creating Marker Path=" + markerPath);
-      fs.create(markerPath, false).close();
+      if (!fs.exists(dirPath)) {

Review comment:
       not sure if this is addressed. can you help me understand how this is 
same as prior to this patch

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java
##########
@@ -78,6 +80,7 @@ public void upgrade(HoodieWriteConfig config, 
HoodieEngineContext context, Strin
   private static void recreateMarkerFiles(final String commitInstantTime,
                                           HoodieSparkTable table,
                                           HoodieEngineContext context,
+                                          MarkerType ioMode,

Review comment:
       fix variable names to markerType. 

##########
File path: 
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -304,7 +304,7 @@ protected void postCommit(HoodieTable<T, 
List<HoodieRecord<T>>, List<HoodieKey>,
                             Option<Map<String, String>> extraMetadata) {
     try {
       // Delete the marker directory for the instant.
-      new MarkerFiles(createTable(config, hadoopConf), instantTime)
+      MarkerFilesFactory.get(config.getMarkersType(), createTable(config, 
hadoopConf), instantTime)

Review comment:
       So, does this new marker strategy work for all engines (spark, flink and 
java) w/o any additional changes? If not, can you file a follow up ticket. 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -227,6 +228,30 @@
           + "files from lake storage, before committing the write. Reduce this 
value, if the high number of tasks incur delays for smaller tables "
           + "or low latency writes.");
 
+  public static final ConfigProperty<String> MARKERS_TYPE_PROP = ConfigProperty
+      .key("hoodie.write.markers.type")
+      .defaultValue(MarkerType.DIRECT.toString())
+      .sinceVersion("0.9.0")
+      .withDocumentation("Marker IO mode to use.  Two modes are supported: "

Review comment:
       guess you are still fixing the docs. just adding a reminder. 

##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+  public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+  private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  // A cached copy of all markers in memory
+  private final Set<String> allMarkers = new HashSet<>();
+  // A cached copy of marker entries in each marker file, stored in 
StringBuilder
+  // for efficient appending
+  // Mapping: {markerFileIndex -> markers}
+  private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+  // A list of use status of underlying files storing markers by a thread.
+  // {@code true} means the file is in use by a {@code 
BatchCreateMarkerRunnable}.
+  // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+  private final List<Boolean> threadUseStatus;
+  // A list of pending futures from async marker creation requests
+  private final List<MarkerCreationCompletableFuture> markerCreationFutures = 
new ArrayList<>();
+  private final int parallelism;
+  private final Object firstRequestLock = new Object();
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+  private boolean hasFirstRequest = false;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
+                        FileSystem fileSystem, HoodieEngineContext 
hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+
+    List<Boolean> isMarkerFileInUseList = new 
ArrayList<>(markerBatchNumThreads);
+    for (int i = 0; i < markerBatchNumThreads; i++) {
+      isMarkerFileInUseList.add(false);
+    }
+    this.threadUseStatus = Collections.synchronizedList(isMarkerFileInUseList);
+  }
+
+  /**
+   * @return  {@code true} if the marker directory exists in the system.
+   */
+  public boolean exists() {
+    try {
+      return fileSystem.exists(new Path(markerDirPath));
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+  }
+
+  /**
+   * @return all markers in the marker directory.
+   */
+  public Set<String> getAllMarkers() {
+    maybeSyncOnFirstRequest();
+    return allMarkers;
+  }
+
+  /**
+   * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+   * creation request to the queue.
+   *
+   * @param future  {@code MarkerCreationCompletableFuture} instance.
+   */
+  public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+    synchronized (markerCreationFutures) {
+      markerCreationFutures.add(future);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests.
+   */
+  public List<MarkerCreationCompletableFuture> 
fetchPendingMarkerCreationRequests() {
+    if (markerCreationFutures.isEmpty()) {
+      return new ArrayList<>();
+    }
+    maybeSyncOnFirstRequest();
+    List<MarkerCreationCompletableFuture> pendingFutures;
+    synchronized (markerCreationFutures) {
+      pendingFutures = new ArrayList<>(markerCreationFutures);
+      markerCreationFutures.clear();
+    }
+    return pendingFutures;
+  }
+
+  /**
+   * Processes pending marker creation requests if possible.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation 
requests
+   * @return A list of processed futures
+   */
+  public List<MarkerCreationCompletableFuture> processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> 
pendingMarkerCreationFutures) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      return Collections.emptyList();
+    }
+    maybeSyncOnFirstRequest();
+
+    int fileIndex = getNextFileIndexToUse();
+    if (fileIndex < 0) {
+      LOG.debug("All marker files are busy, skip batch processing of create 
marker requests in " + markerDirPath);
+      synchronized (markerCreationFutures) {
+        markerCreationFutures.addAll(pendingMarkerCreationFutures);
+      }
+      return Collections.emptyList();
+    }
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " fileIndex=" + 
fileIndex);
+
+    synchronized (markerCreationProcessingLock) {
+      for (MarkerCreationCompletableFuture future : 
pendingMarkerCreationFutures) {
+        String markerName = future.getMarkerName();
+        boolean exists = allMarkers.contains(markerName);
+        if (!exists) {
+          allMarkers.add(markerName);
+          StringBuilder stringBuilder = 
fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384));
+          stringBuilder.append(markerName);
+          stringBuilder.append('\n');
+        }
+        future.setResult(!exists);

Review comment:
       We can never get a repeat create marker request for a given file is it? 
and so we fail such calls? 

##########
File path: 
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/marker/TestMarkerFiles.java
##########
@@ -25,49 +25,27 @@
 import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
 import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.exception.HoodieException;
-import org.apache.hudi.testutils.HoodieClientTestUtils;
 
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.spark.api.java.JavaSparkContext;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.stream.Collectors;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertIterableEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class TestMarkerFiles extends HoodieCommonTestHarness {
-
-  private MarkerFiles markerFiles;
-  private FileSystem fs;
-  private Path markerFolderPath;
-  private JavaSparkContext jsc;
-  private HoodieSparkEngineContext context;
-
-  @BeforeEach
-  public void setup() throws IOException {
-    initPath();
-    initMetaClient();
-    this.jsc = new 
JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(TestMarkerFiles.class.getName()));
-    this.context = new HoodieSparkEngineContext(jsc);
-    this.fs = FSUtils.getFs(metaClient.getBasePath(), 
metaClient.getHadoopConf());
-    this.markerFolderPath =  new Path(metaClient.getMarkerFolderPath("000"));
-    this.markerFiles = new MarkerFiles(fs, metaClient.getBasePath(), 
markerFolderPath.toString(), "000");
-  }
+public abstract class TestMarkerFiles extends HoodieCommonTestHarness {

Review comment:
       rename to TestMarkerFilesBase

##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+  public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+  private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  // A cached copy of all markers in memory
+  private final Set<String> allMarkers = new HashSet<>();
+  // A cached copy of marker entries in each marker file, stored in 
StringBuilder
+  // for efficient appending
+  // Mapping: {markerFileIndex -> markers}
+  private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+  // A list of use status of underlying files storing markers by a thread.
+  // {@code true} means the file is in use by a {@code 
BatchCreateMarkerRunnable}.
+  // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+  private final List<Boolean> threadUseStatus;
+  // A list of pending futures from async marker creation requests
+  private final List<MarkerCreationCompletableFuture> markerCreationFutures = 
new ArrayList<>();
+  private final int parallelism;
+  private final Object firstRequestLock = new Object();
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+  private boolean hasFirstRequest = false;

Review comment:
       lazyInitComplete

##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+  public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+  private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  // A cached copy of all markers in memory
+  private final Set<String> allMarkers = new HashSet<>();
+  // A cached copy of marker entries in each marker file, stored in 
StringBuilder
+  // for efficient appending
+  // Mapping: {markerFileIndex -> markers}
+  private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+  // A list of use status of underlying files storing markers by a thread.
+  // {@code true} means the file is in use by a {@code 
BatchCreateMarkerRunnable}.
+  // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+  private final List<Boolean> threadUseStatus;
+  // A list of pending futures from async marker creation requests
+  private final List<MarkerCreationCompletableFuture> markerCreationFutures = 
new ArrayList<>();
+  private final int parallelism;
+  private final Object firstRequestLock = new Object();

Review comment:
       lazyInitLock

##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.table.view.FileSystemViewManager;
+import org.apache.hudi.timeline.service.TimelineService;
+import 
org.apache.hudi.timeline.service.handlers.marker.MarkerCreationCompletableFuture;
+import 
org.apache.hudi.timeline.service.handlers.marker.MarkerCreationDispatchingRunnable;
+import org.apache.hudi.timeline.service.handlers.marker.MarkerDirState;
+
+import io.javalin.Context;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+
+/**
+ * REST Handler servicing marker requests.
+ *
+ * The marker creation requests are handled asynchronous, while other types of 
requests
+ * are handled synchronous.
+ *
+ * Marker creation requests are batch processed periodically by a thread.  
Each batch
+ * processing thread adds new markers to a marker file.  Given that marker 
file operation
+ * can take time, multiple concurrent threads can run at the same, while they 
operate
+ * on different marker files storing mutually exclusive marker entries.  At 
any given
+ * time, a marker file is touched by at most one thread to guarantee 
consistency.
+ * Below is an example of running batch processing threads.
+ *
+ *           |-----| batch interval
+ * Thread 1  |-------------------------->| writing to MARKERS1
+ * Thread 2        |-------------------------->| writing to MARKERS2
+ * Thread 3               |-------------------------->| writing to MARKERS3
+ */
+public class MarkerHandler extends Handler {
+  private static final Logger LOG = LogManager.getLogger(MarkerHandler.class);
+
+  private final ExecutorService executorService;
+  // Parallelism for reading and deleting marker files
+  private final int parallelism;
+  // Marker directory states, {markerDirPath -> MarkerDirState instance}
+  private final Map<String, MarkerDirState> markerDirStateMap = new 
HashMap<>();
+  // A long-running thread to dispatch marker creation requests to batch 
processing threads
+  private final MarkerCreationDispatchingRunnable 
markerCreationDispatchingRunnable;
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Lock for synchronous processing of marker creating requests
+  private final Object createMarkerRequestLockObject = new Object();
+  // Next batch process timestamp in milliseconds
+  private boolean firstMarkerCreationRequest = true;
+
+  public MarkerHandler(Configuration conf, TimelineService.Config 
timelineServiceConfig,
+                       HoodieEngineContext hoodieEngineContext, FileSystem 
fileSystem,
+                       FileSystemViewManager viewManager, Registry 
metricsRegistry) throws IOException {
+    super(conf, timelineServiceConfig, fileSystem, viewManager);
+    LOG.debug("MarkerHandler FileSystem: " + this.fileSystem.getScheme());
+    LOG.debug("MarkerHandler batching params: batchNumThreads=" + 
timelineServiceConfig.markerBatchNumThreads
+        + " batchIntervalMs=" + timelineServiceConfig.markerBatchIntervalMs + 
"ms");
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = timelineServiceConfig.markerParallelism;
+    this.executorService = Executors.newSingleThreadExecutor();
+    this.markerCreationDispatchingRunnable = new 
MarkerCreationDispatchingRunnable(
+        markerDirStateMap, metricsRegistry, 
timelineServiceConfig.markerBatchNumThreads,
+        timelineServiceConfig.markerBatchIntervalMs);
+  }
+
+  public void stop() {
+    markerCreationDispatchingRunnable.stop();
+  }
+
+  /**
+   * @param markerDir marker directory path
+   * @return all marker paths in the marker directory
+   */
+  public Set<String> getAllMarkers(String markerDir) {
+    MarkerDirState markerDirState = 
markerDirStateMap.computeIfAbsent(markerDir,
+        k -> new MarkerDirState(markerDir, 
timelineServiceConfig.markerBatchNumThreads, fileSystem,
+            hoodieEngineContext, parallelism));
+    return markerDirState.getAllMarkers();
+  }
+
+  /**
+   * @param markerDir marker directory path
+   * @return all marker paths of write IO type "CREATE" and "MERGE"
+   */
+  public Set<String> getCreateAndMergeMarkers(String markerDir) {
+    return getAllMarkers(markerDir).stream()
+        .filter(markerName -> !markerName.endsWith(IOType.APPEND.name()))
+        .collect(Collectors.toSet());
+  }
+
+  /**
+   * @param markerDir  marker directory path
+   * @return {@code true} if the marker directory exists; {@code false} 
otherwise.
+   */
+  public boolean doesMarkerDirExist(String markerDir) {
+    MarkerDirState markerDirState = 
markerDirStateMap.computeIfAbsent(markerDir,
+        k -> new MarkerDirState(markerDir, 
timelineServiceConfig.markerBatchNumThreads, fileSystem,
+            hoodieEngineContext, parallelism));
+    return markerDirState.exists();
+  }
+
+  /**
+   * Generates a future for an async marker creation request
+   *
+   * The future is added to the marker creation future list and waits for the 
next batch processing
+   * of marker creation requests.
+   *
+   * @param context Javalin app context
+   * @param markerDir marker directory path
+   * @param markerName marker name
+   * @return the {@code CompletableFuture} instance for the request
+   */
+  public CompletableFuture<String> createMarker(Context context, String 
markerDir, String markerName) {
+    LOG.info("Request: create marker " + markerDir + " " + markerName);
+    MarkerCreationCompletableFuture future = new 
MarkerCreationCompletableFuture(context, markerDir, markerName);
+    // Add the future to the list
+    MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
+    if (markerDirState == null) {
+      synchronized (markerDirStateMap) {
+        markerDirState = new MarkerDirState(markerDir, 
timelineServiceConfig.markerBatchNumThreads,
+            fileSystem, hoodieEngineContext, parallelism);
+        markerDirStateMap.put(markerDir, markerDirState);
+      }
+    }
+    markerDirState.addMarkerCreationFuture(future);
+    synchronized (createMarkerRequestLockObject) {
+      if (firstMarkerCreationRequest) {

Review comment:
       You can probably make this Atomicboolean.
   And we can do something like
   if (firstMarkerCreationRequest.getAndSet(false)) {
      executorService.execute(markerCreationDispatchingRunnable);
   }

##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+  public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+  private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  // A cached copy of all markers in memory
+  private final Set<String> allMarkers = new HashSet<>();
+  // A cached copy of marker entries in each marker file, stored in 
StringBuilder
+  // for efficient appending
+  // Mapping: {markerFileIndex -> markers}
+  private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+  // A list of use status of underlying files storing markers by a thread.
+  // {@code true} means the file is in use by a {@code 
BatchCreateMarkerRunnable}.
+  // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+  private final List<Boolean> threadUseStatus;
+  // A list of pending futures from async marker creation requests
+  private final List<MarkerCreationCompletableFuture> markerCreationFutures = 
new ArrayList<>();
+  private final int parallelism;
+  private final Object firstRequestLock = new Object();
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+  private boolean hasFirstRequest = false;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
+                        FileSystem fileSystem, HoodieEngineContext 
hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+
+    List<Boolean> isMarkerFileInUseList = new 
ArrayList<>(markerBatchNumThreads);
+    for (int i = 0; i < markerBatchNumThreads; i++) {
+      isMarkerFileInUseList.add(false);
+    }
+    this.threadUseStatus = Collections.synchronizedList(isMarkerFileInUseList);
+  }
+
+  /**
+   * @return  {@code true} if the marker directory exists in the system.
+   */
+  public boolean exists() {
+    try {
+      return fileSystem.exists(new Path(markerDirPath));
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+  }
+
+  /**
+   * @return all markers in the marker directory.
+   */
+  public Set<String> getAllMarkers() {
+    maybeSyncOnFirstRequest();
+    return allMarkers;
+  }
+
+  /**
+   * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+   * creation request to the queue.
+   *
+   * @param future  {@code MarkerCreationCompletableFuture} instance.
+   */
+  public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+    synchronized (markerCreationFutures) {
+      markerCreationFutures.add(future);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests.
+   */
+  public List<MarkerCreationCompletableFuture> 
fetchPendingMarkerCreationRequests() {
+    if (markerCreationFutures.isEmpty()) {
+      return new ArrayList<>();
+    }
+    maybeSyncOnFirstRequest();
+    List<MarkerCreationCompletableFuture> pendingFutures;
+    synchronized (markerCreationFutures) {
+      pendingFutures = new ArrayList<>(markerCreationFutures);
+      markerCreationFutures.clear();
+    }
+    return pendingFutures;
+  }
+
+  /**
+   * Processes pending marker creation requests if possible.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation 
requests
+   * @return A list of processed futures
+   */
+  public List<MarkerCreationCompletableFuture> processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> 
pendingMarkerCreationFutures) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      return Collections.emptyList();
+    }
+    maybeSyncOnFirstRequest();
+
+    int fileIndex = getNextFileIndexToUse();
+    if (fileIndex < 0) {
+      LOG.debug("All marker files are busy, skip batch processing of create 
marker requests in " + markerDirPath);
+      synchronized (markerCreationFutures) {
+        markerCreationFutures.addAll(pendingMarkerCreationFutures);
+      }
+      return Collections.emptyList();
+    }
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " fileIndex=" + 
fileIndex);
+
+    synchronized (markerCreationProcessingLock) {
+      for (MarkerCreationCompletableFuture future : 
pendingMarkerCreationFutures) {
+        String markerName = future.getMarkerName();
+        boolean exists = allMarkers.contains(markerName);
+        if (!exists) {
+          allMarkers.add(markerName);
+          StringBuilder stringBuilder = 
fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384));
+          stringBuilder.append(markerName);
+          stringBuilder.append('\n');
+        }
+        future.setResult(!exists);
+      }
+    }
+    flushMarkersToFile(fileIndex);
+
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+    return pendingMarkerCreationFutures;
+  }
+
+  /**
+   * Deletes markers in the directory.
+   *
+   * @return {@code true} if successful; {@code false} otherwise.
+   */
+  public boolean deleteAllMarkers() {
+    Path dirPath = new Path(markerDirPath);
+    boolean result = false;
+    try {
+      if (fileSystem.exists(dirPath)) {
+        FileStatus[] fileStatuses = fileSystem.listStatus(dirPath);
+        List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
+            .map(fileStatus -> fileStatus.getPath().toString())
+            .collect(Collectors.toList());
+
+        if (markerDirSubPaths.size() > 0) {
+          SerializableConfiguration conf = new 
SerializableConfiguration(fileSystem.getConf());
+          int actualParallelism = Math.min(markerDirSubPaths.size(), 
parallelism);
+          hoodieEngineContext.foreach(markerDirSubPaths, subPathStr -> {
+            Path subPath = new Path(subPathStr);
+            FileSystem fileSystem = subPath.getFileSystem(conf.get());
+            fileSystem.delete(subPath, true);
+          }, actualParallelism);
+        }
+
+        result = fileSystem.delete(dirPath, true);
+        LOG.info("Removing marker directory at " + dirPath);
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+    allMarkers.clear();
+    fileMarkersMap.clear();
+    return result;
+  }
+
+  /**
+   * Syncs the markers from the underlying files for the first request.
+   */
+  private void maybeSyncOnFirstRequest() {
+    synchronized (firstRequestLock) {
+      if (!hasFirstRequest) {
+        syncMarkersFromFileSystem();
+        hasFirstRequest = true;
+      }
+    }
+  }
+
+  /**
+   * @return the next file index to use in a round-robin fashion
+   */
+  private int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {

Review comment:
       for loop is bit confusing me. Ideally if PUT request suceeds within 1 
sec, we should never go in a loop. First index after last known index should be 
available to server. 
    
   but here is my suggestion. Will leave it to you to take a call. 
   We can first check 
   if (lastFileIndex + 1 ) is free to use. 
   If not, we can go into a for loop. 
   Easier to read and reason about. 
   
   
   

##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+  public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+  private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  // A cached copy of all markers in memory
+  private final Set<String> allMarkers = new HashSet<>();
+  // A cached copy of marker entries in each marker file, stored in 
StringBuilder
+  // for efficient appending
+  // Mapping: {markerFileIndex -> markers}
+  private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+  // A list of use status of underlying files storing markers by a thread.
+  // {@code true} means the file is in use by a {@code 
BatchCreateMarkerRunnable}.
+  // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+  private final List<Boolean> threadUseStatus;
+  // A list of pending futures from async marker creation requests
+  private final List<MarkerCreationCompletableFuture> markerCreationFutures = 
new ArrayList<>();
+  private final int parallelism;
+  private final Object firstRequestLock = new Object();
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+  private boolean hasFirstRequest = false;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
+                        FileSystem fileSystem, HoodieEngineContext 
hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+
+    List<Boolean> isMarkerFileInUseList = new 
ArrayList<>(markerBatchNumThreads);
+    for (int i = 0; i < markerBatchNumThreads; i++) {
+      isMarkerFileInUseList.add(false);
+    }
+    this.threadUseStatus = Collections.synchronizedList(isMarkerFileInUseList);
+  }
+
+  /**
+   * @return  {@code true} if the marker directory exists in the system.
+   */
+  public boolean exists() {
+    try {
+      return fileSystem.exists(new Path(markerDirPath));
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+  }
+
+  /**
+   * @return all markers in the marker directory.
+   */
+  public Set<String> getAllMarkers() {
+    maybeSyncOnFirstRequest();
+    return allMarkers;
+  }
+
+  /**
+   * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+   * creation request to the queue.
+   *
+   * @param future  {@code MarkerCreationCompletableFuture} instance.
+   */
+  public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+    synchronized (markerCreationFutures) {
+      markerCreationFutures.add(future);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests.
+   */
+  public List<MarkerCreationCompletableFuture> 
fetchPendingMarkerCreationRequests() {
+    if (markerCreationFutures.isEmpty()) {
+      return new ArrayList<>();
+    }
+    maybeSyncOnFirstRequest();
+    List<MarkerCreationCompletableFuture> pendingFutures;
+    synchronized (markerCreationFutures) {
+      pendingFutures = new ArrayList<>(markerCreationFutures);
+      markerCreationFutures.clear();
+    }
+    return pendingFutures;
+  }
+
+  /**
+   * Processes pending marker creation requests if possible.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation 
requests
+   * @return A list of processed futures
+   */
+  public List<MarkerCreationCompletableFuture> processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> 
pendingMarkerCreationFutures) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      return Collections.emptyList();
+    }
+    maybeSyncOnFirstRequest();

Review comment:
       why call sync again here? I see that 
fetchPendingMarkerCreationRequests() is calling it anyway. Is there a scenario 
where processMarkerCreationRequests will be called w/o 
fetchPendingMarkerCreationRequests? 




-- 
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


> Re-implement marker files via timeline server
> ---------------------------------------------
>
>                 Key: HUDI-1138
>                 URL: https://issues.apache.org/jira/browse/HUDI-1138
>             Project: Apache Hudi
>          Issue Type: Improvement
>          Components: Writer Core
>    Affects Versions: 0.9.0
>            Reporter: Vinoth Chandar
>            Assignee: Ethan Guo
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 0.9.0
>
>
> Even as you can argue that RFC-15/consolidated metadata, removes the need for 
> deleting partial files written due to spark task failures/stage retries. It 
> will still leave extra files inside the table (and users will pay for it 
> every month) and we need the marker mechanism to be able to delete these 
> partial files. 
> Here we explore if we can improve the current marker file mechanism, that 
> creates one marker file per data file written, by 
> Delegating the createMarker() call to the driver/timeline server, and have it 
> create marker metadata into a single file handle, that is flushed for 
> durability guarantees
>  
> P.S: I was tempted to think Spark listener mechanism can help us deal with 
> failed tasks, but it has no guarantees. the writer job could die without 
> deleting a partial file. i.e it can improve things, but cant provide 
> guarantees 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to