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

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_r680716767



##########
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<String, MarkerDirState> 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<String, MarkerDirState> 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<String, List<MarkerCreationCompletableFuture>> 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:
       Per offline discussion, I changed the scheduling logic into two parts: 
(1) dispatching of requests in one scheduled executor service which pulls 
requests and starts a worker thread periodically, and (2) the actual processing 
of requests in the worker runnable, where all the marker creation processing 
and file updates happen.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


> 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