[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

2021-08-05 Thread GitBox


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



##
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##
@@ -601,4 +604,84 @@ public static HoodieWrapperFileSystem getFs(String path, 
SerializableConfigurati
 .filter(fileStatus -> 
!fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
 .collect(Collectors.toList());
   }
+
+  /**
+   * Deletes a directory by deleting sub-paths in parallel on the file system.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @return {@code true} if the directory is delete; {@code false} otherwise.
+   */
+  public static boolean deleteDir(
+  HoodieEngineContext hoodieEngineContext, FileSystem fs, Path dirPath, 
int parallelism) {
+try {
+  if (fs.exists(dirPath)) {
+FSUtils.parallelizeSubPathProcess(hoodieEngineContext, fs, dirPath, 
parallelism, e -> true,
+pairOfSubPathAndConf -> 
deleteSubPath(pairOfSubPathAndConf.getKey(), pairOfSubPathAndConf.getValue())
+);
+
+boolean result = fs.delete(dirPath, true);
+LOG.info("Removed directory at " + dirPath);
+return result;
+  }
+} catch (IOException ioe) {
+  throw new HoodieIOException(ioe.getMessage(), ioe);
+}
+return false;
+  }
+
+  /**
+   * Processes sub-path in parallel.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @param subPathPredicate predicate to use to filter sub-paths for 
processing
+   * @param pairFunction actual processing logic for each sub-path
+   * @param  type of result to return for each sub-path
+   * @return a map of sub-path to result of the processing
+   */
+  public static  Map parallelizeSubPathProcess(

Review comment:
   looks neat :) 

##
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##
@@ -601,4 +604,84 @@ public static HoodieWrapperFileSystem getFs(String path, 
SerializableConfigurati
 .filter(fileStatus -> 
!fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
 .collect(Collectors.toList());
   }
+
+  /**
+   * Deletes a directory by deleting sub-paths in parallel on the file system.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @return {@code true} if the directory is delete; {@code false} otherwise.
+   */
+  public static boolean deleteDir(
+  HoodieEngineContext hoodieEngineContext, FileSystem fs, Path dirPath, 
int parallelism) {
+try {
+  if (fs.exists(dirPath)) {
+FSUtils.parallelizeSubPathProcess(hoodieEngineContext, fs, dirPath, 
parallelism, e -> true,
+pairOfSubPathAndConf -> 
deleteSubPath(pairOfSubPathAndConf.getKey(), pairOfSubPathAndConf.getValue())
+);
+
+boolean result = fs.delete(dirPath, true);
+LOG.info("Removed directory at " + dirPath);
+return result;
+  }
+} catch (IOException ioe) {
+  throw new HoodieIOException(ioe.getMessage(), ioe);
+}
+return false;
+  }
+
+  /**
+   * Processes sub-path in parallel.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @param subPathPredicate predicate to use to filter sub-paths for 
processing
+   * @param pairFunction actual processing logic for each sub-path
+   * @param  type of result to return for each sub-path
+   * @return a map of sub-path to result of the processing
+   */
+  public static  Map parallelizeSubPathProcess(

Review comment:
   Do we have unit tests for these newly added methods. Other devs might 
possible start using it in future. So, lets ensure we have good test coverage. 

##
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##
@@ -601,4 +604,84 @@ public static HoodieWrapperFileSystem getFs(String path, 
SerializableConfigurati
 .filter(fileStatus -> 
!fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
 .collect(Collectors.toList());
   }
+
+  /**
+   * Deletes a directory by deleting sub-paths in parallel on the file system.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @return {@code true} if the directory is delete; 

[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

2021-08-04 Thread GitBox


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



##
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##
@@ -601,4 +604,84 @@ public static HoodieWrapperFileSystem getFs(String path, 
SerializableConfigurati
 .filter(fileStatus -> 
!fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
 .collect(Collectors.toList());
   }
+
+  /**
+   * Deletes a directory by deleting sub-paths in parallel on the file system.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @return {@code true} if the directory is delete; {@code false} otherwise.
+   */
+  public static boolean deleteDir(
+  HoodieEngineContext hoodieEngineContext, FileSystem fs, Path dirPath, 
int parallelism) {
+try {
+  if (fs.exists(dirPath)) {
+FSUtils.parallelizeSubPathProcess(hoodieEngineContext, fs, dirPath, 
parallelism, e -> true,
+pairOfSubPathAndConf -> 
deleteSubPath(pairOfSubPathAndConf.getKey(), pairOfSubPathAndConf.getValue())
+);
+
+boolean result = fs.delete(dirPath, true);
+LOG.info("Removed directory at " + dirPath);
+return result;
+  }
+} catch (IOException ioe) {
+  throw new HoodieIOException(ioe.getMessage(), ioe);
+}
+return false;
+  }
+
+  /**
+   * Processes sub-path in parallel.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @param subPathPredicate predicate to use to filter sub-paths for 
processing
+   * @param pairFunction actual processing logic for each sub-path
+   * @param  type of result to return for each sub-path
+   * @return a map of sub-path to result of the processing
+   */
+  public static  Map parallelizeSubPathProcess(

Review comment:
   looks neat :) 

##
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##
@@ -601,4 +604,84 @@ public static HoodieWrapperFileSystem getFs(String path, 
SerializableConfigurati
 .filter(fileStatus -> 
!fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
 .collect(Collectors.toList());
   }
+
+  /**
+   * Deletes a directory by deleting sub-paths in parallel on the file system.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @return {@code true} if the directory is delete; {@code false} otherwise.
+   */
+  public static boolean deleteDir(
+  HoodieEngineContext hoodieEngineContext, FileSystem fs, Path dirPath, 
int parallelism) {
+try {
+  if (fs.exists(dirPath)) {
+FSUtils.parallelizeSubPathProcess(hoodieEngineContext, fs, dirPath, 
parallelism, e -> true,
+pairOfSubPathAndConf -> 
deleteSubPath(pairOfSubPathAndConf.getKey(), pairOfSubPathAndConf.getValue())
+);
+
+boolean result = fs.delete(dirPath, true);
+LOG.info("Removed directory at " + dirPath);
+return result;
+  }
+} catch (IOException ioe) {
+  throw new HoodieIOException(ioe.getMessage(), ioe);
+}
+return false;
+  }
+
+  /**
+   * Processes sub-path in parallel.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @param subPathPredicate predicate to use to filter sub-paths for 
processing
+   * @param pairFunction actual processing logic for each sub-path
+   * @param  type of result to return for each sub-path
+   * @return a map of sub-path to result of the processing
+   */
+  public static  Map parallelizeSubPathProcess(

Review comment:
   Do we have unit tests for these newly added methods. Other devs might 
possible start using it in future. So, lets ensure we have good test coverage. 

##
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##
@@ -601,4 +604,84 @@ public static HoodieWrapperFileSystem getFs(String path, 
SerializableConfigurati
 .filter(fileStatus -> 
!fileStatus.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))
 .collect(Collectors.toList());
   }
+
+  /**
+   * Deletes a directory by deleting sub-paths in parallel on the file system.
+   *
+   * @param hoodieEngineContext {@code HoodieEngineContext} instance
+   * @param fs file system
+   * @param dirPath directory path
+   * @param parallelism parallelism to use for sub-paths
+   * @return {@code true} if the directory is delete; 

[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

2021-08-03 Thread GitBox


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



##
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java
##
@@ -0,0 +1,189 @@
+/*
+ * 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 markerDirStateMap = new 
HashMap<>();
+  // A long-running 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");
+

[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

2021-08-02 Thread GitBox


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 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();

[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

2021-07-31 Thread GitBox


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



##
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##
@@ -0,0 +1,79 @@
+/*
+ * 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.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+
+/**
+ * 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 markerDirStateMap;
+  private final Registry metricsRegistry;
+  private final ExecutorService executorService;
+  // Batch process interval in milliseconds
+  private final long batchIntervalMs;
+  private boolean isRunning = false;
+
+  public MarkerCreationDispatchingRunnable(
+  Map markerDirStateMap, Registry metricsRegistry,
+  int batchNumThreads, long batchIntervalMs) {
+this.markerDirStateMap = markerDirStateMap;
+this.metricsRegistry = metricsRegistry;
+this.batchIntervalMs = batchIntervalMs;
+this.executorService = Executors.newFixedThreadPool(batchNumThreads);
+this.isRunning = true;
+  }
+
+  public void stop() {
+this.isRunning = false;
+  }
+
+  @Override
+  public void run() {
+while (isRunning) {
+  HoodieTimer timer = new HoodieTimer().startTimer();
+  Map> futureMap =
+  markerDirStateMap.entrySet().stream().collect(
+  Collectors.toMap(Map.Entry::getKey,
+  e -> e.getValue().fetchPendingMarkerCreationRequests()));
+  executorService.execute(
+  new MarkerCreationBatchingRunnable(markerDirStateMap, 
metricsRegistry, futureMap));
+
+  try {
+Thread.sleep(Math.max(batchIntervalMs - timer.endTimer(), 0L));

Review comment:
   @vinothchandar :one nit question. Whats the elegant way to do something 
every 50 ms? 
   Here we are doing thread.sleep in a while loop in this dedicated dispatch 
thread. Is there any other better ways to do it. 
   Guess ethan doesn't want to spin up a new thread every 50 ms and hence went 
with one dedicated thread. Even if we go with wait and notify, some other 
thread or caller should call into notify every 50 ms. We can't rely on new 
create requests bcoz liveness is not guaranteed. We can't do polling no queue, 
since any new create request will wake that up. 
   
   




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




[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

2021-07-31 Thread GitBox


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



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

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

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

Review comment:
   fix variable names to markerType. 

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

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

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

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

##
File path: 
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import