You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/08/09 15:47:04 UTC

[GitHub] [hudi] vinothchandar commented on a change in pull request #3233: [HUDI-1138] Add timeline-server-based marker file strategy for improving marker-related latency

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



##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationCompletableFuture.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.util.HoodieTimer;
+
+import io.javalin.Context;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Future for async marker creation request.
+ */
+public class MarkerCreationCompletableFuture extends CompletableFuture<String> {
+  private static final Logger LOG = LogManager.getLogger(MarkerCreationCompletableFuture.class);
+  private final Context context;
+  private final String markerDirPath;
+  private final String markerName;
+  private boolean result;
+  private final HoodieTimer timer;
+
+  public MarkerCreationCompletableFuture(Context context, String markerDirPath, String markerName) {
+    super();
+    this.timer = new HoodieTimer().startTimer();
+    this.context = context;
+    this.markerDirPath = markerDirPath;
+    this.markerName = markerName;
+    this.result = false;
+  }
+
+  public Context getContext() {
+    return context;
+  }
+
+  public String getMarkerDirPath() {
+    return markerDirPath;
+  }
+
+  public String getMarkerName() {
+    return markerName;
+  }
+
+  public boolean getResult() {

Review comment:
       rename: isSuccessful()

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.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;
+
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+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 Registry metricsRegistry;
+  // a scheduled executor service to schedule dispatching of marker creation requests
+  private final ScheduledExecutorService dispatchingExecutorService;
+  // an executor service to schedule the worker threads of batch processing marker creation requests
+  private final ExecutorService batchingExecutorService;
+  // 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 thread to dispatch marker creation requests to batch processing threads
+  private final MarkerCreationDispatchingRunnable markerCreationDispatchingRunnable;
+  private final AtomicBoolean firstCreationRequestSeen;
+  private transient HoodieEngineContext hoodieEngineContext;
+  private ScheduledFuture<?> dispatchingScheduledFuture;

Review comment:
       more descriptive name?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.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;
+
+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.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+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 Registry metricsRegistry;
+  // a scheduled executor service to schedule dispatching of marker creation requests
+  private final ScheduledExecutorService dispatchingExecutorService;
+  // an executor service to schedule the worker threads of batch processing marker creation requests
+  private final ExecutorService batchingExecutorService;
+  // 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 thread to dispatch marker creation requests to batch processing threads
+  private final MarkerCreationDispatchingRunnable markerCreationDispatchingRunnable;
+  private final AtomicBoolean firstCreationRequestSeen;
+  private transient HoodieEngineContext hoodieEngineContext;
+  private ScheduledFuture<?> dispatchingScheduledFuture;
+
+  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.metricsRegistry = metricsRegistry;
+    this.parallelism = timelineServiceConfig.markerParallelism;
+    this.dispatchingExecutorService = Executors.newSingleThreadScheduledExecutor();
+    this.batchingExecutorService = Executors.newFixedThreadPool(timelineServiceConfig.markerBatchNumThreads);
+    this.markerCreationDispatchingRunnable =
+        new MarkerCreationDispatchingRunnable(markerDirStateMap, batchingExecutorService);
+    this.firstCreationRequestSeen = new AtomicBoolean(false);
+  }
+
+  /**
+   * Stops the dispatching of marker creation requests.
+   */
+  public void stop() {
+    if (dispatchingScheduledFuture != null) {
+      dispatchingScheduledFuture.cancel(true);
+    }
+    dispatchingExecutorService.shutdown();
+    batchingExecutorService.shutdown();
+  }
+
+  /**
+   * @param markerDir marker directory path
+   * @return all marker paths in the marker directory
+   */
+  public Set<String> getAllMarkers(String markerDir) {
+    MarkerDirState markerDirState = getMarkerDirState(markerDir);
+    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 = getMarkerDirState(markerDir);
+    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 = getMarkerDirState(markerDir);
+    markerDirState.addMarkerCreationFuture(future);
+    if (!firstCreationRequestSeen.getAndSet(true)) {
+      dispatchingScheduledFuture = dispatchingExecutorService.scheduleAtFixedRate(markerCreationDispatchingRunnable,

Review comment:
       just for correctness. the setting can succeed above and we could fail to while scheduling in L159? Would tht leave the flag in an inconsistent state?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A runnable / worker for scheduling batch processing of marker creation requests.
+ */
+public class MarkerCreationDispatchingRunnable implements Runnable {
+  public static final Logger LOG = LogManager.getLogger(MarkerCreationDispatchingRunnable.class);
+
+  // Marker directory states, {markerDirPath -> MarkerDirState instance}
+  private final Map<String, MarkerDirState> markerDirStateMap;
+  private final ExecutorService executorService;
+
+  public MarkerCreationDispatchingRunnable(
+      Map<String, MarkerDirState> markerDirStateMap, ExecutorService executorService) {
+    this.markerDirStateMap = markerDirStateMap;
+    this.executorService = executorService;
+  }
+
+  /**
+   * Dispatches the marker creation requests that can be process to a worker thread of batch
+   * processing the requests.
+   *
+   * For each marker directory, goes through the following steps:
+   * (1) find the next available file index for writing.  If no file index is available,
+   *   skip the processing of this marker directory;
+   * (2) fetch the pending marker creation requests for this marker directory.  If there is
+   *   no request, skip this marker directory;
+   * (3) put the marker directory, marker dir state, list of requests futures, and the file index
+   *   to a {@code MarkerDirRequestContext} instance and add the instance to the request context list.
+   *
+   * If the request context list is not empty, spins up a worker thread, {@code MarkerCreationBatchingRunnable},
+   * and pass all the request context to the thread for batch processing.  The thread is responsible
+   * for responding to the request futures directly.
+   */
+  @Override
+  public void run() {
+    List<MarkerDirRequestContext> requestContextList = new ArrayList<>();
+
+    // Only fetch pending marker creation requests that can be processed,
+    // i.e., that markers can be written to a underlying file
+    for (String markerDir : markerDirStateMap.keySet()) {
+      MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
+      int fileIndex = markerDirState.getNextFileIndexToUse();

Review comment:
       instead of returning an int with < 0 value. can we use an Option<Integer>

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  public List<MarkerCreationCompletableFuture> fetchPendingMarkerCreationRequests() {
+    if (markerCreationFutures.isEmpty()) {

Review comment:
       just a note to check this is safe to execute with concurrent appends to the list?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {

Review comment:
       one line comment stating that this is scanning for the next free index? I wonder if you can just avoid the entire if block, by simply starting with `i = 0` in this loop? no?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationBatchingRunnable.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.HoodieTimer;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+
+/**
+ * A runnable for batch processing marker creation requests.
+ */
+public class MarkerCreationBatchingRunnable implements Runnable {

Review comment:
       rename. `BatchedMarkerCreationRunnable` 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * A runnable / worker for scheduling batch processing of marker creation requests.

Review comment:
       "Runnable that performs periodic, batched creation of markers for write operations" ?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {

Review comment:
       generally speaking, can all synchronization in this class be based on the lock object? it makes it easy to maintain. Otherwise, one needs to thinking about "thread x holding y lock" when  "thread A holds B lock" scenarios

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirRequestContext.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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 java.util.List;
+
+/**
+ * Input of batch processing of marker creation requests for a single marker directory.
+ */
+public class MarkerDirRequestContext {

Review comment:
       rename: "MarkerCreationRequestContext". This is more about the actual batched creation?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationCompletableFuture.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.util.HoodieTimer;
+
+import io.javalin.Context;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Future for async marker creation request.
+ */
+public class MarkerCreationCompletableFuture extends CompletableFuture<String> {

Review comment:
       rename: MarkerCreationFuture

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  public List<MarkerCreationCompletableFuture> fetchPendingMarkerCreationRequests() {
+    if (markerCreationFutures.isEmpty()) {
+      return new ArrayList<>();
+    }
+    maybeSyncOnFirstRequest();

Review comment:
       why not just do this once during constructor? We know the markerDir anyway? this will make rest of the code, not deal with initing.  e.g if we introduce new methods, then that author should remember to propogate the syncing .

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  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.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation requests
+   * @param fileIndex file index to use to write markers
+   */
+  public void processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> pendingMarkerCreationFutures, int fileIndex) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      markFileAsAvailable(fileIndex);
+      return;
+    }
+
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath
+        + " numRequests=" + pendingMarkerCreationFutures.size() + " 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);
+    markFileAsAvailable(fileIndex);
+
+    for (MarkerCreationCompletableFuture future : pendingMarkerCreationFutures) {
+      try {
+        future.complete(jsonifyResult(
+            future.getContext(), future.getResult(), metricsRegistry, OBJECT_MAPPER, LOG));
+      } catch (JsonProcessingException e) {
+        throw new HoodieException("Failed to JSON encode the value", e);
+      }
+    }
+  }
+
+  /**
+   * 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();

Review comment:
       any synchrnoization needed?  for e.g if there were concurent adds, this will throw a ConcurrentModificationException. Just think through those scenarios?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  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.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation requests
+   * @param fileIndex file index to use to write markers
+   */
+  public void processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> pendingMarkerCreationFutures, int fileIndex) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      markFileAsAvailable(fileIndex);
+      return;
+    }
+
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath
+        + " numRequests=" + pendingMarkerCreationFutures.size() + " 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);
+    markFileAsAvailable(fileIndex);
+
+    for (MarkerCreationCompletableFuture future : pendingMarkerCreationFutures) {
+      try {
+        future.complete(jsonifyResult(
+            future.getContext(), future.getResult(), metricsRegistry, OBJECT_MAPPER, LOG));
+      } catch (JsonProcessingException e) {
+        throw new HoodieException("Failed to JSON encode the value", e);
+      }
+    }
+  }
+
+  /**
+   * 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);

Review comment:
       do we need recursive delete calls if we are already doing sub path listing and deleting sub paths?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  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.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation requests
+   * @param fileIndex file index to use to write markers
+   */
+  public void processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> pendingMarkerCreationFutures, int fileIndex) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      markFileAsAvailable(fileIndex);
+      return;
+    }
+
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath
+        + " numRequests=" + pendingMarkerCreationFutures.size() + " 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);

Review comment:
       if we fail here due to s3 error lets say. then the in-memory state is not consistent? for future requests? may be it does not matter tht there are marker entries that have no file associated (this case can always happen if the executor dies right after the marker request returns and before the actual data file is created)

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  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.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation requests
+   * @param fileIndex file index to use to write markers
+   */
+  public void processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> pendingMarkerCreationFutures, int fileIndex) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      markFileAsAvailable(fileIndex);
+      return;
+    }
+
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath
+        + " numRequests=" + pendingMarkerCreationFutures.size() + " 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);
+    markFileAsAvailable(fileIndex);
+
+    for (MarkerCreationCompletableFuture future : pendingMarkerCreationFutures) {
+      try {
+        future.complete(jsonifyResult(
+            future.getContext(), future.getResult(), metricsRegistry, OBJECT_MAPPER, LOG));
+      } catch (JsonProcessingException e) {
+        throw new HoodieException("Failed to JSON encode the value", e);
+      }
+    }
+  }
+
+  /**
+   * 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 (lazyInitLock) {
+      if (!lazyInitComplete) {
+        syncMarkersFromFileSystem();
+        lazyInitComplete = true;
+      }
+    }
+  }
+
+  /**
+   * Syncs all markers maintained in the underlying files under the marker directory in the file system.
+   */
+  private void syncMarkersFromFileSystem() {
+    Path dirPath = new Path(markerDirPath);
+    try {
+      if (fileSystem.exists(dirPath)) {
+        FileStatus[] fileStatuses = fileSystem.listStatus(dirPath);
+        List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
+            .map(fileStatus -> fileStatus.getPath().toString())
+            .filter(pathStr -> pathStr.contains(MARKERS_FILENAME_PREFIX))
+            .collect(Collectors.toList());
+
+        if (markerDirSubPaths.size() > 0) {
+          SerializableConfiguration conf = new SerializableConfiguration(fileSystem.getConf());
+          int actualParallelism = Math.min(markerDirSubPaths.size(), parallelism);
+          Map<String, Set<String>> fileMarkersSetMap =
+              hoodieEngineContext.mapToPair(markerDirSubPaths, markersFilePathStr -> {
+                Path markersFilePath = new Path(markersFilePathStr);
+                FileSystem fileSystem = markersFilePath.getFileSystem(conf.get());
+                FSDataInputStream fsDataInputStream = null;
+                BufferedReader bufferedReader = null;
+                Set<String> markers = new HashSet<>();
+                try {
+                  LOG.debug("Read marker file: " + markersFilePathStr);
+                  fsDataInputStream = fileSystem.open(markersFilePath);
+                  bufferedReader = new BufferedReader(new InputStreamReader(fsDataInputStream, StandardCharsets.UTF_8));
+                  markers = bufferedReader.lines().collect(Collectors.toSet());
+                  bufferedReader.close();
+                  fsDataInputStream.close();
+                } catch (IOException e) {
+                  throw new HoodieIOException("Failed to read MARKERS file " + markerDirPath, e);
+                } finally {
+                  closeQuietly(bufferedReader);
+                  closeQuietly(fsDataInputStream);
+                }
+                return new ImmutablePair<>(markersFilePathStr, markers);
+              }, actualParallelism);
+
+          for (String markersFilePathStr: fileMarkersSetMap.keySet()) {
+            Set<String> fileMarkers = fileMarkersSetMap.get(markersFilePathStr);
+            if (!fileMarkers.isEmpty()) {
+              int index = parseMarkerFileIndex(markersFilePathStr);
+
+              if (index >= 0) {
+                fileMarkersMap.put(index, new StringBuilder(StringUtils.join(",", fileMarkers)));
+                allMarkers.addAll(fileMarkers);
+              }
+            }
+          }
+        }
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+  }
+
+  /**
+   * Parses the marker file index from the marker file path.
+   *
+   * E.g., if the marker file path is /tmp/table/.hoodie/.temp/000/MARKERS3, the index returned is 3.
+   *
+   * @param markerFilePathStr full path of marker file
+   * @return the marker file index
+   */
+  private int parseMarkerFileIndex(String markerFilePathStr) {
+    String markerFileName = new Path(markerFilePathStr).getName();
+    int prefixIndex = markerFileName.indexOf(MARKERS_FILENAME_PREFIX);
+    if (prefixIndex < 0) {
+      return -1;
+    }
+    try {
+      return Integer.parseInt(markerFileName.substring(prefixIndex + MARKERS_FILENAME_PREFIX.length()));
+    } catch (NumberFormatException nfe) {
+      LOG.error("Failed to parse marker file index from " + markerFilePathStr);
+      throw new HoodieException(nfe.getMessage(), nfe);
+    }
+  }
+
+  /**
+   * Flushes markers to the underlying file.
+   *
+   * @param markerFileIndex  file index to use.
+   */
+  private void flushMarkersToFile(int markerFileIndex) {
+    LOG.debug("Write to " + markerDirPath + "/" + MARKERS_FILENAME_PREFIX + markerFileIndex);
+    HoodieTimer timer = new HoodieTimer().startTimer();
+    Path markersFilePath = new Path(markerDirPath, MARKERS_FILENAME_PREFIX + markerFileIndex);
+    Path dirPath = markersFilePath.getParent();
+    try {
+      if (!fileSystem.exists(dirPath)) {
+        fileSystem.mkdirs(dirPath);
+      }
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to make dir " + dirPath, e);
+    }
+    FSDataOutputStream fsDataOutputStream = null;
+    BufferedWriter bufferedWriter = null;
+    try {
+      fsDataOutputStream = fileSystem.create(markersFilePath);
+      bufferedWriter = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8));
+      bufferedWriter.write(fileMarkersMap.get(markerFileIndex).toString());

Review comment:
       reg whether this works on HDFS. I think this write can fail midway, right?  then we would n't be able to recover this partial file read? (or all the marker creations will timeout, causing new requests to be made and this marker file will be overwritten?) 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  // Marker directory
+  private final String markerDirPath;
+  private final FileSystem fileSystem;
+  private final Registry metricsRegistry;
+  // 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 lazyInitLock = 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 lastFileIndexUsed = 0;
+  private boolean lazyInitComplete;
+
+  public MarkerDirState(String markerDirPath, int markerBatchNumThreads, FileSystem fileSystem,
+                        Registry metricsRegistry, HoodieEngineContext hoodieEngineContext, int parallelism) {
+    this.markerDirPath = markerDirPath;
+    this.fileSystem = fileSystem;
+    this.metricsRegistry = metricsRegistry;
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.parallelism = parallelism;
+    this.threadUseStatus =
+        Stream.generate(() -> false).limit(markerBatchNumThreads).collect(Collectors.toList());
+    this.lazyInitComplete = false;
+  }
+
+  /**
+   * @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 the next file index to use in a round-robin fashion,
+   * or -1 if no file is available.
+   */
+  public int getNextFileIndexToUse() {
+    int fileIndex = -1;
+    synchronized (threadUseStatus) {
+      int nextIndex = (lastFileIndexUsed + 1) % threadUseStatus.size();
+      if (!threadUseStatus.get(nextIndex)) {
+        fileIndex = nextIndex;
+        threadUseStatus.set(nextIndex, true);
+      } else {
+        for (int i = 1; i < threadUseStatus.size(); i++) {
+          int index = (lastFileIndexUsed + 1 + i) % threadUseStatus.size();
+          if (!threadUseStatus.get(index)) {
+            fileIndex = index;
+            threadUseStatus.set(index, true);
+            break;
+          }
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndexUsed = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file as available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAsAvailable(int fileIndex) {
+    synchronized (threadUseStatus) {
+      threadUseStatus.set(fileIndex, false);
+    }
+  }
+
+  /**
+   * @return  futures of pending marker creation requests and removes them from the list.
+   */
+  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.
+   *
+   * @param pendingMarkerCreationFutures futures of pending marker creation requests
+   * @param fileIndex file index to use to write markers
+   */
+  public void processMarkerCreationRequests(
+      final List<MarkerCreationCompletableFuture> pendingMarkerCreationFutures, int fileIndex) {
+    if (pendingMarkerCreationFutures.isEmpty()) {
+      markFileAsAvailable(fileIndex);
+      return;
+    }
+
+    LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" + markerDirPath
+        + " numRequests=" + pendingMarkerCreationFutures.size() + " 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);
+    markFileAsAvailable(fileIndex);
+
+    for (MarkerCreationCompletableFuture future : pendingMarkerCreationFutures) {
+      try {
+        future.complete(jsonifyResult(
+            future.getContext(), future.getResult(), metricsRegistry, OBJECT_MAPPER, LOG));
+      } catch (JsonProcessingException e) {
+        throw new HoodieException("Failed to JSON encode the value", e);
+      }
+    }
+  }
+
+  /**
+   * 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 (lazyInitLock) {
+      if (!lazyInitComplete) {
+        syncMarkersFromFileSystem();
+        lazyInitComplete = true;
+      }
+    }
+  }
+
+  /**
+   * Syncs all markers maintained in the underlying files under the marker directory in the file system.
+   */
+  private void syncMarkersFromFileSystem() {
+    Path dirPath = new Path(markerDirPath);
+    try {
+      if (fileSystem.exists(dirPath)) {
+        FileStatus[] fileStatuses = fileSystem.listStatus(dirPath);
+        List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
+            .map(fileStatus -> fileStatus.getPath().toString())
+            .filter(pathStr -> pathStr.contains(MARKERS_FILENAME_PREFIX))
+            .collect(Collectors.toList());
+
+        if (markerDirSubPaths.size() > 0) {
+          SerializableConfiguration conf = new SerializableConfiguration(fileSystem.getConf());
+          int actualParallelism = Math.min(markerDirSubPaths.size(), parallelism);
+          Map<String, Set<String>> fileMarkersSetMap =
+              hoodieEngineContext.mapToPair(markerDirSubPaths, markersFilePathStr -> {
+                Path markersFilePath = new Path(markersFilePathStr);
+                FileSystem fileSystem = markersFilePath.getFileSystem(conf.get());
+                FSDataInputStream fsDataInputStream = null;
+                BufferedReader bufferedReader = null;
+                Set<String> markers = new HashSet<>();
+                try {
+                  LOG.debug("Read marker file: " + markersFilePathStr);
+                  fsDataInputStream = fileSystem.open(markersFilePath);
+                  bufferedReader = new BufferedReader(new InputStreamReader(fsDataInputStream, StandardCharsets.UTF_8));
+                  markers = bufferedReader.lines().collect(Collectors.toSet());
+                  bufferedReader.close();
+                  fsDataInputStream.close();
+                } catch (IOException e) {
+                  throw new HoodieIOException("Failed to read MARKERS file " + markerDirPath, e);
+                } finally {
+                  closeQuietly(bufferedReader);
+                  closeQuietly(fsDataInputStream);
+                }
+                return new ImmutablePair<>(markersFilePathStr, markers);
+              }, actualParallelism);
+
+          for (String markersFilePathStr: fileMarkersSetMap.keySet()) {
+            Set<String> fileMarkers = fileMarkersSetMap.get(markersFilePathStr);
+            if (!fileMarkers.isEmpty()) {
+              int index = parseMarkerFileIndex(markersFilePathStr);
+
+              if (index >= 0) {
+                fileMarkersMap.put(index, new StringBuilder(StringUtils.join(",", fileMarkers)));
+                allMarkers.addAll(fileMarkers);
+              }
+            }
+          }
+        }
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+  }
+
+  /**
+   * Parses the marker file index from the marker file path.
+   *
+   * E.g., if the marker file path is /tmp/table/.hoodie/.temp/000/MARKERS3, the index returned is 3.
+   *
+   * @param markerFilePathStr full path of marker file
+   * @return the marker file index
+   */
+  private int parseMarkerFileIndex(String markerFilePathStr) {

Review comment:
       Can we handle the cases where the number of marker files is different from what's configured now? e.g `markerBatchNumThreads` was 10 before and now it's only 8. 




-- 
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-unsubscribe@hudi.apache.org

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