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/02 20:12:57 UTC

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

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



##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.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 executorService;
+  // Parallelism for reading and deleting marker files
+  private final int parallelism;
+  // Marker directory states, {markerDirPath -> MarkerDirState instance}
+  private final Map<String, MarkerDirState> markerDirStateMap = new HashMap<>();
+  // A long-running thread to dispatch marker creation requests to batch processing threads
+  private final MarkerCreationDispatchingRunnable markerCreationDispatchingRunnable;
+  private final AtomicBoolean firstMarkerCreationRequest;
+  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.executorService = Executors.newSingleThreadScheduledExecutor();
+    this.markerCreationDispatchingRunnable = new MarkerCreationDispatchingRunnable(
+        markerDirStateMap, timelineServiceConfig.markerBatchNumThreads);
+    this.firstMarkerCreationRequest = new AtomicBoolean(true);

Review comment:
       firstRequestSeen. And let's init to false. Logically that would make sense.

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.concurrent.atomic.AtomicBoolean;
+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 AtomicBoolean lazyInitComplete;
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+
+  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 = new AtomicBoolean(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) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {
+        int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+        if (!threadUseStatus.get(index)) {
+          fileIndex = index;
+          threadUseStatus.set(index, true);
+          break;
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndex = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAvailable(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()) {
+      markFileAvailable(fileIndex);
+    }
+
+    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);
+    markFileAvailable(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() {
+    if (!lazyInitComplete.getAndSet(true)) {

Review comment:
       Don't think we can use AtomicBoolean here. For eg, lets say there are two concurrent requests, one among them is getAllMarkers(). And getAllMarkers() reaches this method later while the other thread is executing syncMarkersFromFileSystem. So, getAllMarkers() will skip the if block and will proceed on to return allMarkers. But until the other thread is complete executing syncMarkersFromFileSystem, no requests should proceed. 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java
##########
@@ -228,7 +342,9 @@ public static void main(String[] args) throws Exception {
 
     Configuration conf = FSUtils.prepareHadoopConf(new Configuration());
     FileSystemViewManager viewManager = buildFileSystemViewManager(cfg, new SerializableConfiguration(conf));
-    TimelineService service = new TimelineService(cfg.serverPort, viewManager);
+    TimelineService service = new TimelineService(
+        new HoodieLocalEngineContext(FSUtils.prepareHadoopConf(new Configuration())),

Review comment:
       I see we are doing new Configuration() thrice. Can we do it once and re-use. Or does each need to be separate copy? 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable 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, int batchNumThreads) {
+    this.markerDirStateMap = markerDirStateMap;
+    this.executorService = Executors.newFixedThreadPool(batchNumThreads);

Review comment:
       I guess we should move this outside of this runnable. We create new instances of this class every 50 ms. Don't think we need to keep instantiating new executorService for worker threads.

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirRequestContext.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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 {
+  // List of marker creation futures to process
+  private final List<MarkerCreationCompletableFuture> futures;

Review comment:
       Not sure if we really need a class just for 2 variables. I would prefer to use Pair<List, index>. Let's see if vinoth has any suggestions. 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable 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, int batchNumThreads) {
+    this.markerDirStateMap = markerDirStateMap;
+    this.executorService = Executors.newFixedThreadPool(batchNumThreads);
+  }
+
+  @Override
+  public void run() {
+    Map<String, MarkerDirRequestContext> requestContextMap = new HashMap<>();
+

Review comment:
       Can we add java docs as to what we are doing in this method. Something like 
   1. find next available maker file index
   2. Fetch pending marker create requests
   3. spin up a worker thread and assign these to it. Worker thread will be responsible to respond to futures directly. 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable for scheduling batch processing of marker creation requests.
+ */
+public class MarkerCreationDispatchingRunnable implements Runnable {

Review comment:
       Should we add a stop() method to this runnable and shutdown executorService? 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.concurrent.atomic.AtomicBoolean;
+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 AtomicBoolean lazyInitComplete;
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+
+  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 = new AtomicBoolean(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) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {
+        int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+        if (!threadUseStatus.get(index)) {
+          fileIndex = index;
+          threadUseStatus.set(index, true);
+          break;
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndex = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAvailable(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()) {
+      markFileAvailable(fileIndex);

Review comment:
       shouldn't we return here?

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.concurrent.atomic.AtomicBoolean;
+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 AtomicBoolean lazyInitComplete;
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+
+  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 = new AtomicBoolean(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) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {
+        int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+        if (!threadUseStatus.get(index)) {
+          fileIndex = index;
+          threadUseStatus.set(index, true);
+          break;
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndex = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAvailable(int fileIndex) {

Review comment:
       nit: markFileAsAvailable

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.concurrent.atomic.AtomicBoolean;
+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 AtomicBoolean lazyInitComplete;
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+
+  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 = new AtomicBoolean(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) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {
+        int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+        if (!threadUseStatus.get(index)) {
+          fileIndex = index;
+          threadUseStatus.set(index, true);
+          break;
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndex = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAvailable(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()) {
+      markFileAvailable(fileIndex);

Review comment:
       also, I guess we can't mark file as available here. markFileAsAvailable should be called from within MarkerCreationBatchingRunnable.run() towards the end once all entries are processed. 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable for scheduling batch processing of marker creation requests.
+ */
+public class MarkerCreationDispatchingRunnable implements Runnable {

Review comment:
       or within interruptedException catch block within run() method.

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.concurrent.atomic.AtomicBoolean;
+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 AtomicBoolean lazyInitComplete;
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+
+  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 = new AtomicBoolean(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) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {
+        int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+        if (!threadUseStatus.get(index)) {
+          fileIndex = index;
+          threadUseStatus.set(index, true);
+          break;
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndex = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAvailable(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()) {
+      markFileAvailable(fileIndex);
+    }
+
+    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);
+    markFileAvailable(fileIndex);

Review comment:
       again, don't think we can make it as available here. 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationBatchingRunnable.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.Map;
+
+/**
+ * A runnable for batch processing marker creation requests.
+ */
+public class MarkerCreationBatchingRunnable implements Runnable {
+  private static final Logger LOG = LogManager.getLogger(MarkerCreationBatchingRunnable.class);
+
+  private final Map<String, MarkerDirState> markerDirStateMap;
+  private final Map<String, MarkerDirRequestContext> requestContextMap;
+
+  public MarkerCreationBatchingRunnable(
+      Map<String, MarkerDirState> markerDirStateMap,
+      Map<String, MarkerDirRequestContext> requestContextMap) {
+    this.markerDirStateMap = markerDirStateMap;
+    this.requestContextMap = requestContextMap;
+  }
+
+  @Override
+  public void run() {
+    LOG.debug("Start processing create marker requests");
+    HoodieTimer timer = new HoodieTimer().startTimer();
+
+    for (String markerDir : requestContextMap.keySet()) {
+      MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
+
+      if (markerDirState == null) {
+        LOG.error("MarkerDirState of " + markerDir + " does not exist!");

Review comment:
       shouldn't we throw here. What happens to requests given to this thread? 

##########
File path: hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.concurrent.atomic.AtomicBoolean;
+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 AtomicBoolean lazyInitComplete;
+  private final Object markerCreationProcessingLock = new Object();
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Last underlying file index used, for finding the next file index
+  // in a round-robin fashion
+  private int lastFileIndex = 0;
+
+  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 = new AtomicBoolean(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) {
+      for (int i = 0; i < threadUseStatus.size(); i++) {
+        int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+        if (!threadUseStatus.get(index)) {
+          fileIndex = index;
+          threadUseStatus.set(index, true);
+          break;
+        }
+      }
+      if (fileIndex >= 0) {
+        lastFileIndex = fileIndex;
+      }
+    }
+    return fileIndex;
+  }
+
+  /**
+   * Marks the file available to use again.
+   *
+   * @param fileIndex file index
+   */
+  public void markFileAvailable(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()) {
+      markFileAvailable(fileIndex);
+    }
+
+    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);
+    markFileAvailable(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() {
+    if (!lazyInitComplete.getAndSet(true)) {

Review comment:
       may I was the one who proposed. sorry about the mis-guidance. 




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