jon-wei commented on a change in pull request #8107: Add CliIndexer process type and initial task runner implementation URL: https://github.com/apache/incubator-druid/pull/8107#discussion_r307948439
########## File path: indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java ########## @@ -0,0 +1,566 @@ +/* + * 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.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableSet; +import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; +import org.apache.commons.io.FileUtils; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexer.RunnerTaskState; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.server.DruidNode; +import org.apache.druid.tasklogs.TaskLogPusher; +import org.apache.druid.tasklogs.TaskLogStreamer; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +/** + * TaskRunner implemention for the CliIndexer task execution service, which runs all tasks in a single process. + * + * Two thread pools are used: + * - A task execution pool, sized to number of worker slots. This is used to execute the Task run() methods. + * - A control thread pool, sized to worker slots * 2. The control threads setup and submit work to the + * task execution pool, and are also responsible for running graceful shutdown on the Task objects. + * Only one shutdown per-task can be running at a given time, and there is one control thread per task, + * thus the pool has 2 * worker slots. + * + * Note that separate task logs are not supported, all task log entries will be written to the Indexer process log + * instead. + */ +public class ThreadingTaskRunner + extends BaseRestorableTaskRunner<ThreadingTaskRunner.ThreadingTaskRunnerWorkItem> + implements TaskLogStreamer, QuerySegmentWalker +{ + private static final EmittingLogger LOGGER = new EmittingLogger(ThreadingTaskRunner.class); + + private final TaskToolboxFactory toolboxFactory; + private final TaskLogPusher taskLogPusher; + private final DruidNode node; + private final AppenderatorsManager appenderatorsManager; + private final TaskReportFileWriter taskReportFileWriter; + private final ListeningExecutorService taskExecutor; + private final ListeningExecutorService controlThreadExecutor; + + private volatile boolean stopping = false; + + @Inject + public ThreadingTaskRunner( + TaskToolboxFactory toolboxFactory, + TaskConfig taskConfig, + WorkerConfig workerConfig, + TaskLogPusher taskLogPusher, + ObjectMapper jsonMapper, + AppenderatorsManager appenderatorsManager, + TaskReportFileWriter taskReportFileWriter, + @Self DruidNode node + ) + { + super(jsonMapper, taskConfig); + this.toolboxFactory = toolboxFactory; + this.taskLogPusher = taskLogPusher; + this.node = node; + this.appenderatorsManager = appenderatorsManager; + this.taskReportFileWriter = taskReportFileWriter; + this.taskExecutor = MoreExecutors.listeningDecorator( + Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-executor-%d") + ); + this.controlThreadExecutor = MoreExecutors.listeningDecorator( + Execs.multiThreaded(workerConfig.getCapacity() * 2, "threading-task-runner-control-%d") + ); + } + + @Override + public Optional<ByteSource> streamTaskLog(String taskid, long offset) throws IOException + { + // task logs will appear in the main indexer log, streaming individual task logs is not supported + return Optional.absent(); + } + + @Override + public void start() + { + // Nothing to start. + } + + @Override + public ListenableFuture<TaskStatus> run(Task task) + { + synchronized (tasks) { + tasks.computeIfAbsent( + task.getId(), k -> + new ThreadingTaskRunnerWorkItem( + task, + controlThreadExecutor.submit( + new Callable<TaskStatus>() { + @Override + public TaskStatus call() + { + final String attemptUUID = UUID.randomUUID().toString(); + final File taskDir = taskConfig.getTaskDir(task.getId()); + final File attemptDir = new File(taskDir, attemptUUID); + + final TaskLocation taskLocation = TaskLocation.create( + node.getHost(), + node.getPlaintextPort(), + node.getTlsPort() + ); + + try { + final Closer closer = Closer.create(); + try { + if (!attemptDir.mkdirs()) { + throw new IOE("Could not create directories: %s", attemptDir); + } + + final File taskFile = new File(taskDir, "task.json"); + final File reportsFile = new File(attemptDir, "report.json"); + taskReportFileWriter.add(task.getId(), reportsFile); + + final ThreadingTaskRunnerWorkItem taskWorkItem; + // time to adjust process holders + synchronized (tasks) { + taskWorkItem = tasks.get(task.getId()); + + if (taskWorkItem == null) { + LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); + } + + if (taskWorkItem.shutdown) { + throw new IllegalStateException("Task has been shut down!"); + } + } + + if (!taskFile.exists()) { + jsonMapper.writeValue(taskFile, task); + } + + // This will block for a while. So we append the thread information with more details + final String priorThreadName = Thread.currentThread().getName(); + Thread.currentThread().setName(StringUtils.format("%s-[%s]", priorThreadName, task.getId())); + + TaskStatus taskStatus = null; + final TaskToolbox toolbox = toolboxFactory.build(task); + try { + ListenableFuture<TaskStatus> taskStatusFuture = taskExecutor.submit( + new Callable<TaskStatus>() + { + @Override + public TaskStatus call() + { + taskWorkItem.setThread(Thread.currentThread()); + try { + return task.run(toolbox); + } + catch (Exception e) { + LOGGER.error(e, "Task[%s] exited with exception.", task.getId()); + return null; + } + finally { + taskWorkItem.setThread(null); + } + } + } + ); + TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); + TaskRunnerUtils.notifyStatusChanged( + listeners, + task.getId(), + TaskStatus.running(task.getId()) + ); + taskStatus = taskStatusFuture.get(); + } + finally { + taskWorkItem.setFinished(true); + Thread.currentThread().setName(priorThreadName); + if (reportsFile.exists()) { + taskLogPusher.pushTaskReports(task.getId(), reportsFile); + } + if (taskStatus == null) { + taskStatus = TaskStatus.failure(task.getId()); + } + } + + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus); + return taskStatus; + } + catch (Throwable t) { + throw closer.rethrow(t); + } + finally { + closer.close(); + } + } + catch (Throwable t) { + LOGGER.info(t, "Exception caught during execution"); + throw new RuntimeException(t); + } + finally { + taskReportFileWriter.delete(task.getId()); + appenderatorsManager.removeAppenderatorForTask(task.getId()); + + try { + synchronized (tasks) { + final ThreadingTaskRunnerWorkItem taskWorkItem = tasks.remove(task.getId()); + if (!stopping) { + saveRunningTasks(); + } + } + + try { + if (!stopping && taskDir.exists()) { + LOGGER.info("Removing task directory: %s", taskDir); + FileUtils.deleteDirectory(taskDir); + } + } + catch (Exception e) { + LOGGER.makeAlert(e, "Failed to delete task directory") + .addData("taskDir", taskDir.toString()) + .addData("task", task.getId()) + .emit(); + } + } + catch (Exception e) { + LOGGER.error(e, "Suppressing exception caught while cleaning up task"); + } + } + } + } + ) + ) + ); + saveRunningTasks(); + return tasks.get(task.getId()).getResult(); + } + } + + @Override + public void shutdown(String taskid, String reason) + { + LOGGER.info("Shutdown [%s] because: [%s]", taskid, reason); + final ThreadingTaskRunnerWorkItem taskInfo; + + synchronized (tasks) { + taskInfo = tasks.get(taskid); + + if (taskInfo == null) { + LOGGER.info("Ignoring request to cancel unknown task: %s", taskid); + return; + } + + if (taskInfo.shutdown) { + LOGGER.info( + "Task [%s] is already shutting down, ignoring duplicate shutdown request with reason [%s]", + taskid, + reason + ); + } else { + taskInfo.shutdown = true; + scheduleTaskShutdown(taskInfo); + } + } + } + + /** + * Submits a callable to the control thread pool that attempts a task graceful shutdown, + * if shutdown is not already scheduled. + * + * The shutdown will wait for the configured timeout and then interrupt the thread if the timeout is exceeded. + */ + private ListenableFuture scheduleTaskShutdown(ThreadingTaskRunnerWorkItem taskInfo) + { + synchronized (tasks) { + if (taskInfo.shutdownFuture != null) { + return taskInfo.shutdownFuture; + } + + ListenableFuture shutdownFuture = controlThreadExecutor.submit( + new Callable<Void>() + { + @Override + public Void call() + { + LOGGER.info("Stopping thread for task: %s", taskInfo.getTaskId()); + taskInfo.getTask().stopGracefully(taskConfig); + + try { + TaskStatus status = taskInfo.getResult().get( + taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis(), + TimeUnit.MILLISECONDS + ); + + if (status == null) { + if (taskInfo.thread != null) { + taskInfo.thread.interrupt(); Review comment: For the first point, I added a call to `Thread.interrupted()` at the very end of the outermost finally in the task callable. For the second point, the interrupt here is meant as a "last resort" if the graceful shutdown didn't complete for some reason within the configured timeout, the interrupt within the tasks should normally end the shutdown before this is called ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org For additional commands, e-mail: commits-h...@druid.apache.org