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

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

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



##########
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java
##########
@@ -0,0 +1,484 @@
+/*
+ * 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.config.SerializableConfiguration;
+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.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.timeline.service.TimelineService;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.javalin.Context;
+import org.apache.hadoop.conf.Configuration;
+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.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * 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 {
+  public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private static final Logger LOG = LogManager.getLogger(MarkerHandler.class);
+  // Margin time for scheduling the processing of the next batch of marker 
creation requests
+  private static final long SCHEDULING_MARGIN_TIME_MS = 5L;
+
+  private final Registry metricsRegistry;
+  private final ScheduledExecutorService executorService;
+  // A cached copy of all markers in memory
+  // Mapping: {markerDirPath -> all markers}
+  private final Map<String, Set<String>> allMarkersMap = new HashMap<>();
+  // A cached copy of marker entries in each marker file, stored in 
StringBuilder for efficient appending
+  // Mapping: {markerDirPath -> {markerFileIndex -> markers}}
+  private final Map<String, Map<Integer, StringBuilder>> fileMarkersMap = new 
HashMap<>();
+  // A list of pending futures from async marker creation requests
+  private final List<CreateMarkerCompletableFuture> createMarkerFutures = new 
ArrayList<>();
+  // A list of use status of marker files. {@code true} means the file is in 
use by a {@code BatchCreateMarkerRunnable}
+  private final List<Boolean> markerFilesUseStatus;
+  // Batch process interval in milliseconds
+  private final long batchIntervalMs;
+  // Parallelism for reading and deleting marker files
+  private final int parallelism;
+  private transient HoodieEngineContext hoodieEngineContext;
+  // Lock for synchronous processing of marker creating requests
+  private volatile Object createMarkerRequestLockObject = new Object();
+  // Next batch process timestamp in milliseconds
+  private long nextBatchProcessTimeMs = 0L;
+  // Last marker file index used, for finding the next marker file index in a 
round-robin fashion
+  private int lastMarkerFileIndex = 0;
+
+  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 Params: batchNumThreads=" + 
timelineServiceConfig.markerBatchNumThreads
+        + " batchIntervalMs=" + timelineServiceConfig.markerBatchIntervalMs + 
"ms");
+    this.hoodieEngineContext = hoodieEngineContext;
+    this.metricsRegistry = metricsRegistry;
+    this.batchIntervalMs = timelineServiceConfig.markerBatchIntervalMs;
+    this.parallelism = timelineServiceConfig.markerParallelism;
+    this.executorService = 
Executors.newScheduledThreadPool(timelineServiceConfig.markerBatchNumThreads);
+
+    List<Boolean> isMarkerFileInUseList = new 
ArrayList<>(timelineServiceConfig.markerBatchNumThreads);
+    for (int i = 0; i < timelineServiceConfig.markerBatchNumThreads; i++) {
+      isMarkerFileInUseList.add(false);
+    }
+    this.markerFilesUseStatus = 
Collections.synchronizedList(isMarkerFileInUseList);
+  }
+
+  /**
+   * @param markerDirPath marker directory path
+   * @return all marker paths in the marker directory
+   */
+  public Set<String> getAllMarkers(String markerDirPath) {
+    return allMarkersMap.getOrDefault(markerDirPath, new HashSet<>());
+  }
+
+  /**
+   * @param markerDirPath marker directory path
+   * @return all marker paths of write IO type "CREATE" and "MERGE"
+   */
+  public Set<String> getCreateAndMergeMarkers(String markerDirPath) {
+    return allMarkersMap.getOrDefault(markerDirPath, new HashSet<>()).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) {
+    Path markerDirPath = new Path(markerDir);
+    try {
+      return fileSystem.exists(markerDirPath);
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+  }
+
+  /**
+   * 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 markerDirPath marker directory path
+   * @param markerName marker name
+   * @return the {@code CompletableFuture} instance for the request
+   */
+  public CompletableFuture<String> createMarker(Context context, String 
markerDirPath, String markerName) {
+    LOG.info("Request: create marker " + markerDirPath + " " + markerName);
+    CreateMarkerCompletableFuture future = new 
CreateMarkerCompletableFuture(context, markerDirPath, markerName);
+    synchronized (createMarkerRequestLockObject) {
+      // Add the future to the list
+      createMarkerFutures.add(future);
+      // Update the next batch processing time and schedule the batch 
processing if necessary
+      long currTimeMs = System.currentTimeMillis();
+      // If the current request may miss the next batch processing, schedule a 
new batch processing thread
+      // A margin time is always considered for checking the tiemstamp to make 
sure no request is missed
+      if (currTimeMs >= nextBatchProcessTimeMs - SCHEDULING_MARGIN_TIME_MS) {
+        if (currTimeMs < nextBatchProcessTimeMs + batchIntervalMs - 
SCHEDULING_MARGIN_TIME_MS) {
+          // within the batch interval from the latest batch processing thread
+          // increment nextBatchProcessTimeMs by batchIntervalMs
+          nextBatchProcessTimeMs += batchIntervalMs;
+        } else {
+          // Otherwise, wait for batchIntervalMs based on the current timestamp
+          nextBatchProcessTimeMs = currTimeMs + batchIntervalMs;
+        }
+
+        long waitMs = nextBatchProcessTimeMs - System.currentTimeMillis();
+        executorService.schedule(
+            new BatchCreateMarkerRunnable(), Math.max(0L, waitMs), 
TimeUnit.MILLISECONDS);
+        LOG.info("Wait for " + waitMs + " ms, next batch time: " + 
nextBatchProcessTimeMs);
+      }
+    }
+    return future;
+  }
+
+  /**
+   * Deletes markers in the directory.
+   *
+   * @param markerDir marker directory path
+   * @return {@code true} if successful; {@code false} otherwise.
+   */
+  public Boolean deleteMarkers(String markerDir) {
+    Path markerDirPath = new Path(markerDir);
+    boolean result = false;
+    try {
+      if (fileSystem.exists(markerDirPath)) {
+        FileStatus[] fileStatuses = fileSystem.listStatus(markerDirPath);
+        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(markerDirPath, true);
+        LOG.info("Removing marker directory at " + markerDirPath);
+      }
+      allMarkersMap.remove(markerDir);
+      fileMarkersMap.remove(markerDir);
+    } catch (IOException ioe) {
+      throw new HoodieIOException(ioe.getMessage(), ioe);
+    }
+    return result;
+  }
+
+  /**
+   * Gets 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 getMarkerFileIndex(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) {
+      nfe.printStackTrace();
+    }
+    return -1;
+  }
+
+  /**
+   * Syncs all markers maintained in the marker files from the marker 
directory in the file system.
+   *
+   * @param markerDir marker directory path
+   */
+  private void syncAllMarkersFromFile(String markerDir) {

Review comment:
       Fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

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


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



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

Reply via email to