jihoonson commented on a change in pull request #5492: Native parallel batch 
indexing without shuffle
URL: https://github.com/apache/incubator-druid/pull/5492#discussion_r201892100
 
 

 ##########
 File path: 
indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java
 ##########
 @@ -0,0 +1,502 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import io.druid.client.indexing.IndexingServiceClient;
+import io.druid.client.indexing.TaskStatusResponse;
+import io.druid.indexer.TaskState;
+import io.druid.indexer.TaskStatusPlus;
+import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.concurrent.Execs;
+import io.druid.java.util.common.logger.Logger;
+
+import javax.annotation.Nullable;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Responsible for submitting tasks, monitoring task statuses, resubmitting 
failed tasks, and returning the final task
+ * status.
+ */
+public class TaskMonitor<T extends Task>
+{
+  private static final Logger log = new Logger(TaskMonitor.class);
+
+  private final ScheduledExecutorService taskStatusChecker = 
Execs.scheduledSingleThreaded(("task-monitor-%d"));
+
+  /**
+   * A map of subTaskSpecId to {@link MonitorEntry}. This map stores the state 
of running {@link SubTaskSpec}s. This is
+   * read in {@link java.util.concurrent.Callable} executed by {@link 
#taskStatusChecker} and updated in {@link #submit}
+   * and {@link #retry}. This can also be read by calling {@link 
#getRunningTaskMonitorEntory},
+   * {@link #getRunningTaskIds}, and {@link #getRunningSubTaskSpecs}.
+   */
+  private final ConcurrentMap<String, MonitorEntry> runningTasks = new 
ConcurrentHashMap<>();
+
+  /**
+   * A map of subTaskSpecId to {@link TaskHistory}. This map stores the 
history of complete {@link SubTaskSpec}s
+   * whether their final state is succeeded or failed. This is updated in 
{@link MonitorEntry#setLastStatus} which is
+   * called by the {@link java.util.concurrent.Callable} executed by {@link 
#taskStatusChecker} and can be
+   * read by outside of this class.
+   */
+  private final ConcurrentMap<String, TaskHistory<T>> taskHistories = new 
ConcurrentHashMap<>();
+
+  // lock for updating numRunningTasks, numSucceededTasks, and numFailedTasks
+  private final Object taskCountLock = new Object();
+
+  // overlord client
+  private final IndexingServiceClient indexingServiceClient;
+  private final int maxRetry;
+  private final int expectedNumSucceededTasks;
+
+  private int numRunningTasks;
+  private int numSucceededTasks;
+  private int numFailedTasks;
+
+  private volatile boolean running = false;
+
+  TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry, int 
expectedNumSucceededTasks)
+  {
+    this.indexingServiceClient = 
Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient");
+    this.maxRetry = maxRetry;
+    this.expectedNumSucceededTasks = expectedNumSucceededTasks;
+
+    log.info("TaskMonitor is initialized with expectedNumSucceededTasks[%d]", 
expectedNumSucceededTasks);
+  }
+
+  public void start(long taskStatusCheckingPeriod)
+  {
+    running = true;
+    log.info("Starting taskMonitor");
+    // NOTE: This polling can be improved to event-driven pushing by 
registering TaskRunnerListener to TaskRunner.
+    // That listener should be able to send the events reported to TaskRunner 
to this TaskMonitor.
+    taskStatusChecker.scheduleAtFixedRate(
+        () -> {
+          try {
+            final Iterator<Entry<String, MonitorEntry>> iterator = 
runningTasks.entrySet().iterator();
+            while (iterator.hasNext()) {
+              final Entry<String, MonitorEntry> entry = iterator.next();
+              final String specId = entry.getKey();
+              final MonitorEntry monitorEntry = entry.getValue();
+              final String taskId = monitorEntry.runningTask.getId();
+              final TaskStatusResponse taskStatusResponse = 
indexingServiceClient.getTaskStatus(taskId);
+              final TaskStatusPlus taskStatus = taskStatusResponse.getStatus();
+              if (taskStatus != null) {
+                switch (Preconditions.checkNotNull(taskStatus.getState(), 
"taskState")) {
+                  case SUCCESS:
+                    incrementNumSucceededTasks();
+
+                    // Remote the current entry after updating taskHistories 
to make sure that task history
+                    // exists either runningTasks or taskHistories.
+                    monitorEntry.setLastStatus(taskStatus);
+                    iterator.remove();
+                    break;
+                  case FAILED:
+                    incrementNumFailedTasks();
+
+                    log.warn("task[%s] failed!", taskId);
+                    if (monitorEntry.numTries() < maxRetry) {
+                      log.info(
+                          "We still have chances[%d/%d] to complete for 
spec[%s].",
+                          monitorEntry.numTries(),
+                          maxRetry,
+                          monitorEntry.spec.getId()
+                      );
+                      retry(specId, monitorEntry, taskStatus);
+                    } else {
+                      log.error(
+                          "spec[%s] failed after [%d] tries",
+                          monitorEntry.spec.getId(),
+                          monitorEntry.numTries()
+                      );
+                      // Remote the current entry after updating taskHistories 
to make sure that task history
+                      // exists either runningTasks or taskHistories.
+                      monitorEntry.setLastStatus(taskStatus);
+                      iterator.remove();
+                    }
+                    break;
+                  case RUNNING:
+                    monitorEntry.updateStatus(taskStatus);
+                    break;
+                  default:
+                    throw new ISE("Unknown taskStatus[%s] for task[%s[", 
taskStatus.getState(), taskId);
+                }
+              }
+            }
+          }
+          catch (Throwable t) {
+            log.error(t, "Error while monitoring");
+            throw t;
+          }
+        },
+        taskStatusCheckingPeriod,
+        taskStatusCheckingPeriod,
+        TimeUnit.MILLISECONDS
+    );
+  }
+
+  public void stop()
+  {
+    running = false;
+    taskStatusChecker.shutdownNow();
+    log.info("Stopped taskMonitor");
+  }
+
+  public ListenableFuture<SubTaskCompleteEvent<T>> submit(SubTaskSpec<T> spec)
+  {
+    if (!running) {
+      return Futures.immediateFailedFuture(new ISE("TaskMonitore is not 
running"));
+    }
+    final T task = spec.newSubTask(0);
+    log.info("Submitting a new task[%s] for spec[%s]", task.getId(), 
spec.getId());
+    indexingServiceClient.runTask(task);
+    incrementNumRunningTasks();
+
+    final SettableFuture<SubTaskCompleteEvent<T>> taskFuture = 
SettableFuture.create();
+    runningTasks.put(
+        spec.getId(),
+        new MonitorEntry(spec, task, 
indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture)
+    );
+
+    return taskFuture;
+  }
+
+  /**
+   * Submit a retry task for a failed spec. This method should be called 
inside of the
+   * {@link java.util.concurrent.Callable} executed by {@link 
#taskStatusChecker}.
+   */
+  private void retry(String subTaskSpecId, MonitorEntry monitorEntry, 
TaskStatusPlus lastFailedTaskStatus)
+  {
+    if (running) {
+      final SubTaskSpec<T> spec = monitorEntry.spec;
+      final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1);
+      log.info("Submitting a new task[%s] for retrying spec[%s]", 
task.getId(), spec.getId());
+      indexingServiceClient.runTask(task);
+      incrementNumRunningTasks();
+
+      runningTasks.put(
+          subTaskSpecId,
+          monitorEntry.withNewRunningTask(
+              task,
+              indexingServiceClient.getTaskStatus(task.getId()).getStatus(),
+              lastFailedTaskStatus
+          )
+      );
+    }
+  }
+
+  /**
+   * This method should be called after {@link #stop()} to make sure no 
additional tasks are submitted.
+   */
+  void killAll()
+  {
+    runningTasks.values().forEach(entry -> {
+      final String taskId = entry.runningTask.getId();
+      log.info("Request to kill subtask[%s]", taskId);
+      indexingServiceClient.killTask(taskId);
+    });
+    runningTasks.clear();
+  }
+
+  void incrementNumRunningTasks()
+  {
+    synchronized (taskCountLock) {
+      numRunningTasks++;
+    }
+  }
+
+  void incrementNumSucceededTasks()
+  {
+    synchronized (taskCountLock) {
+      numRunningTasks--;
+      numSucceededTasks++;
+      log.info("[%d/%d] tasks succeeded", numSucceededTasks, 
expectedNumSucceededTasks);
+    }
+  }
+
+  void incrementNumFailedTasks()
+  {
+    synchronized (taskCountLock) {
+      numRunningTasks--;
+      numFailedTasks++;
+    }
+  }
+
+  boolean isSucceeded()
+  {
+    synchronized (taskCountLock) {
+      return numSucceededTasks == expectedNumSucceededTasks;
+    }
+  }
+
+  int getNumRunningTasks()
+  {
+    synchronized (taskCountLock) {
+      return numRunningTasks;
+    }
+  }
+
+  Status getStatus()
+  {
+    synchronized (taskCountLock) {
+      return new Status(
+          numRunningTasks,
+          numSucceededTasks,
+          numFailedTasks,
+          numSucceededTasks + numFailedTasks,
+          numRunningTasks + numSucceededTasks + numFailedTasks,
+          expectedNumSucceededTasks
+      );
+    }
+  }
+
+  Set<String> getRunningTaskIds()
+  {
+    return runningTasks.values().stream().map(entry -> 
entry.runningTask.getId()).collect(Collectors.toSet());
+  }
+
+  List<SubTaskSpec<T>> getRunningSubTaskSpecs()
 
 Review comment:
   Same here. The return value should always be materialized.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscr...@druid.apache.org
For additional commands, e-mail: dev-h...@druid.apache.org

Reply via email to