From de947d52fd9ab45246d543c81eee726ef0cac510 Mon Sep 17 00:00:00 2001 From: Nils Homer Date: Sat, 10 Jun 2017 22:32:44 -0700 Subject: [PATCH] Adding a new execution system. This system can be used with the --experimental-execution option. --- .../dagr/core/cmdline/DagrCoreMain.scala | 3 + .../main/scala/dagr/core/exec/Executor.scala | 12 +- .../dagr/core/execsystem/TaskTracker.scala | 1 - .../core/execsystem2/DependencyGraph.scala | 145 +++ .../dagr/core/execsystem2/GraphExecutor.scala | 423 ++++++++ .../dagr/core/execsystem2/TaskExecutor.scala | 67 ++ .../dagr/core/execsystem2/TaskInfo.scala | 45 + .../dagr/core/execsystem2/TaskStatus.scala | 64 ++ .../execsystem2/TopLikeStatusReporter.scala | 63 ++ .../execsystem2/local/LocalTaskExecutor.scala | 331 +++++++ .../execsystem2/local/LocalTaskRunner.scala | 157 +++ .../util/InterruptableFuture.scala | 138 +++ .../reporting/TopLikeStatusReporter.scala | 7 + .../execsystem2/DependencyGraphTest.scala | 203 ++++ .../execsystem2/GraphExecutorImplTest.scala | 914 ++++++++++++++++++ .../core/execsystem2/GraphExecutorTest.scala | 459 +++++++++ .../execsystem2/GraphExecutorUnitSpec.scala | 200 ++++ .../dagr/core/execsystem2/PromiseTask.scala | 54 ++ .../dagr/core/execsystem2/TaskInfoTest.scala | 58 ++ .../TopLikeStatusReporterTest.scala | 103 ++ .../local/LocalTaskExecutorTest.scala | 271 ++++++ .../local/LocalTaskRunnerTest.scala | 211 ++++ .../util/InterruptableFutureTest.scala | 165 ++++ .../scala/dagr/pipelines/SleepyPipeline.scala | 89 ++ 24 files changed, 4180 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/dagr/core/execsystem2/DependencyGraph.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/GraphExecutor.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/TaskExecutor.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/TaskInfo.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/TaskStatus.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/TopLikeStatusReporter.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/local/LocalTaskExecutor.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/local/LocalTaskRunner.scala create mode 100644 core/src/main/scala/dagr/core/execsystem2/util/InterruptableFuture.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/DependencyGraphTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/GraphExecutorImplTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/GraphExecutorTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/GraphExecutorUnitSpec.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/PromiseTask.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/TaskInfoTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/TopLikeStatusReporterTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/local/LocalTaskExecutorTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/local/LocalTaskRunnerTest.scala create mode 100644 core/src/test/scala/dagr/core/execsystem2/util/InterruptableFutureTest.scala create mode 100644 pipelines/src/main/scala/dagr/pipelines/SleepyPipeline.scala diff --git a/core/src/main/scala/dagr/core/cmdline/DagrCoreMain.scala b/core/src/main/scala/dagr/core/cmdline/DagrCoreMain.scala index aa895e27..85450256 100644 --- a/core/src/main/scala/dagr/core/cmdline/DagrCoreMain.scala +++ b/core/src/main/scala/dagr/core/cmdline/DagrCoreMain.scala @@ -108,6 +108,8 @@ class DagrCoreArgs( val report: Option[Path] = None, @arg(doc = "Provide an top-like interface for tasks with the give delay in seconds. This suppress info logging.") var interactive: Boolean = false, + @arg(doc = "Use the experimental execution system.") + val experimentalExecution: Boolean = false, @arg(doc = "Attempt to replay using the provided replay log") val replayLog: Option[FilePath] = None ) extends LazyLogging { @@ -157,6 +159,7 @@ class DagrCoreArgs( val resources = SystemResources(cores = cores.map(Cores(_)), totalMemory = memory.map(Memory(_))) this.executor = Some( Executor( + experimentalExecution = experimentalExecution, resources = resources, scriptsDirectory = scriptsDirectory, logDirectory = logDirectory diff --git a/core/src/main/scala/dagr/core/exec/Executor.scala b/core/src/main/scala/dagr/core/exec/Executor.scala index 0af96c1a..83be8a2c 100644 --- a/core/src/main/scala/dagr/core/exec/Executor.scala +++ b/core/src/main/scala/dagr/core/exec/Executor.scala @@ -27,6 +27,8 @@ package dagr.core.exec import com.fulcrumgenomics.commons.CommonsDef.{DirPath, yieldAndThen} import dagr.core.execsystem.{SystemResources, TaskManager} +import dagr.core.execsystem2.GraphExecutor +import dagr.core.execsystem2.local.LocalTaskExecutor import dagr.core.reporting.ReportingDef.{TaskLogger, TaskRegister} import dagr.core.reporting.{FinalStatusReporter, TaskStatusLogger} import dagr.core.tasksystem.Task @@ -37,11 +39,17 @@ import scala.concurrent.ExecutionContext object Executor { /** Create a new executor. */ - def apply(resources: SystemResources, + def apply(experimentalExecution: Boolean, + resources: SystemResources, scriptsDirectory: Option[DirPath], logDirectory: Option[DirPath] )(implicit ex: ExecutionContext): Executor = { - new TaskManager(taskManagerResources=resources, scriptsDirectory=scriptsDirectory, logDirectory=logDirectory) + if (experimentalExecution) { + GraphExecutor(new LocalTaskExecutor(systemResources=resources, scriptsDirectory=scriptsDirectory, logDirectory=logDirectory)) + } + else { + new TaskManager(taskManagerResources=resources, scriptsDirectory=scriptsDirectory, logDirectory=logDirectory) + } } } diff --git a/core/src/main/scala/dagr/core/execsystem/TaskTracker.scala b/core/src/main/scala/dagr/core/execsystem/TaskTracker.scala index 198752c7..46a046a8 100644 --- a/core/src/main/scala/dagr/core/execsystem/TaskTracker.scala +++ b/core/src/main/scala/dagr/core/execsystem/TaskTracker.scala @@ -30,7 +30,6 @@ import com.fulcrumgenomics.commons.collection.BiMap import com.fulcrumgenomics.commons.util.LazyLogging import dagr.core.DagrDef._ import dagr.core.execsystem.TaskStatus._ -import dagr.core.reporting.ReportingDef.TaskRegister import dagr.core.tasksystem.Task import dagr.core.tasksystem.Task.{TaskStatus => RootTaskStatus} diff --git a/core/src/main/scala/dagr/core/execsystem2/DependencyGraph.scala b/core/src/main/scala/dagr/core/execsystem2/DependencyGraph.scala new file mode 100644 index 00000000..9d8bc0e8 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/DependencyGraph.scala @@ -0,0 +1,145 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicInteger + +import com.fulcrumgenomics.commons.util.LazyLogging +import dagr.core.exec.ExecDef +import dagr.core.tasksystem.Task + +/** + * A trait that classes that track and update dependencies on tasks should implement. + */ +trait DependencyGraph { + + /** Add a task to the dependency graph and returns true if the task has no dependencies, false otherwise. */ + def add(task: Task): Boolean + + /** None if the task was already added, true if the task was added and has no dependencies, false otherwise. */ + def maybeAdd(task: Task): Option[Boolean] + + /** Removes this task as a dependency for all other tasks in this dependency graph. The task should not depend on + * any tasks, and all tasks that depend on it will have their dependency on this task removed. + */ + def remove(task: Task): Seq[Task] + + /** Returns None if the task is not in the graph, true if it has dependencies, false otherwise. + */ + def hasDependencies(task: Task): Option[Boolean] + + /** Returns true if the task is in the graph, false otherwise. */ + def contains(task: Task): Boolean + + /** The number of tasks in the dependency graph. */ + def size: Int + + /** Throws an exception if there is a cycle in the dependency graph. The exception may have relevant debug + * information. + * @param task + */ + def exceptIfCyclicalDependency(task: Task): Unit +} + +object DependencyGraph { + /** Returns a default implementation of a dependency graph (i.e. a [[SimpleDependencyGraph]])*/ + def apply(): DependencyGraph = new SimpleDependencyGraph +} + +/** + * A very simple dependency graph that uses a [[CountDownLatch]] on the number of dependencies for a [[Task]] to + * block until a task has no dependencies. + */ +private class SimpleDependencyGraph extends DependencyGraph with LazyLogging { + import scala.collection.mutable + + /** The map of tasks to the number of remaining unsatisfied dependencies. */ + private val graph: mutable.Map[Task, AtomicInteger] = ExecDef.concurrentMap() + + /** Adds the task to the dependency graph. Returns None if the task has already been added, true if the task + * has no dependencies, and false if it has dependencies. */ + def maybeAdd(task: Task): Option[Boolean] = this.synchronized { if (contains(task)) None else Some(add(task)) } + + /** Adds the task to the dependency graph. The task should not already be part of the graph. Returns true if the task + * has no dependencies, and false if it has dependencies */ + def add(task: Task): Boolean = this.synchronized { + require(!this.graph.contains(task), s"Task '${task.name}' is already part of the dependency graph") + this.graph.put(task, new AtomicInteger(task.tasksDependedOn.size)) + !this.hasDependencies(task).get + } + + /** Removes this task from the dependency graph. It should not depend on any tasks itself, and all tasks that depend + * on it will have their dependency on this task removed. Returns any dependent task that now has no more + * dependencies. + */ + def remove(task: Task): Seq[Task] = { + require(task.tasksDependedOn.isEmpty, + s"Removing a task '${task.name}' from the dependency graph that has dependencies: " + + task.tasksDependedOn.map(_.name).mkString(", ")) + // remove this as a dependency for all other tasks that depend on this task + task.tasksDependingOnThisTask.flatMap { dependent => + dependent.synchronized { + require(this.graph.contains(dependent), s"Dependent '${dependent.name}' not in the dependency graph") + task !=> dependent + val latch = this.graph(dependent) + if (latch.decrementAndGet() == 0) Some(dependent) else None + } + }.toSeq + } + + /** Returns None if the task is not part of the graph, true if the task has dependencies, false otherwise. */ + def hasDependencies(task: Task): Option[Boolean] = { + this.graph.get(task).map { e => e.get() > 0 } + } + + /** Returns true if the task is part of the graph, false otherwise. */ + def contains(task: Task): Boolean = this.graph.contains(task) + + /** Returns the number of tasks in the graph. */ + def size: Int = this.graph.size + + // NB: I think that the dependents in Task could be updated while were are doing this! How do we synchronize? Do we + // have a global lock in the Task object? + /** Throws an exception if there is a cycle in the dependency graph. The exception may have relevant debug + * information. + * @param task + */ + def exceptIfCyclicalDependency(task: Task): Unit = this.synchronized { + // check for cycles + if (Task.hasCycle(task)) { + logger.error("Task was part of a graph that had a cycle") + for (component <- Task.findStronglyConnectedComponents(task = task)) { + if (Task.isComponentACycle(component = component)) { + logger.error("Tasks were part of a strongly connected component with a cycle: " + + component.map(t => s"'${t.name}'").mkString(", ")) + } + } + throw new IllegalArgumentException(s"Task was part of a graph that had a cycle '${task.name}'") + } + } +} + diff --git a/core/src/main/scala/dagr/core/execsystem2/GraphExecutor.scala b/core/src/main/scala/dagr/core/execsystem2/GraphExecutor.scala new file mode 100644 index 00000000..ba9b6346 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/GraphExecutor.scala @@ -0,0 +1,423 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import com.fulcrumgenomics.commons.CommonsDef.DirPath +import com.fulcrumgenomics.commons.util.LazyLogging +import dagr.core.exec.{ExecDef, Executor} +import dagr.core.execsystem.SystemResources +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.tasksystem.Task.{TaskInfo => RootTaskInfo} +import dagr.core.tasksystem.{Retry, Task} + +import scala.collection.mutable +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Future, blocking} +import scala.util.{Failure, Success} + +/** Coordinates between the dependency graph ([[DependencyGraph]]) and task executor ([[TaskExecutor]]) given a (root) + * task to execute. + */ +trait GraphExecutor[T<:Task] extends Executor { + /** Start the execution of this task and all tasks that depend on it. Returns the number of tasks that were not + * executed. A given task should only be attempted once. */ + def execute(rootTask: Task): Int + + /** Returns true if the task is part of the execution graph, false otherwise. Since the graph is lazily built, + * a task may not yet be contained in the graph. */ + def contains(task: Task): Boolean + + /** Returns the resources used by the underlying task executor(s). */ + def resources: Option[SystemResources] = Some(this.taskExecutor.resources) + + /** Returns the task status by ordinal */ + final def from(ordinal: Int): TaskStatus = TaskStatus.withValue(ordinal) + + /** Returns the executor that execute tasks. */ + protected def taskExecutor: TaskExecutor[T] + + /** Returns the graph that tracks dependencies. */ + protected def dependencyGraph: DependencyGraph +} + +object GraphExecutor { + /** Greates a default graph executor given a task executor */ + def apply[T<:Task](taskExecutor: TaskExecutor[T])(implicit ex: ExecutionContext): GraphExecutor[T] = + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) +} + +/** An implementation of an executor of a tasks that have dependencies. + * + * Currently, only a single task executor is supported. In the future, we could have a list of partial functions that + * map a type of task to its associated task executor. + * + * @param taskExecutor the executor for tasks of type [[T]]. + * @param dependencyGraph the depdendency graph to track when tasks have no more dependencies. + * @param ex the execution context in which to run execution (but not the task execution themselves). + * @tparam T the type of task that can be individually executed. + */ +private class GraphExecutorImpl[T<:Task](protected val taskExecutor: TaskExecutor[T], + protected val dependencyGraph: DependencyGraph=DependencyGraph()) + (implicit ex: ExecutionContext) + extends GraphExecutor[T] with LazyLogging { + + /** The tasks currently known by the executor. */ + private val _tasks: mutable.Set[Task] = ExecDef.concurrentSet() + + /** A lock to synchronize when the task execution or dependency information is updated. */ + private val lock: Object = (dependencyGraph, _tasks) + + /** The tasks currently known by the executor. */ + def tasks: Traversable[Task] = this._tasks + + /** Returns true if the task is known */ + override def contains(task: Task): Boolean = _tasks.contains(task) + + /** Returns the log directory. */ + def logDir: DirPath = this.taskExecutor.logDir + + /** Start the execution of this task and all tasks that depend on it. Returns the number of tasks that were not + * executed. A given task should only be attempted once. */ + def _execute(rootTask: Task): Int = { + + // Catch failure if the initial registration fails. + val rootFuture: Future[Task] = failFutureWithTaskStatus(rootTask) { + Future { + lockIt { + registerTask(rootTask) match { + case None => throw new IllegalArgumentException(s"Task '${rootTask.name}' already attempted.") + case Some(false) => throw new IllegalArgumentException(s"Task '${rootTask.name}' depends on ${rootTask.tasksDependedOn.size} tasks.") + case Some(true) => Unit + } + } + rootTask + } + } flatMap { task: Task => + processTask(task) + } + + // Wait forever for it to run + Await.ready(rootFuture, Duration.Inf) + rootFuture onComplete { + case Success(t) => logger.info(s"Completed root task '${t.name}' successfully") + case Failure(thr) => logger.error(thr, thr.getMessage) + } + + // Return the number of tasks known not to succeed + this._tasks.map(_.taskInfo).count { info => !info.status.isInstanceOf[Succeeded] } + } + + /** Process a given task, its sub-tree, and tasks that only depend on this task. + * 1. add any tasks that depend on this task to the dependency graph, if not already added. + * 2. execute this task and it's sub-tree recursively. + * 3. wait on tasks that depend on this task that were added in #1 to complete. + * Returns a future that completes if all dependent tasks complete (from #1), failure otherwise. Note: the future + * is not the result of the given task's execution, but of the execution of itself, its sub-tree, and any tasks that + * depended on it. + */ + private def processTask(task: Task): Future[Task] = { + // add any tasks that depend on this task to the graph. NB: add all the dependents at once for expedience + // (too many futures if done individually) + Future { + lockIt { + task.tasksDependingOnThisTask map { dependent => + // NB: this dependent may have been added by another task + registerTask(dependent) // returns Option[Boolean] + } + } + } flatMap { _ => + buildAndExecute(task) // build and execute the task itself, including its sub-tree + } map { t: Task => + // Get any dependents that can be processed now that this task and its sub-tree have complete execution. + // Process those tasks. + this.dependencyGraph.remove(t).map(processTask) + } flatMap { dependentFutures: Seq[Future[Task]] => + // wait until all tasks that depended on this task are processed + Future.sequence(dependentFutures) map { _: Seq[Task] => task } + } + } + + /** Builds a task and proceeds based on if it create other tasks or itself should be executed. Completes when + * itself and any children have executed. It does not update the dependency graph. */ + private def buildAndExecute(parent: Task): Future[Task] = failFutureWithTaskStatus(parent) { // tag any failures during build or execution + requireNoDependencies(parent) + + updateMetadata(parent, Queued) + + buildTask(parent) flatMap { + case x :: Nil if x == parent => // one task and it returned itself, so execute it + requireNoDependencies(parent) + if (this.taskCaches.isEmpty || this.taskCaches.exists(_.execute(parent))) { + executeWithTaskExecutor(parent) + } + else { + updateMetadata(parent, ManuallySucceeded) + Future.successful(parent) + } + case childTasks => // a different task, or more than one task, so build those tasks and execute them + requireNoDependencies(parent) + executeMultiTask(parent, childTasks) + } + } + + /** Build the task and catch any exceptions during the call to [[Task.getTasks()]]. */ + private def buildTask(task: Task): Future[Seq[Task]] = failWithFailedToBuild(task) { + requireNoDependencies(task) + val tasks = task.make().toList + if (tasks.isEmpty) throw new IllegalArgumentException(s"No tasks built from task: '${task.name}'") + tasks + } + + /** Executes the task given a set of child tasks that are to be executed instead. The success of the task depends + * on the success of its children. Isn't that true for most of life. */ + private def executeMultiTask(parent: Task, childTasks: Seq[Task]): Future[Task] = { + // use `recoverWith` so we set the parent status correctly on failure + failFutureWithFailedExecution(parent) { + // fail with [[FailedExecution]] with any failure. + updateMetadata(parent, Running) + + // For each child task, first add them to the dependency graph. Next, add any dependent tasks (task that depend + // on the child task), to the graph. The parent task can complete once they have all been added to the + // dependency graph. + val childFutures: Future[Seq[Task]] = Future { + // check that children of the parent haven't already been added to the graph. Not currently allowed. A child + // can only have one parent! + childTasks.foreach { child => + require(!this.dependencyGraph.contains(child), s"child '${child.name}' of parent '${parent.name}' already in the graph") + } + + // Register them all at once, since locking is expensive, and we may end up creating many, many futures + Future { + lockIt { + val result = childTasks map { child => + (registerTask(child).contains(true), child) + } + // check for cyclical dependencies since a new sub-tree has been added + this.dependencyGraph.exceptIfCyclicalDependency(parent) + // NB: returns true if the task has no dependencies, false otherwise + result + } + } + } flatMap { future: Future[Seq[(Boolean, Task)]] => + // Process each child task that has no dependencies, otherwise just return a success for it. In the latter + // case, it will be processed once all its dependencies have been met (executed). Note this will happen only + // since presumably children with dependencies depend on children without dependencies, and so will be handled + // in their processTask call. + future flatMap { things: Seq[(Boolean, Task)] => + val futures = things.map { case (hasNoDependencies: Boolean, child: Task) => + if (hasNoDependencies) processTask(child) else Future.successful(child) + } + val result = Future.sequence(futures) + result + } + } + + childFutures map { _ => // is of type Seq[Task] + // Update the status of the parent to succeeded if *all* children succeeded. + updateMetadata(parent, SucceededExecution) + parent + } recoverWith { + // in the case of a failure with tagged exception, fail with the underlying throwable + case thr: TaggedException => Future.failed(thr.thr) + } + } + } + + /** Submits and executes a task vai the task executor. */ + private def executeWithTaskExecutor(task: Task): Future[T] = { + Future { task.asInstanceOf[T] } flatMap { t: T => + val subFuture = submissionFuture (t) + val execFuture = executionFuture (t, subFuture=subFuture) + val onComplFuture = onCompleteFuture (t, execFuture=execFuture) + val complFuture = completedTaskFuture(t, onComplFuture=onComplFuture) + complFuture + } + } + + /** Try and submit the task and execute it with the task executor. Returns a future that completes when the task + * can start executing, due to any reason (ex. scheduling or assigning resources, locality, etc.). The inner future + * completes after the task has successfully executed. Returns a future that is wrapped so + * that any failure is tagged as [[FailedSubmission]]. */ + private def submissionFuture(task: T): Future[Future[T]] = failFutureWithFailedSubmission(task) { + // Update the task to be submitted ot the task executor + requireTaskStatus(task, Queued) + updateMetadata(task, Submitted) + + // NB: convert to type [[T]] here so that the exception is caught during submission + Future { task.asInstanceOf[T] } flatMap { t: T => + // This future completes when the task has started execution. A delay may occur to scheduling, resourcing, or any + // other multitude of reasons. + // TODO: partial functions here + this.taskExecutor.execute(t) + // TODO: could have a status for 'Scheduled' (eg. Submitted -> Scheduled -> Running) + } + } + + /** Update the task status to [[Running]] when the task can start executing (when the outer future completes). + * Wraps the future that completes when the execution completes so that any failure is tagged as [[FailedExecution]]. + * */ + private def executionFuture(task: T, subFuture: Future[Future[T]]): Future[T] = { + subFuture flatMap { execFuture: Future[T] => + // NB: the future we return has already been created (by the task executor), but we need to update the task + // status after it starts executing and before it completes. + requireTaskStatus(task, Submitted) + // The task is now running, so wait for it to finish running + updateMetadata(task, Running) + failFutureWithFailedExecution(task) { execFuture } + } + } + + /** Run the task's [[Task.onComplete]] method once it has executed successfully. Returns a future that is wrapped so + * that any failure is tagged as [[FailedOnComplete]]. w*/ + private def onCompleteFuture(task: T, execFuture: Future[T]): Future[T] = { + // Wrap running the onComplete method in a future! + def onComplete(): Future[T] = failWithFailedOnComplete(task) { + task.taskInfo.exitCode.foreach { code => + if (!task.onComplete(code)) throw new IllegalArgumentException(s"onComplete failed for task: ${task.name}") + } + task + } + + execFuture flatMap { task: T => + requireTaskStatus(task, Running) + onComplete() + } recoverWith { + // only run the onComplete method if the it failed when running + case taggedException: TaggedException if taggedException.status == FailedExecution => + // if the onComplete future completes, fail with the original exception, otherwise it should fail with an + // onComplete failure + onComplete() flatMap { _ => Future.failed(taggedException) } + } + } + + /** If the given future completes, update the status to [[SucceededExecution]], otherwise attempt to retry the task + * if it failed during execution. */ + private def completedTaskFuture(task: T, onComplFuture: Future[T]): Future[T] = failFutureWithTaskStatus(task) { + onComplFuture map { t: T => + updateMetadata(t, SucceededExecution) + require(!this.taskExecutor.contains(t), s"Task was still tracked '${t.name}'") + t + } recoverWith { + // retry only if we failed when running + case taggedException: TaggedException if taggedException.status == FailedExecution => + task match { + case r: Retry if r.retry(resources=taskExecutor.resources, task.taskInfo) => + // Queue and execute it again + task.taskInfo.attempts += 1 + updateMetadata(task, Queued) + this.executeWithTaskExecutor(task) + case _ => + Future.failed(taggedException) + } + } + } + + /** Update the status of of a task and returns the most current copy of the metadata */ + private def updateMetadata(task: Task, status: TaskStatus): RootTaskInfo = lockIt { + if (!this._tasks.contains(task)) { + // NB: TaskInfo's constructor will assign a reference from task to info + new TaskInfo( + task = task, + initStatus = status + ) + this._tasks.add(task) + } + else { + task.taskInfo.status = status + } + task.taskInfo + } + + /** Registers the task. If it already has been registered, does nothing. Adds the task to the dependency graph + * and sets the status to Pending. Returns None if the task was previously added, true if the task was added and + * has no dependencies, and false otherwise */ + private def registerTask(task: Task): Option[Boolean] = lockIt { + val result = this.dependencyGraph.maybeAdd(task) + result.foreach { _ => + // dependent was added + updateMetadata(task, Pending) + } + result + } + + /** Provides synchronization and signals that this may block. */ + private def lockIt[A](body: =>A): A = blocking { this.lock.synchronized(body) } + + /** Set the status to the failed and add the throwable to the failures map for this task */ + private def fail(task: Task, thr: Throwable, status: TaskStatus): Unit = lockIt { + require(status.isInstanceOf[Failed], s"$status was not of type Failed") + updateMetadata(task, status) + } + + /** Ensure that the task has the given status. */ + private def requireTaskStatus(task: Task, status: TaskStatus): Unit = { + val _status = task.taskInfo.status + require(_status == status, s"Task '${task.name}' was not in ${status.name} state: ${_status.name}") + } + + /** Ensure that the task has no dependencies. If missingOk is set, then if do not throw an exception if the task + * is not known. */ + private def requireNoDependencies(task: Task, missingOk: Boolean = false): Unit = { + this.dependencyGraph.hasDependencies(task) match { + case Some(true) => throw new IllegalArgumentException(s"Task ${task.name} has dependencies.") + case None if !missingOk => throw new IllegalArgumentException(s"Task ${task.name} is not in the dependency graph.") + case _ => Unit + } + } + + /** Returns the [[Future[T]] when failed has its throwable tagged with a status using [[TaggedException]] */ + private def failFutureWithTaskStatus[A](task: Task, status: TaskStatus=FailedUnknown)(future: Future[A]): Future[A] = { + // If there was any failure, the throwable should be a [[TaggedException]] so that the appropriate failed status can + // be set. + future recoverWith { + case thr: TaggedException => + require(thr.status.isInstanceOf[Failed] || Seq(Running, Queued).contains(thr.status), + s"Expected status to be Failed, Running, or Queued, but found ${thr.status}") + Future.failed[A](thr) + case thr: Throwable => + fail(task, thr, status) + Future.failed[A](TaggedException(thr=thr, status=status)) + case other => throw new IllegalArgumentException(s"Expected a throwable, found $other") + } + } + //def failFutureWithFailedToBuild [A](task: Task)(future: Future[A]): Future[A] = failFutureWithTaskStatus(task, FailedToBuild )(future) + def failFutureWithFailedSubmission[A](task: Task)(future: Future[A]): Future[A] = failFutureWithTaskStatus(task, FailedSubmission)(future) + def failFutureWithFailedExecution [A](task: Task)(future: Future[A]): Future[A] = failFutureWithTaskStatus(task, FailedExecution )(future) + //def failFutureWithFailedOnComplete[A](task: Task)(future: Future[A]): Future[A] = failFutureWithTaskStatus(task, FailedOnComplete)(future) + + /** Returns a [[Future[T]] with a given body that when failed has its throwable tagged with a status using + * [[TaggedException]] */ + private def failWithTaskStatus[A](task: Task, status: TaskStatus=FailedUnknown, body: =>A): Future[A] = { + failFutureWithTaskStatus(task=task, status=status)(future = Future[A] { body }) + } + def failWithFailedToBuild [A](task: Task)(body: => A): Future[A] = failWithTaskStatus(task, FailedToBuild, body) + //def failWithFailedSubmission[A](task: Task)(body: => A): Future[A] = failWithTaskStatus(task, FailedSubmission, body) + //def failWithFailedExecution [A](task: Task)(body: => A): Future[A] = failWithTaskStatus(task, FailedExecution, body) + def failWithFailedOnComplete[A](task: Task)(body: => A): Future[A] = failWithTaskStatus(task, FailedOnComplete, body) +} + +/** A little class to store an exception and associated task status. */ +private[execsystem2] case class TaggedException(thr: Throwable, status: TaskStatus) extends Exception(thr) \ No newline at end of file diff --git a/core/src/main/scala/dagr/core/execsystem2/TaskExecutor.scala b/core/src/main/scala/dagr/core/execsystem2/TaskExecutor.scala new file mode 100644 index 00000000..3df2b7b1 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/TaskExecutor.scala @@ -0,0 +1,67 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import com.fulcrumgenomics.commons.CommonsDef.DirPath +import dagr.core.execsystem.SystemResources +import dagr.core.tasksystem.Task + +import scala.concurrent.Future +import scala.concurrent.duration.Duration + + +/** An executor is responsible for executing one or more tasks. That's it. How the the tasks are executed is + * entirely up to the concrete implementations, including scheduling with resource management. This leaves A LOT + * of things not specified for various executors (ex. SGE, PBS, local, mixed) on purpose. */ +trait TaskExecutor[T<:Task] { + /** simple name (not unique) for the executor. */ + def name: String = getClass.getSimpleName + + /** Execute a task. The first outer returned completes when a task is eligible for execution. It may delay, for + * example, if there are not enough system resources to run. The inner future completes when the task has completed + * executing. If the task can never by run, a failure should return immediately. + */ + def execute(task: T): Future[Future[T]] + + /** terminate a task, returns true if successful, false otherwise, None if it knows nothing about the task. */ + def kill(task: T, duration: Duration = Duration.Zero): Option[Boolean] + + /** true if the task is running, false if not running or not tracked */ + def running(task: T): Boolean + + /** true if the task executor knows about the task but has not run, false otherwise. */ + def waiting(task: T): Boolean + + /** true if the task executor knows about the task, false otherwise */ + def contains(task: T): Boolean + + /** The amount of resources this task executor can use to execute tasks. */ + def resources: SystemResources + + /** Returns the log directory. */ + def logDir: DirPath +} + diff --git a/core/src/main/scala/dagr/core/execsystem2/TaskInfo.scala b/core/src/main/scala/dagr/core/execsystem2/TaskInfo.scala new file mode 100644 index 00000000..d4bb0ce7 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/TaskInfo.scala @@ -0,0 +1,45 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import java.time.Instant + +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.tasksystem.Task + +/** [[dagr.core.tasksystem.Task.TaskInfo]] implementation specific to [[dagr.core.execsystem2]]. */ +class TaskInfo(task: Task, initStatus: TaskStatus) + extends Task.TaskInfo(task=task, initStatus=initStatus) { + + /** Gets the instant that the task was submitted to the execution system. */ + override protected[core] def submissionDate: Option[Instant] = this(Pending) + + /** The instant the task started executing. */ + override protected[core] def startDate: Option[Instant] = this(Running) + + /** The instant that the task finished executing. */ + override protected[core] def endDate: Option[Instant] = latestStatus[Completed] +} diff --git a/core/src/main/scala/dagr/core/execsystem2/TaskStatus.scala b/core/src/main/scala/dagr/core/execsystem2/TaskStatus.scala new file mode 100644 index 00000000..25527786 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/TaskStatus.scala @@ -0,0 +1,64 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import dagr.core.execsystem2.TaskStatus.{ManuallySucceeded, SucceededExecution} +import dagr.core.tasksystem.Task +import enumeratum.values.{IntEnum, IntEnumEntry} + +/** The root of all task statuses in [[dagr.core.execsystem2]]. */ +sealed abstract class TaskStatus extends IntEnumEntry with Task.TaskStatus { + override def ordinal = this.value + /** Returns true if this status indicates any type of success, false otherwise. */ + def success: Boolean = this.ordinal == SucceededExecution.ordinal || this.ordinal == ManuallySucceeded.ordinal +} + +case object TaskStatus extends IntEnum[TaskStatus] { + val values = findValues + + /** Trait for all statuses prior to submission for execution */ + sealed trait PreSubmission extends TaskStatus + case object Pending extends PreSubmission { val description: String = "Has unmet dependencies"; val value: Int = 0 } + case object Queued extends PreSubmission { val description: String = "Ready for execution"; val value: Int = 1 } + + /** Trait for all statuses during execution */ + sealed trait Executing extends TaskStatus + case object Submitted extends Executing { val description: String = "Submitted for execution"; val value: Int = 2 } + case object Running extends Executing { val description: String = "Executing"; val value: Int = 3 } + + /** Trait for all statuses after execution has completed */ + sealed trait Completed extends TaskStatus + sealed trait Failed extends Completed + sealed trait Succeeded extends Completed + case object Stopped extends Completed { val description: String = "Stopped prior to completion"; val value: Int = 4 } + case object FailedToBuild extends Failed { val description: String = "Failed to build the task"; val value: Int = 5 } + case object FailedSubmission extends Failed { val description: String = "Could not be submitted to the executor"; val value: Int = 6 } + case object FailedExecution extends Failed { val description: String = "Failed during execution"; val value: Int = 7 } + case object FailedOnComplete extends Failed { val description: String = "Failed during the onComplete callback"; val value: Int = 8 } + case object FailedUnknown extends Failed { val description: String = "Failed for unknown reasons"; val value: Int = 9 } + case object SucceededExecution extends Succeeded { val description: String = "Succeeded execution"; val value: Int = 10 } + case object ManuallySucceeded extends Succeeded { val description: String = "Manually marked as succeeded"; val value: Int = 11 } +} diff --git a/core/src/main/scala/dagr/core/execsystem2/TopLikeStatusReporter.scala b/core/src/main/scala/dagr/core/execsystem2/TopLikeStatusReporter.scala new file mode 100644 index 00000000..5b6a1f39 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/TopLikeStatusReporter.scala @@ -0,0 +1,63 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import java.io.ByteArrayOutputStream + +import dagr.core.execsystem.SystemResources +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.reporting.{TopLikeStatusReporter => BaseTopLikeStatusReporter} +import dagr.core.tasksystem.Task + +/** A simple top-like status reporter for [[dagr.core.execsystem2.GraphExecutor]]. + * @param systemResources the system resources used while executing + * @param loggerOut the stream to which log messages are written, or none if no stream is available. + * @param print the method to use to write task status information, one line at a time. + */ +class TopLikeStatusReporter(val systemResources: SystemResources, + protected val loggerOut: Option[ByteArrayOutputStream] = None, + protected val print: String => Unit = print) + extends BaseTopLikeStatusReporter { + + + /** True if the task is running, false otherwise. */ + protected[execsystem2] def running(task: Task): Boolean = task.taskInfo.status == Running + + /** True if the task is ready for execution (no dependencies), false otherwise. */ + protected[execsystem2] def queued(task: Task): Boolean = task.taskInfo.status == Queued || task.taskInfo.status == Submitted + + /** True if the task has failed, false otherwise. */ + protected[execsystem2] def failed(task: Task): Boolean = task.taskInfo.status.isInstanceOf[Failed] + + /** True if the task has succeeded, false otherwise. */ + protected[execsystem2] def succeeded(task: Task): Boolean = task.taskInfo.status == SucceededExecution + + /** True if the task has completed regardless of status, false otherwise. */ + protected[execsystem2] def completed(task: Task): Boolean = task.taskInfo.status.isInstanceOf[Completed] + + /** True if the task has unmet dependencies, false otherwise. */ + protected[execsystem2] def pending(task: Task): Boolean = task.taskInfo.status == Pending +} diff --git a/core/src/main/scala/dagr/core/execsystem2/local/LocalTaskExecutor.scala b/core/src/main/scala/dagr/core/execsystem2/local/LocalTaskExecutor.scala new file mode 100644 index 00000000..d57adf1f --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/local/LocalTaskExecutor.scala @@ -0,0 +1,331 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2.local + +import java.nio.file.Path +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicInteger + +import com.fulcrumgenomics.commons.CommonsDef.DirPath +import com.fulcrumgenomics.commons.io.{Io, PathUtil} +import com.fulcrumgenomics.commons.util.LazyLogging +import dagr.core.DagrDef.TaskId +import dagr.core.exec._ +import dagr.core.execsystem._ +import dagr.core.execsystem2.TaskExecutor +import dagr.core.execsystem2.util.InterruptableFuture +import dagr.core.execsystem2.util.InterruptableFuture.Interruptable +import dagr.core.tasksystem._ + +import scala.collection.mutable +import scala.concurrent._ +import scala.concurrent.duration.Duration + +/** Various defaults for executing locally.*/ +object LocalTaskExecutorDefaults extends LazyLogging { + + /** The default resources to use when executing locally. This includes both resources for [[InJvmTask]]s and + * [[ProcessTask]]s (JVM and System resources respectively). */ + def defaultSystemResources: SystemResources = { + val resources = SystemResources(cores=None, totalMemory=None) // Let the apply method figure it all out + logger.debug("Defaulting System Resources to " + resources.cores.value + " cores and " + Resource.parseBytesToSize(resources.systemMemory.value) + " memory") + logger.debug("Defaulting JVM Resources to " + Resource.parseBytesToSize(resources.jvmMemory.value) + " memory") + resources + } + + /** @return the default scheduler */ + def defaultScheduler: Scheduler = new NaiveScheduler +} + +object LocalTaskExecutor { + /** A class to facilitate scheduling and executing a task. + * + * @param taskRunner the task runner used to execute the task + * @param latch the latch that will be zero when the task is scheduled to execute. + * @param submitFuture the future that will complete once the task is scheduled. + * @param executeFuture the future that will complete once the task has finished executing. None if it has not + * started executing. + * @param resources the resources that the task was scheduled with, None if not scheduled. + */ + protected case class TaskInfo + (taskRunner: LocalTaskRunner, + latch: CountDownLatch, + submitFuture: InterruptableFuture[LocalTaskRunner], + executeFuture: Option[InterruptableFuture[UnitTask]] = None, + resources: Option[ResourceSet] = None + ) +} + +/** + * + * @param systemResources the system (JVM and Process) resources to use + * @param scriptsDirectory the directory to which to write script files (mainly for [[ProcessTask]]s). + * @param logDirectory the directory to which to write log files (mainly for [[ProcessTask]]s). + * @param scheduler the scheduler to use to allocate resources to tasks and decided in which order they should execute. + * @param ex the execution context. + */ +class LocalTaskExecutor(systemResources: SystemResources = LocalTaskExecutorDefaults.defaultSystemResources, + scriptsDirectory: Option[Path] = None, + logDirectory: Option[Path] = None, + scheduler: Scheduler = LocalTaskExecutorDefaults.defaultScheduler + )(implicit ex: ExecutionContext) extends TaskExecutor[UnitTask] with LazyLogging { + import LocalTaskExecutor.TaskInfo + + /** Provides a unique identifier for tasks. */ + private val nextTaskId = new AtomicInteger(1) + + /** The actual directory for script files. */ + private val actualScriptsDirectory = scriptsDirectory getOrElse Io.makeTempDir("scripts") + + /** The actual directory for log files. */ + private val actualLogsDirectory = logDirectory getOrElse Io.makeTempDir("logs") + + /** The map between the tasks and information about their scheduling and execution. */ + protected val taskInfo: mutable.Map[UnitTask, TaskInfo] = ExecDef.concurrentMap() + + logger.debug(s"Executing with ${systemResources.cores} cores and ${systemResources.systemMemory.prettyString} system memory.") + logger.debug("Script files will be written to: " + actualScriptsDirectory) + logger.debug("Logs will be written to: " + actualLogsDirectory) + + /** The system resources used by this executor */ + def resources: SystemResources = this.systemResources + + /** Returns the log directory. */ + override def logDir: DirPath = actualLogsDirectory + + /** Returns true if the task is running, false if not running or not tracked */ + def running(task: UnitTask): Boolean = { + val info = this.taskInfo(task) + info.resources.isDefined && info.executeFuture.isDefined + } + + // true if the task has been scheduled for execution or is executing. + //def scheduled(task: UnitTask): Boolean = this.taskInfo(task).resources.isDefined + + /** Returns true if the task has been submitted (i.e. is ready to execute) but has not been scheduled, false otherwise */ + def waiting(task: UnitTask): Boolean = this.taskInfo(task).resources.isEmpty + + /** Returns true if we know about this task, false otherwise */ + def contains(task: UnitTask): Boolean = this.taskInfo.contains(task) + + /** Kill the underlying task. Returns None if the task is not being tracked. Returns true if the task was explicitly + * stopped. Returns false if either if the task has not been started. or if the task could + * not be stopped. + */ + def kill(task: UnitTask, duration: Duration = Duration.Zero): Option[Boolean] = this.synchronized { + this.taskInfo.get(task) match { + case None => None + case Some(info) => + val isRunning = running(task) // save this for later + interrupt(future=info.submitFuture, duration=duration) + info.executeFuture.foreach(f => interrupt(future=f, duration=duration)) + if (this.runningTasks.contains(task)) { + this.schedule() // since the resources have been freed up + } + this.taskInfo.remove(task) + Some(isRunning) + } + } + + /** Schedule and execute at task. The outer future will complete once the task has been scheduled, while the inner + * future will complete once the task has completed execution. + */ + def execute(task: UnitTask): Future[Future[UnitTask]] = { + throwableIfCanNeverBeScheduled(task) match { + case Some(throwable) => Future.failed[Future[UnitTask]](throwable) + case None => + // Submit it for scheduling + val submitFuture = buildTaskInfoAndSubmitFuture(task=task) + // Run the scheduler just in case it can be run immediately. + this.schedule() + // Execute it once it has been scheduled. + submitFuture.future map { taskRunner => blocking { executeTaskRunner(taskRunner) } } + } + } + + /** Create the necessary information to schedule and execute a task, and returns a future that completes when + * the task is scheduled for execution. */ + private def buildTaskInfoAndSubmitFuture(task: UnitTask): InterruptableFuture[LocalTaskRunner] = { + require(task._taskInfo.isDefined, "Task._taskInfo is not defined") + + // Set some basic task info directly on the task + val taskId = nextTaskId.getAndIncrement() + task.taskInfo.id = Some(taskId) + task.taskInfo.script = Some(scriptPathFor(task, taskId, task.taskInfo.attempts)) + task.taskInfo.log = Some(logPathFor(task, taskId, task.taskInfo.attempts)) + val taskRunner = LocalTaskRunner(task=task) + + // Create a future that waits until the task is scheduled for execution (the latch reaches zero). It returusn the + // task runner that executes the task. + val latch = new CountDownLatch(1) + val submitFuture = Future { + blocking { + latch.await() + } + taskRunner + } interruptable() + + // Track the task. + this.taskInfo.synchronized { + this.taskInfo(task) = TaskInfo( + taskRunner = taskRunner, + latch = latch, + submitFuture = submitFuture + ) + } + + submitFuture + } + + /** Run the task to completion and handle its completion. */ + private def executeTaskRunner(taskRunner: LocalTaskRunner): Future[UnitTask] = { + val task = taskRunner.task + // execute the task + val executeFuture: InterruptableFuture[UnitTask] = taskRunner.execute() interruptable() + // update the future that complets when the task execution complets + this.taskInfo.synchronized { + this.taskInfo.update(taskRunner.task, this.taskInfo(task).copy(executeFuture=Some(executeFuture))) + } + // update task information when it completes + executeFuture map { t => + this.complete(t) // Regardless of success or failure, stop tracking the task + this.schedule() // since maybe other tasks can now run with the freed up resources + // Throw the throwable if it exists. + taskRunner.throwable.foreach { thr => + throw new IllegalArgumentException(s"Task '${task.name}': ${thr.getMessage}", thr) + } + // Throw an exception if the task had a non-zero exit code, indicating failure + if (!taskRunner.exitCode.contains(0)) throw new IllegalStateException(s"Task '${task.name}' had exit code '${taskRunner.exitCode.getOrElse("None")}'") + task + } + } + + /** Gets the path to a task specific script or log file. */ + private def pathFor(task: UnitTask, taskId: TaskId, attemptIndex: Int, directory: Path, ext: String): Path = { + val sanitizedName: String = PathUtil.sanitizeFileName(task.name) + PathUtil.pathTo(directory.toString, s"$sanitizedName.$taskId.$attemptIndex.$ext") + } + + /** Gets the path to a task specific script file. */ + private def scriptPathFor(task: UnitTask, taskId: TaskId, attemptIndex: Int): Path = + pathFor(task, taskId, attemptIndex, actualScriptsDirectory, "sh") + + /** Gets the path to a task specific log file. */ + private def logPathFor(task: UnitTask, taskId: TaskId, attemptIndex: Int): Path = + pathFor(task, taskId, attemptIndex, actualLogsDirectory, "log") + + /** Gets all tasks that have been submitted for execution but have not been scheduled. */ + private def toScheduleTasks: Set[UnitTask] = { + this.taskInfo.flatMap { case (task, info) => + if (info.resources.isEmpty) Some(task) else None + }.toSet + } + + /** Gets a map from task to the resources it was scheduled with or None if not scheduled. */ + private def runningTasks: Map[UnitTask, ResourceSet] = this.taskInfo.flatMap { case (task, info) => + info.resources match { + case Some(resources) => Some(task -> resources) + case None => None + } + }.toMap + + /** Waits for at most the given duration for the given future to complete, otherwise interrupting it. */ + private def interrupt[T](future: InterruptableFuture[T], duration: Duration = Duration.Zero): Unit = { + try { + Await.result(future.future, duration) + } catch { + case _: InterruptedException => future.interrupt() + case _: TimeoutException => future.interrupt() + } + } + + /** Completes the execution of this task, killing it if necessary. The task will no longer be tracked. */ + private def complete(task: UnitTask): Boolean = kill(task=task) match { + case None => throw new IllegalArgumentException(s"Tried to remove a task that was not tracked: '${task.name}'") + case Some(_) => true + } + + /** Returns None if the task can be scheduled using all available system resources, otherwise, some exception that + * can be thrown. */ + protected def throwableIfCanNeverBeScheduled(task: UnitTask): Option[Throwable] = { + val canSchedule = scheduler.schedule( + runningTasks = Map.empty, + readyTasks = Seq(task), + systemCores = systemResources.cores, + systemMemory = systemResources.systemMemory, + jvmMemory = systemResources.jvmMemory + ).nonEmpty + + if (canSchedule) None + else { + val msg = new StringBuilder + msg.append(s"There will never be enough resources to schedule the task: '${task.name}'") + + // Attempt to estimate how much resources _it would take_ to execute the task. + val resourcesType: String = task match { + case _: FixedResources => "FixedResources" + case _: VariableResources => "VariableResources" + case _: Schedulable => "Schedulable" + case _ => "Unknown Type" + } + val resources: Option[ResourceSet] = task match { + case t: Schedulable => t.minResources(new ResourceSet(systemResources.cores, systemResources.systemMemory)) + case _ => None + } + val cores = resources.map(_.cores.toString).getOrElse("?") + val memory = resources.map(_.memory.prettyString).getOrElse("?") + msg.append(s"\nTask with name '${task.name}' requires $cores cores and $memory memory (task schedulable type: $resourcesType)") + msg.append(s"\nThere are ${systemResources.cores} core(s) and ${systemResources.systemMemory.prettyString} system memory available.") + + Some(new IllegalArgumentException(msg.toString)) + } + } + + /** Attempts to schedule tasks that at tracked but have not been scheduled. */ + private def schedule(): Boolean = this.synchronized { + logger.debug("found " + toScheduleTasks.size + " ready tasks") + this.toScheduleTasks.foreach { task => + logger.debug("ready task: " + task.name) + } + + val tasksToSchedule: Map[UnitTask, ResourceSet] = scheduler.schedule( + runningTasks = this.runningTasks, + readyTasks = this.toScheduleTasks, + systemCores = systemResources.cores, + systemMemory = systemResources.systemMemory, + jvmMemory = systemResources.jvmMemory + ) + + tasksToSchedule.foreach { case (task, resources) => + task.applyResources(resources) + val info = this.taskInfo(task) + this.taskInfo.update(task, info.copy(resources=Some(resources))) + info.latch.countDown() // release them from pre-submission + } + + tasksToSchedule.nonEmpty + } +} diff --git a/core/src/main/scala/dagr/core/execsystem2/local/LocalTaskRunner.scala b/core/src/main/scala/dagr/core/execsystem2/local/LocalTaskRunner.scala new file mode 100644 index 00000000..288d9249 --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/local/LocalTaskRunner.scala @@ -0,0 +1,157 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2.local + +import com.fulcrumgenomics.commons.CommonsDef.{FilePath, yieldAndThen} +import dagr.core.execsystem2.util.InterruptableFuture +import dagr.core.execsystem2.util.InterruptableFuture.Interruptable +import dagr.core.tasksystem.{InJvmTask, ProcessTask, UnitTask} + +import scala.concurrent._ +import scala.concurrent.duration.Duration + +/** + * Trait for tasks that execute locally, for example via a process or in the JVM, i.e. + * [[dagr.core.execsystem2.local.LocalTaskRunner.ProcessTaskExecutionRunner]] or + * [[dagr.core.execsystem2.local.LocalTaskRunner.InJvmTaskExecutionRunner]]. + */ +sealed trait LocalTaskRunner { + private var interruptibleFuture: Option[InterruptableFuture[UnitTask]] = None + + /** The task to execute */ + def task: UnitTask + + /** The path to the script where the commands are stored. */ + def script: FilePath = task.taskInfo.script.getOrElse(throw new IllegalArgumentException(s"Task '${task.name}' does not have a script file")) + + /** THe path to the log file where logging information are stored. */ + def log: FilePath = task.taskInfo.log.getOrElse(throw new IllegalArgumentException(s"Task '${task.name}' does not have a log file")) + + /** The exit code from executing the task, if the task has completed, None otherwise. A non-zero exit code indicates + * a failure. */ + def exitCode: Option[Int] = this.task.taskInfo.exitCode + + /** The throwable if thrown during execution, None otherwise. A throwable indicates a failure. */ + def throwable: Option[Throwable] = this.task.taskInfo.throwable + + /** The method to execute the underlying task. Completes when the underlying task completes. */ + final def execute()(implicit ex: ExecutionContext): Future[UnitTask] = { + val future = blocking { _execute() } interruptable() + interruptibleFuture = Some(future) + future.future + } + + /** Interrupts the execution of the task. Returns the task if the task already completed, None otherwise. */ + def interrupt(): Option[UnitTask] = { + updateExitCodeAndThrowable(code=Some(LocalTaskRunner.InterruptedExitCode)) + this.interruptibleFuture.flatMap(_.interrupt()) + } + + /** Returns true if the task was interrupted, false otherwise. */ + def interrupted(): Boolean = this.interruptibleFuture.exists(_.interrupted) + + /** Attempts to wait at most the given duration for the task to complete. If the task does not complete, it is + * interrupted. Returns the task if the task completed successfully, None otherwise. */ + def join(atMost: Duration): Option[UnitTask] = { + try { + Some(Await.result(this.interruptibleFuture.get.future, atMost)) + } catch { + case e: CancellationException => updateExitCodeAndThrowable(thr=Some(e)); interrupt() + case e: TimeoutException => updateExitCodeAndThrowable(thr=Some(e)); interrupt() + case e: InterruptedException => updateExitCodeAndThrowable(thr=Some(e)); interrupt() + } + } + + /** All sub-classes should implement this method to complete when the underlying task completes. This will be called + * by the [[execute()]] method. */ + protected def _execute()(implicit ex: ExecutionContext): Future[UnitTask] + + /** Updates the exit code and throwable upon completion of the task. Only updates if the exit code or throwable are + * defined respectively. */ + protected def updateExitCodeAndThrowable(code: Option[Int] = None, + thr: Option[Throwable] = None): Unit = this.synchronized { + val info = task.taskInfo + code.foreach { c => info.exitCode = Some(c) } + thr.foreach { t => info.throwable = Some(t) } + } +} + +object LocalTaskRunner { + + /** The exit code of any interrupted execution of a task. */ + val InterruptedExitCode = 255 + + /** Creates a task runner for the given task. Currently supports [[InJvmTask]]s or [[ProcessTask]]s only. */ + def apply(task: UnitTask): LocalTaskRunner = task match { + case t: InJvmTask => new InJvmTaskExecutionRunner(task=t) + case t: ProcessTask => new ProcessTaskExecutionRunner(task=t) + case _ => throw new RuntimeException(s"Cannot call execute on task '${task.name}' that are not 'UnitTask's.") + } + + /** Simple class that runs the given task. Wrap this in a thread, + * and will set the exit code to one if the parent thread was interrupted, + * otherwise the exit code will be set to that of the task's process. + * + * @param task the task to run + */ + class ProcessTaskExecutionRunner(val task: ProcessTask) extends LocalTaskRunner { + def _execute()(implicit ex: ExecutionContext): Future[UnitTask] = Future { + var process: Option[scala.sys.process.Process] = None + try { + val processBuilder: scala.sys.process.ProcessBuilder = task.processBuilder(script=script, logFile=log) + process = Some(processBuilder.run()) + updateExitCodeAndThrowable(code=process.map(_.exitValue()).orElse(Some(1))) + } catch { + case e: InterruptedException => + updateExitCodeAndThrowable(code=Some(InterruptedExitCode), thr=Some(e)) + case t: Throwable => + updateExitCodeAndThrowable(code=Some(1), thr=Some(t)) + } + + process.foreach(p => p.destroy()) + task + } + } + + /** Simple class that runs the given task's method. + * + * @param task the task to run + */ + class InJvmTaskExecutionRunner(val task: InJvmTask) extends LocalTaskRunner { + def _execute()(implicit ex: ExecutionContext): Future[UnitTask] = Future { + try { + val code = task.inJvmMethod(script=script, logFile=log) + yieldAndThen(code)(updateExitCodeAndThrowable(code=Some(code))) + } catch { + case e: InterruptedException => + updateExitCodeAndThrowable(code=Some(InterruptedExitCode), thr=Some(e)) + case t: Throwable => + updateExitCodeAndThrowable(code=Some(1), thr=Some(t)) + } + task + } + } +} diff --git a/core/src/main/scala/dagr/core/execsystem2/util/InterruptableFuture.scala b/core/src/main/scala/dagr/core/execsystem2/util/InterruptableFuture.scala new file mode 100644 index 00000000..4c2b637e --- /dev/null +++ b/core/src/main/scala/dagr/core/execsystem2/util/InterruptableFuture.scala @@ -0,0 +1,138 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2.util + +import java.util.concurrent.CancellationException + +import com.fulcrumgenomics.commons.CommonsDef.yieldAndThen + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Future, Promise} + +object InterruptableFuture { + /** Creates an interruptable future given the method that produces a result of type [[T]] */ + def apply[T](work: => T)(implicit ex: ExecutionContext): InterruptableFuture[T] = { + new InterruptableFuture[T]({ future => if (future.isCompleted) throw new InterruptedException else work }) + } + + /** + * Do not use the future that you pass to this method. Instead, rely on the returned [[InterruptableFuture.future]]. + */ + def apply[T](future: Future[T])(implicit ex: ExecutionContext): InterruptableFuture[T] = { + val work = () => Await.result(future, Duration.Inf) + InterruptableFuture(work=work()) + } + + /** A little implicit to convert a future to an interruptable future. */ + implicit class Interruptable[T](private val future: Future[T])(implicit ex: ExecutionContext) { + def interruptable() : InterruptableFuture[T] = InterruptableFuture(future) + } + + /** Converts a [[Future]] to an [[InterruptableFuture]] */ + implicit def toInterruptable[T](future: InterruptableFuture[T])(implicit ex: ExecutionContext): Future[T] = + future.future +} + +/** + * A wrapper for a future such that the future can be interrupted. The provided future should not be used, but + * instead, the future returned by the [[future()]] method. The [[interrupt()]] method can be used to interrupt the + * underlying future. + * + * Adapted from https://gist.github.com/viktorklang/5409467 + */ +class InterruptableFuture[T](fun: Future[T] => T)(implicit ex: ExecutionContext) { + // True if the underlying future has been interrupted. + private var _interrupted: Boolean = false + + // Some result if the underlying future completed with a result. + private var result: Option[T] = None + + // The current thread to interrupt. + private var currentThread: Option[Thread] = None + + // A lock to synchronize interruption and completion + private val lock = new Object + + // A promise used to return a new future on which callers can await or depend + private val promise = Promise[T]() + + // The future we promised to the caller. + private def promisedFuture = promise.future + + /** Sets the interrupt flag to true, and throws a [[CancellationException]]. */ + private def interruptWithException(): T = { + this._interrupted = true + throw new CancellationException + } + + /** The future on which the caller should depend. */ + def future: Future[T] = this.promise.future + + /** Interrupts the underlying future. Returns the result if the future has already completed, None otherwise. */ + def interrupt(): Option[T] = this.lock.synchronized { + // only interrupt if the previous thread was null + this.currentThread.foreach(_.interrupt()) + this.currentThread = None + this.promise.tryFailure(new CancellationException) + this._interrupted = true + this.result + } + + /** Returns true if the underlying future as interrupted, false otherwise. */ + def interrupted: Boolean = this.result.isEmpty && this._interrupted + + // Complete the promise with the given future. + this.promise tryCompleteWith Future { + // Get the current thread. May be None if the promise completed while waiting for the lock. + val thread = this.lock.synchronized { + if (this.promisedFuture.isCompleted) { + interruptWithException() + } + else { + val t = Thread.currentThread + yieldAndThen(Some(t))(this.currentThread=Some(t)) + } + } + thread match { + case None => interruptWithException() + case Some(t: Thread) => + try { + // execute the future and return the result! + val r = fun(this.promisedFuture) + yieldAndThen(r)(this.result=Some(r)) + } catch { + case _: InterruptedException => interruptWithException() + } finally { + // Synchronously set the thread to None and return the original thread + val originalThread = this.lock.synchronized { + yieldAndThen(this.currentThread)(this.currentThread=None) + } + // It is only interrupted if the current execution thread was set to None + this._interrupted = !originalThread.contains(t) + } + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/dagr/core/reporting/TopLikeStatusReporter.scala b/core/src/main/scala/dagr/core/reporting/TopLikeStatusReporter.scala index 4fbff63a..67235baf 100644 --- a/core/src/main/scala/dagr/core/reporting/TopLikeStatusReporter.scala +++ b/core/src/main/scala/dagr/core/reporting/TopLikeStatusReporter.scala @@ -34,6 +34,7 @@ import com.fulcrumgenomics.commons.util.TimeUtil._ import dagr.core.exec.ExecDef.concurrentSet import dagr.core.exec.Executor import dagr.core.execsystem.{SystemResources, TaskManager} +import dagr.core.execsystem2.GraphExecutor import dagr.core.reporting.ReportingDef.TaskLogger import dagr.core.tasksystem.Task.TaskInfo import dagr.core.tasksystem.{InJvmTask, ProcessTask, Task, UnitTask} @@ -131,6 +132,12 @@ object TopLikeStatusReporter { loggerOut = Some(loggerOutputStream), print = s => System.out.print(s) ) + case graphExecutor: GraphExecutor[_] => + new dagr.core.execsystem2.TopLikeStatusReporter( + systemResources = graphExecutor.resources.getOrElse(throw new IllegalArgumentException("No resource set defined")), + loggerOut = Some(loggerOutputStream), + print = s => System.out.print(s) + ) case _ => throw new IllegalArgumentException(s"Unknown executor: '${executor.getClass.getSimpleName}'") } diff --git a/core/src/test/scala/dagr/core/execsystem2/DependencyGraphTest.scala b/core/src/test/scala/dagr/core/execsystem2/DependencyGraphTest.scala new file mode 100644 index 00000000..f4802944 --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/DependencyGraphTest.scala @@ -0,0 +1,203 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import dagr.core.UnitSpec +import dagr.core.tasksystem.{NoOpInJvmTask, Task} +import org.scalatest.OptionValues + +class DependencyGraphTest extends UnitSpec with OptionValues { + + def dependencyGraph: DependencyGraph = DependencyGraph() + def task: Task = new NoOpInJvmTask("noop") + + "DependencyGraph.add" should "add a task to the graph that has no dependencies" in { + val graph = this.dependencyGraph + val task = this.task + graph.add(task) shouldBe true + } + + it should "not throw an exception when there are cycles in the graph" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child ==> root + graph.add(root) shouldBe false + graph.add(child) shouldBe false + } + + it should "add a task that has dependencies" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child + graph.add(root) shouldBe true + graph.add(child) shouldBe false + } + + "DependencyGraph.remove" should "throw an exception if the task to be removed still has dependencies" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child + graph.add(root) + graph.add(child) + an[IllegalArgumentException] should be thrownBy graph.remove(child) + } + + it should "remove all dependencies" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child + graph.add(root) shouldBe true + graph.add(child) shouldBe false + + graph.remove(root) should contain theSameElementsInOrderAs Seq(child) + graph.hasDependencies(child).value shouldBe false + graph.remove(child) shouldBe 'empty + } + + it should "throw an exception if a dependent is not found" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child + graph.add(root) shouldBe true + + an[IllegalArgumentException] should be thrownBy graph.remove(root) + } + + "DependencyGraph.hasDependencies" should "should return None when a task is not part of the graph" in { + val graph = this.dependencyGraph + val task = this.task + graph.hasDependencies(task) shouldBe 'empty + } + + it should "return Some(false) when a task has no dependencies" in { + val graph = this.dependencyGraph + val task = this.task + graph.add(task) shouldBe true + graph.hasDependencies(task).value shouldBe false + } + + it should "return Some(true) when a task has some dependencies" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child + graph.add(root) shouldBe true + graph.add(child) shouldBe false + graph.hasDependencies(root).value shouldBe false + graph.hasDependencies(child).value shouldBe true + } + + "DependencyGraph.contains" should "should return false when a task is not part of the graph" in { + val graph = this.dependencyGraph + val task = this.task + graph.contains(task) shouldBe false + } + + it should "should return true when a task is part of the graph" in { + val graph = this.dependencyGraph + val task = this.task + graph.add(task) shouldBe true + graph.contains(task) shouldBe true + } + + "DependencyGraph.size" should "should return the number of tasks in the graph" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child + graph.add(root) shouldBe true + graph.size shouldBe 1 + graph.add(child) shouldBe false + graph.size shouldBe 2 + } + + "DependencyGraph.exceptIfCyclicalDependency" should "throw an exception when there are cycles in the graph" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val child = this.task withName "child" + root ==> child ==> root + graph.add(root) shouldBe false + graph.add(child) shouldBe false + an[IllegalArgumentException] should be thrownBy graph.exceptIfCyclicalDependency(root) + an[IllegalArgumentException] should be thrownBy graph.exceptIfCyclicalDependency(child) + + } + + "DependencyGraph" should "do some operations" in { + val graph = this.dependencyGraph + val root = this.task withName "root" + val left = this.task withName "left" + val right = this.task withName "right" + val leaf = this.task withName "leaf" + root ==> (left :: right) ==> leaf + val tasks = Seq(root, left, right, leaf) + + // add to the graph all but the leaf + Seq(root, left, right).foreach { task => + graph.add(task) shouldBe (task == root) + } + graph.size shouldBe 3 + Seq(root, left, right).foreach { task => + graph.hasDependencies(task).value shouldBe (task != root) + } + graph.hasDependencies(leaf) shouldBe None + + // remove root + graph.remove(root) should contain theSameElementsInOrderAs Seq(left, right) + // add the leaf + graph.add(leaf) shouldBe false + graph.size shouldBe 4 + Seq(left, right, leaf).foreach { task => + graph.hasDependencies(task).value shouldBe (task == leaf) + } + + // remove left + graph.remove(left) shouldBe 'empty + graph.size shouldBe 4 + Seq(right, leaf).foreach { task => + graph.hasDependencies(task).value shouldBe (task == leaf) + } + + // remove right + graph.remove(right) should contain theSameElementsInOrderAs Seq(leaf) + graph.size shouldBe 4 + graph.hasDependencies(leaf).value shouldBe false + + // remove leaf + graph.remove(leaf) shouldBe 'empty + graph.size shouldBe 4 + + tasks.foreach { task => + graph.hasDependencies(task).value shouldBe false + } + } + +} diff --git a/core/src/test/scala/dagr/core/execsystem2/GraphExecutorImplTest.scala b/core/src/test/scala/dagr/core/execsystem2/GraphExecutorImplTest.scala new file mode 100644 index 00000000..c8a43b43 --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/GraphExecutorImplTest.scala @@ -0,0 +1,914 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import dagr.core.exec.ResourceSet +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.execsystem2.local.LocalTaskExecutor +import dagr.core.tasksystem.Task.{TaskInfo => RootTaskInfo} +import dagr.core.tasksystem._ +import org.scalatest.PrivateMethodTester + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future} +import scala.reflect.ClassTag + +/** A place to store methods that call private methods in [[GraphExecutorImpl]]. */ +private object GraphExecutorImplTest extends PrivateMethodTester { + + private val processTaskDecorate = PrivateMethod[Future[Task]]('processTask) + def processTask(graphExecutor: GraphExecutorImpl[UnitTask], task: Task): Future[Task] = { + graphExecutor invokePrivate processTaskDecorate(task) + } + + private val buildAndExecuteDecorate = PrivateMethod[Future[Task]]('buildAndExecute) + def buildAndExecute(graphExecutor: GraphExecutorImpl[UnitTask], task: Task): Future[Task] = { + graphExecutor invokePrivate buildAndExecuteDecorate(task) + } + + private val buildTaskDecorate = PrivateMethod[Future[Seq[Task]]]('buildTask) + def buildTask(graphExecutor: GraphExecutorImpl[UnitTask], task: Task): Future[Seq[Task]] = { + graphExecutor invokePrivate buildTaskDecorate(task) + } + + private val executeWithTaskExecutorDecorate = PrivateMethod[Future[UnitTask]]('executeWithTaskExecutor) + def executeWithTaskExecutor(graphExecutor: GraphExecutorImpl[UnitTask], task: Task): Future[UnitTask] = { + graphExecutor invokePrivate executeWithTaskExecutorDecorate(task) + } + + private val executeMultiTaskDecorate = PrivateMethod[Future[Task]]('executeMultiTask) + def executeMultiTask(graphExecutor: GraphExecutorImpl[UnitTask], task: Task, childTasks: Seq[Task]): Future[Task] = { + graphExecutor invokePrivate executeMultiTaskDecorate(task, childTasks) + } + + private val submissionFutureDecorate = PrivateMethod[Future[Future[UnitTask]]]('submissionFuture) + def submissionFuture(graphExecutor: GraphExecutorImpl[UnitTask], task: Task): Future[Future[UnitTask]] = { + graphExecutor invokePrivate submissionFutureDecorate(task) + } + + private val executionFutureDecorate = PrivateMethod[Future[UnitTask]]('executionFuture) + def executionFuture(graphExecutor: GraphExecutorImpl[UnitTask], task: Task, subFuture: Future[Future[UnitTask]]): Future[UnitTask] = { + graphExecutor invokePrivate executionFutureDecorate(task, subFuture) + } + + private val onCompleteFutureDecorate = PrivateMethod[Future[UnitTask]]('onCompleteFuture) + def onCompleteFuture(graphExecutor: GraphExecutorImpl[UnitTask], task: Task, execFuture: Future[UnitTask]): Future[UnitTask] = { + graphExecutor invokePrivate onCompleteFutureDecorate(task, execFuture) + } + + private val completedTaskFutureDecorate = PrivateMethod[Future[UnitTask]]('completedTaskFuture) + def completedTaskFuture(graphExecutor: GraphExecutorImpl[UnitTask], task: Task, onComplFuture: Future[UnitTask]): Future[UnitTask] = { + graphExecutor invokePrivate completedTaskFutureDecorate(task, onComplFuture) + } + + private val requireNoDependenciesDecorate = PrivateMethod[Unit]('requireNoDependencies) + def requireNoDependencies(graphExecutor: GraphExecutorImpl[UnitTask], task: Task, missingOk: Boolean): Unit = { + graphExecutor invokePrivate requireNoDependenciesDecorate(task, missingOk) + } + + private val updateMetadataDecorate = PrivateMethod[RootTaskInfo]('updateMetadata) + def updateMetadata(graphExecutor: GraphExecutorImpl[UnitTask], task: Task, status: TaskStatus): RootTaskInfo = { + graphExecutor invokePrivate updateMetadataDecorate(task, status) + } +} + +class GraphExecutorImplTest extends GraphExecutorUnitSpec with PrivateMethodTester { + import GraphExecutorImplTest._ + + private def checkTaggedException[T<:Throwable](thr: Throwable, status: TaskStatus)(implicit evidence: ClassTag[T]) : Unit = { + thr shouldBe a[TaggedException] + val taggedException = thr.asInstanceOf[TaggedException] + taggedException.status shouldBe status + taggedException.thr shouldBe a[T] + } + + "GraphExecutorImpl.tasks" should "return the list of currently tracked tasks" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + + graphExecutor.contains(task) shouldBe false + + whenReady(Future(graphExecutor.execute(task))) { t => + t shouldBe 0 + graphExecutor.tasks.toSeq should contain theSameElementsInOrderAs Seq(task) + checkStatus(graphExecutor, task, SucceededExecution) + } + } + + "GraphExecutorImpl.info" should "return metadata about known tasks" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + + graphExecutor.contains(task) shouldBe false + + whenReady(Future(graphExecutor.execute(task))) { t => + t shouldBe 0 + checkStatus(graphExecutor, task, SucceededExecution) + } + } + + "GraphExecutorImpl.status" should "return the status about known tasks" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + + graphExecutor.contains(task) shouldBe false + + whenReady(Future(graphExecutor.execute(task))) { t => + t shouldBe 0 + graphExecutor.contains(task) shouldBe true + checkStatus(graphExecutor, task, SucceededExecution) + } + } + + "GraphExecutorImpl.execute" should "run a single task end-to-end with success" in { + val graphExecutor = this.graphExecutor + val root = successfulTask + graphExecutor.execute(root) shouldBe 0 + checkStatus(graphExecutor, root, SucceededExecution) + } + + it should "run a single task end-to-end with failure" in { + val graphExecutor = this.graphExecutor + val root = new ShellCommand("exit", "1") withName "exit 1" + graphExecutor.execute(root) shouldBe 1 + checkStatus(graphExecutor, root, FailedExecution) + } + + it should "run a pipeline end-to-end with success" in { + val graphExecutor = this.graphExecutor + val pipeline = this.pipeline + graphExecutor.execute(pipeline) shouldBe 0 + checkStatus(graphExecutor, pipeline, SucceededExecution) + } + + it should "run a pipeline end-to-end with failure" in { + val graphExecutor = this.graphExecutor + val pipeline = this.pipelineFailure + graphExecutor.execute(pipeline) shouldBe 3 // pipeline and two tasks + checkStatus(graphExecutor, pipeline, FailedExecution) + } + + it should "fails when the task executor does not support the task" in { + val graphExecutor = this.graphExecutor + val root = new Task { + final def getTasks: Traversable[_ <: this.type] = List(this) + } + graphExecutor.execute(root) shouldBe 1 + checkStatus(graphExecutor, root, FailedSubmission) + } + + it should "fails when a task cannot be scheduled" in { + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) + val graphExecutor = new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + val root = infiniteResourcesTask + + val execute = Future { graphExecutor.execute(root) } + whenReady(execute) { result => + result shouldBe 1 + checkStatus(graphExecutor, root, FailedSubmission) + } + } + + it should "should not execute the same task twice" in { + val root = successfulTask withName "root" + val graphExecutor = this.graphExecutor + + whenReady(Future { graphExecutor.execute(root) }) { res1 => + res1 shouldBe 0 + checkStatus(graphExecutor, root, SucceededExecution) + + whenReady(Future { graphExecutor.execute(root) }) { res2 => + res2 shouldBe 1 + checkStatus(graphExecutor, root, FailedUnknown) + } + } + } + + /** Process a given task, its sub-tree, tasks that only depend on this task. + * 1. add any tasks that depend on this task to the dependency graph if not already added + * 2. execute this task and it's sub-tree recursively + * 3. wait on tasks that depend on this task that were added in #1. + */ + + "GraphExecutorImpl.processTask" should "execute with no dependent tasks" in { + val root = successfulTask withName "root" + val (_, graphExecutor) = graphAndExecutor(root) + + updateMetadata(graphExecutor, root, Pending) + + whenReady(processTask(graphExecutor, root)) { _ => + checkStatus(graphExecutor, root, SucceededExecution) + } + } + + it should "execute after adding a dependent task to the dependency graph" in { + val root = successfulTask withName "root" + val child = successfulTask withName "child" + root ==> child + val (dependencyGraph, graphExecutor) = graphAndExecutor(root) + + updateMetadata(graphExecutor, root, Pending) + dependencyGraph.contains(child) shouldBe false + + whenReady(processTask(graphExecutor, root)) { _ => + dependencyGraph.contains(child) shouldBe true + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, child, SucceededExecution) + } + } + + it should "execute after not adding a dependent task to the dependency graph since it was already added" in { + // NB: I have no way to verify that it was actually ** not ** added + val root = successfulTask withName "root" + val child = successfulTask withName "child" + root ==> child + val (dependencyGraph, graphExecutor) = graphAndExecutor(root) + dependencyGraph.add(child) + + updateMetadata(graphExecutor, root, Pending) + updateMetadata(graphExecutor, child, Pending) + dependencyGraph.contains(child) shouldBe true + + whenReady(processTask(graphExecutor, root)) { _ => + dependencyGraph.contains(child) shouldBe true + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, child, SucceededExecution) + } + } + + it should "execute and recurse on children" in { + val root = successfulTask withName "root" + val middle = successfulTask withName "middle" + val leaf = successfulTask withName "leaf" + root ==> middle ==> leaf + val (dependencyGraph, graphExecutor) = graphAndExecutor(root) + + updateMetadata(graphExecutor, root, Pending) + dependencyGraph.contains(middle) shouldBe false + dependencyGraph.contains(leaf) shouldBe false + + whenReady(processTask(graphExecutor, root)) { _ => + dependencyGraph.contains(middle) shouldBe true + dependencyGraph.contains(leaf) shouldBe true + + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, middle, SucceededExecution) + checkStatus(graphExecutor, leaf, SucceededExecution) + } + } + + it should "execute and recurse on children when the middle child fails" in { + val root = successfulTask withName "root" + val middle = failureTask withName "middle-fail" + val leaf = failureTask withName "leaf" // use a failureTask to prove it never gets run! + root ==> middle ==> leaf + val (dependencyGraph, graphExecutor) = graphAndExecutor(root) + + updateMetadata(graphExecutor, root, Pending) + dependencyGraph.contains(middle) shouldBe false + dependencyGraph.contains(leaf) shouldBe false + + whenReady(processTask(graphExecutor, root).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + dependencyGraph.contains(middle) shouldBe true + dependencyGraph.contains(leaf) shouldBe true + + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, middle, FailedSubmission) + checkStatus(graphExecutor, leaf, Pending) + } + } + + it should "execute and recurse on children when the leaf child fails" in { + val root = successfulTask withName "root" + val middle = successfulTask withName "middle" + val leaf = failureTask withName "leaf-fail" + root ==> middle ==> leaf + val (dependencyGraph, graphExecutor) = graphAndExecutor(root) + + updateMetadata(graphExecutor, root, Pending) + dependencyGraph.contains(middle) shouldBe false + dependencyGraph.contains(leaf) shouldBe false + + whenReady(processTask(graphExecutor, root).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + dependencyGraph.contains(middle) shouldBe true + dependencyGraph.contains(leaf) shouldBe true + + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, middle, SucceededExecution) + checkStatus(graphExecutor, leaf, FailedSubmission) + } + } + + it should "complete when there was a dependent task with an unmet dependency" in { + val root = successfulTask withName "root" + val child = successfulTask withName "middle" + val otherRoot = successfulTask withName "middle" + root ==> child + otherRoot ==> child + val (dependencyGraph, graphExecutor) = graphAndExecutor(root) + dependencyGraph.add(otherRoot) + + updateMetadata(graphExecutor, root, Pending) + updateMetadata(graphExecutor, otherRoot, Pending) + dependencyGraph.contains(child) shouldBe false + + // run root, but child depends on other-root + whenReady(processTask(graphExecutor, root)) { _ => + dependencyGraph.contains(root) shouldBe true + dependencyGraph.contains(child) shouldBe true + dependencyGraph.contains(otherRoot) shouldBe true + + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, child, Pending) + checkStatus(graphExecutor, otherRoot, Pending) + } + + // run other-root to release child + whenReady(processTask(graphExecutor, otherRoot)) { _ => + dependencyGraph.contains(root) shouldBe true + dependencyGraph.contains(child) shouldBe true + dependencyGraph.contains(otherRoot) shouldBe true + + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, child, SucceededExecution) + checkStatus(graphExecutor, otherRoot, SucceededExecution) + } + } + + "GraphExecutorImpl.buildAndExecute" should "build and execute a unit task successfully" in { + val task = successfulTask + val (_, graphExecutor) = graphAndExecutor(task) + + whenReady(buildAndExecute(graphExecutor, task)) { t => + checkStatus(graphExecutor, t, SucceededExecution) + } + } + + it should "build and execute a pipeline successfully" in { + val pipeline = this.pipeline + val (_, graphExecutor) = graphAndExecutor(pipeline) + + whenReady(buildAndExecute(graphExecutor, pipeline)) { t => + checkStatus(graphExecutor, t, SucceededExecution) + } + } + + it should "build and execute a pipeline to failure" in { + val pipeline = this.pipelineFailure + val (_, graphExecutor) = graphAndExecutor(pipeline) + + whenReady(buildAndExecute(graphExecutor, pipeline).failed) { thr: Throwable => + checkTaggedException[IllegalStateException](thr, FailedExecution) + } + } + + it should "throw an exception if the task has dependencies" in { + val graphExecutor = this.graphExecutor + val root = failureTask + + val child = successfulTask + root ==> child + updateMetadata(graphExecutor, root, Pending).status shouldBe Pending + updateMetadata(graphExecutor, child, Pending).status shouldBe Pending + an[IllegalArgumentException] should be thrownBy { + Await.result(buildAndExecute(graphExecutor, child), Duration("1s")) + } + } + + "GraphExecutorImpl.buildTask" should "succeed if one or more tasks where built" in { + val task = failureTask + val (dependencyGraph, graphExecutor) = graphAndExecutor(task) + + whenReady(buildTask(graphExecutor, task)) { tasks => + tasks should contain theSameElementsInOrderAs Seq(task) + } + } + + it should "fail if no tasks were built" in { + val graphExecutor = this.graphExecutor + val task = new Task { + override def getTasks = Nil + } + + whenReady(buildTask(graphExecutor, task).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedToBuild) + } + } + + "GraphExecutorImpl.executeWithTaskExecutor" should "execute a task successfully end-to-end" in { + val task = successfulTask + val (dependencyGraph, graphExecutor) = graphAndExecutor(task) + + // scheduled and run immediately, onComplete throws + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(executeWithTaskExecutor(graphExecutor, task)) { t: UnitTask => + checkStatus(graphExecutor, t, SucceededExecution) + } + } + + it should "fail when any exception was thrown (not in a future)" in { + val graphExecutor = this.graphExecutorSubmissionException + // scheduled and run immediately + val task = successfulTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(executeWithTaskExecutor(graphExecutor, task).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + } + } + + it should "fail when any future fails" in { + { + val graphExecutor = this.graphExecutorFailedSubmission + // scheduled and run immediately + val task = successfulTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(executeWithTaskExecutor(graphExecutor, task).failed) { thr => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + } + } + + { + val graphExecutor = this.graphExecutorFailedExecution + // scheduled and run immediately, onComplete throws + val task = successfulTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(executeWithTaskExecutor(graphExecutor, task).failed) { thr => + checkTaggedException[IllegalArgumentException](thr, FailedExecution) + } + } + } + + "GraphExecutorImpl.executeMultiTask" should "execute a pipeline successfully" in { + val parent = this.pipeline + val pipeline = this.pipeline + val children = pipeline.getTasks.toSeq + + // break the link with root + children.foreach { child => + pipeline.root !=> child + } + + val (_, graphExecutor) = this.graphAndExecutor(parent) + + whenReady(executeMultiTask(graphExecutor, parent, children)) { t => + t shouldBe parent + val tasks = parent +: children + tasks.foreach { task => + checkStatus(graphExecutor, task, SucceededExecution) + } + val parentInstant = parent.taskInfo(SucceededExecution).value + children.foreach { child => + val childInstant = child.taskInfo(SucceededExecution).value + checkInstants(Seq(childInstant, parentInstant)) + } + } + } + + it should "execute successfully when the parent is a [[UnitTask]]" in { + val graphExecutor = this.graphExecutor + val parent = successfulTask + val pipeline = this.pipeline + val children = pipeline.getTasks.toSeq + + // break the link with root + children.foreach { child => + pipeline.root !=> child + } + + whenReady(executeMultiTask(graphExecutor, parent, children)) { t => + t shouldBe parent + val tasks = parent +: children + tasks.foreach { task => + checkStatus(graphExecutor, task, SucceededExecution) + } + val parentInstant = parent.taskInfo(SucceededExecution).value + children.foreach { child => + val childInstant = child.taskInfo(SucceededExecution).value + checkInstants(Seq(childInstant, parentInstant)) + } + } + } + + it should "execute a pipeline of pipelines" in { + val graphExecutor = this.graphExecutor + val pipeline2 = this.pipeline + val pipeline1 = new Pipeline() { + override def build() = root ==> pipeline2 + } + val parent = pipeline1 + val children = parent.getTasks.toSeq + + whenReady(executeMultiTask(graphExecutor, parent, children)) { t => + t shouldBe pipeline1 + val tasks = Seq(parent) ++ children ++ pipeline2.tasksDependingOnThisTask + tasks.foreach { task => + checkStatus(graphExecutor, task, SucceededExecution) + } + val parentInstant = parent.taskInfo(SucceededExecution).value + children.foreach { child => + val childInstant = child.taskInfo(SucceededExecution).value + checkInstants(Seq(childInstant, parentInstant)) + } + } + } + + it should "fail if any child fails" in { + val graphExecutor = this.graphExecutor + val pipeline2 = this.pipelineFailure withName "p2" + val pipeline1 = new Pipeline() { + name = "p1" + override def build() = root ==> pipeline2 + } + val parent = pipeline1 + val children = parent.getTasks.toSeq + + whenReady(executeMultiTask(graphExecutor, parent, children).failed) { thr: Throwable => + checkTaggedException[IllegalStateException](thr, FailedExecution) + + val okTasks = Seq(pipeline2.okTasks.head) + okTasks.foreach { task => + checkStatus(graphExecutor, task, SucceededExecution) + } + + val failTasks = Seq(pipeline2.failTask, pipeline2, pipeline1) + val failInstants = failTasks.map { task => + checkStatus(graphExecutor, task, FailedExecution) + task.taskInfo(FailedExecution).value + } + checkInstants(failInstants) + + checkStatus(graphExecutor, pipeline2.okTasks.last, Pending) + + // NB: pipeline2.okTasks(1) may have completed + Seq(SucceededExecution, Running, Submitted, Queued) should contain (pipeline2.okTasks(1).taskInfo.status) + } + } + + it should "fail if any child is already part of the dependency graph" in { + val root = successfulTask withName "root" + val child = successfulTask withName "child" + + root ==> child + + val (dependencyGraph, graphExecutor) = this.graphAndExecutor(root) + updateMetadata(graphExecutor, root, Running) + + // naughty + dependencyGraph.add(child) + updateMetadata(graphExecutor, child, Pending) + + whenReady(executeMultiTask(graphExecutor, root, Seq(child)).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedExecution) + + checkStatus(graphExecutor, child, Pending) + checkStatus(graphExecutor, root, FailedExecution) + } + } + + "GraphExecutorImpl.submissionFuture" should "throw an exception if the task status was not Queued" in { + val graphExecutor = this.graphExecutor + val task = infiniteResourcesTask + + an[NoSuchElementException] should be thrownBy submissionFuture(graphExecutor, task) + + updateMetadata(graphExecutor, task, Pending).status shouldBe Pending + an[IllegalArgumentException] should be thrownBy submissionFuture(graphExecutor, task) + } + + it should "fail if TaskExecutor.execute throws an exception" in { + // A task executor that throws an exception + val graphExecutor = graphExecutorSubmissionException + val task = infiniteDurationTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(submissionFuture(graphExecutor, task).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + } + } + + + it should "fail if the task could not be submitted" in { + // A task executor that fails the outer future execute method + val graphExecutor = graphExecutorFailedSubmission + val task = infiniteDurationTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(submissionFuture(graphExecutor, task).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + } + } + + it should "fails when a task cannot be scheduled" in { + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) + val graphExecutor = new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + val root = infiniteResourcesTask + + updateMetadata(graphExecutor, root, Queued).status shouldBe Queued + + whenReady(submissionFuture(graphExecutor, root).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + } + } + + it should "have status FailedExecution when the task fails execution" in { + val graphExecutor = graphExecutorFailedExecution + val task = infiniteResourcesTask // NB: infinite resources, cannot be scheduled + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(submissionFuture(graphExecutor, task)) { execFuture => + checkStatus(graphExecutor, task, Submitted) + whenReady(execFuture.failed) { thr => + checkStatus(graphExecutor, task, Submitted) + } + } + } + + it should "have status FailedSubmission when the task cannot be scheduled" in { + val graphExecutor = this.graphExecutor + val task = infiniteResourcesTask // NB: infinite resources + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(submissionFuture(graphExecutor, task).failed) { thr => + checkTaggedException[IllegalArgumentException](thr=thr, status=FailedSubmission) + checkStatus(graphExecutor, task, FailedSubmission) + } + } + + it should "have status Submitted when the task can be scheduled (but not yet running)" in { + val graphExecutor = this.graphExecutor + val task = infiniteDurationTask // NB: no resources + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + whenReady(submissionFuture(graphExecutor, task)) { execFuture => + execFuture.isCompleted shouldBe false + checkStatus(graphExecutor, task, Submitted) + } + } + + "GraphExecutorImpl.executionFuture" should "return a failure if the submission future failed" in { + // A task executor that fails the inner future execute method + val graphExecutor = graphExecutorFailedSubmission + val task = infiniteDurationTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + + whenReady(execFuture.failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedSubmission) + } + } + + it should "return a failure if the execution future failed" in { + // A task executor that fails the outer future execute method + val graphExecutor = graphExecutorFailedExecution + val task = infiniteDurationTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + + whenReady(execFuture.failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedExecution) + } + } + + it should "throw an exception if the status was not Submitted" in { + val graphExecutor = this.graphExecutor + val task = infiniteDurationTask // scheduled immediately + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + + whenReady(subFuture) { _ => + // Update to Queued + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + } + val execFuture = executionFuture(graphExecutor, task, subFuture) + + an[IllegalArgumentException] should be thrownBy { + Await.result(execFuture, Duration("1s")) + } + } + + it should "have status Running while running" in { + val graphExecutor = this.graphExecutor + val task = infiniteDurationTask // scheduled immediately + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + + // wait until it is scheduled + whenReady(subFuture) { _ => } + + checkStatus(graphExecutor, task, Running) + execFuture.isCompleted shouldBe false + } + + + it should "have status Running after executing" in { + val graphExecutor = this.graphExecutor + val task = successfulTask // scheduled and run immediately + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + + // wait until it is scheduled + whenReady(subFuture) { _ => + checkStatus(graphExecutor, task, Submitted) + val execFuture = executionFuture(graphExecutor, task, subFuture) + whenReady(execFuture) { t => + checkStatus(graphExecutor, t, Running) + } + } + } + + "GraphExecutorImpl.onCompleteFuture" should "throw an exception if the status was not running" in { + val graphExecutor = this.graphExecutor + val task = successfulTask // scheduled and run immediately + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + + whenReady(execFuture) { _ => + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + } + + an[IllegalArgumentException] should be thrownBy { + Await.result(onCompleteFuture(graphExecutor, task, execFuture), Duration("1s")) + } + } + + it should "fail when the onComplete method returns false" in { + val graphExecutor = this.graphExecutor + // scheduled and run immediately, onComplete throws + val task = new PromiseTask(Duration.Zero, ResourceSet.empty) with UnitTask { + override def onComplete(exitCode: Int): Boolean = false + } + + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + val onComplFuture = onCompleteFuture(graphExecutor, task, execFuture=execFuture) + + whenReady(onComplFuture.failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedOnComplete) + } + } + + it should "fail when the execFuture has failed" in { + val graphExecutor = this.graphExecutor + // scheduled and run immediately, onComplete throws + val task = new PromiseTask(Duration.Zero, ResourceSet.empty) with UnitTask { + override def onComplete(exitCode: Int): Boolean = false + } + + updateMetadata(graphExecutor, task, Running).status shouldBe Running + + val execFuture: Future[UnitTask] = Future.failed(TaggedException(thr=new IllegalArgumentException, status=FailedExecution)) + val onComplFuture = onCompleteFuture(graphExecutor, task, execFuture=execFuture) + + whenReady(onComplFuture.failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedExecution) + } + } + + it should "succeed when the onComplete method returns true" in { + val graphExecutor = this.graphExecutor + // scheduled and run immediately, onComplete throws + val task = successfulTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + val onComplFuture = onCompleteFuture(graphExecutor, task, execFuture=execFuture) + + whenReady(onComplFuture) { t => + checkStatus(graphExecutor, t, Running) + } + } + + "GraphExecutorImpl.completedTaskFuture" should "fail unknown if task could not be removed from the task executor" in { + val graphExecutor = { + // an executor that adds the task info back in before returning the future from execute() + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) { + override def execute(task: UnitTask): Future[Future[UnitTask]] = { + val future = super.execute(task) + val info = this.taskInfo(task) + // add the task info back in before completing! + future map { execFuture => + execFuture map { t => this.taskInfo(task) = info; t } + } + } + override def throwableIfCanNeverBeScheduled(task: UnitTask): Option[Throwable] = None + } + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + } + // scheduled and runs immediately + val task = successfulTask + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + val subFuture = submissionFuture(graphExecutor, task) + val execFuture = executionFuture(graphExecutor, task, subFuture) + val onComplFuture = onCompleteFuture(graphExecutor, task, execFuture=execFuture) + + whenReady(completedTaskFuture(graphExecutor, task, onComplFuture=onComplFuture).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedUnknown) + } + } + + it should "throw an exception if the throwable is not either Failed or Running" in { + val graphExecutor = this.graphExecutor + // scheduled immediately, but runs forever + val task = infiniteDurationTask + val onComplFuture = Future.failed(TaggedException(thr=new IllegalArgumentException, status=Pending)) + an[IllegalArgumentException] should be thrownBy { + Await.result(completedTaskFuture(graphExecutor, task, onComplFuture=onComplFuture), Duration("1s")) + } + } + + it should "update to the failed status when the the future failed" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + val onComplFuture = Future.failed(TaggedException(thr=new IllegalArgumentException, status=FailedOnComplete)) + whenReady(completedTaskFuture(graphExecutor, task, onComplFuture=onComplFuture).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedOnComplete) + } + } + + it should "fail with the throwable if it is not a TaggedException, and update the status to FailedUnknown" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + val onComplFuture = Future.failed(new IllegalArgumentException) + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + whenReady(completedTaskFuture(graphExecutor, task, onComplFuture=onComplFuture).failed) { thr: Throwable => + checkTaggedException[IllegalArgumentException](thr, FailedUnknown) + } + } + + it should "complete successfully and update the status to SucceededExecution" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + val onComplFuture = Future.successful(task) + updateMetadata(graphExecutor, task, Queued).status shouldBe Queued + whenReady(completedTaskFuture(graphExecutor, task, onComplFuture=onComplFuture)) { t => + checkStatus(graphExecutor, task, SucceededExecution) + } + } + + "GraphExecutorImpl.requireNoDependencies" should "should throw an exception if the task is not tracked and missingOk is false" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + an[IllegalArgumentException] should be thrownBy requireNoDependencies(graphExecutor, task, missingOk=false) + } + + it should "should not throw an exception if the task is not tracked and missingOk is true" in { + val graphExecutor = this.graphExecutor + val task = successfulTask + requireNoDependencies(graphExecutor, task, missingOk=true) + } + + it should "should throw an exception only if the task has dependencies" in { + val root = successfulTask + val child = successfulTask + val (_, graphExecutor) = this.graphAndExecutor(root) + + root ==> child + requireNoDependencies(graphExecutor, root, missingOk=false) + an[IllegalArgumentException] should be thrownBy requireNoDependencies(graphExecutor, child, missingOk=false) + } + + // TODO: test that onComplete runs when it fails or succeeds + +} diff --git a/core/src/test/scala/dagr/core/execsystem2/GraphExecutorTest.scala b/core/src/test/scala/dagr/core/execsystem2/GraphExecutorTest.scala new file mode 100644 index 00000000..b96cf13e --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/GraphExecutorTest.scala @@ -0,0 +1,459 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import dagr.core.TestTags +import dagr.core.exec.{Cores, Memory, ResourceSet} +import dagr.core.execsystem.SystemResources +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.execsystem2.local.LocalTaskExecutor +import dagr.core.tasksystem.Task.{TaskInfo => RootTaskInfo} +import dagr.core.tasksystem.{Retry, _} +import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.time.{Seconds, Span} + +import scala.concurrent.Future +import scala.concurrent.duration.Duration + +class GraphExecutorTest extends GraphExecutorUnitSpec { + + //////////////////////////////////////////////////////////////////////////////// + // Basic tests + //////////////////////////////////////////////////////////////////////////////// + + "GraphExecutor" should "run a single task end-to-end with success" in { + val graphExecutor = this.graphExecutor + val root = successfulTask withName "p1" + graphExecutor.execute(root) shouldBe 0 + checkStatus(graphExecutor, root, SucceededExecution) + } + + it should "run a single task end-to-end with failure" in { + val graphExecutor = this.graphExecutor + val root = new ShellCommand("exit", "1") withName "exit 1" + graphExecutor.execute(root) shouldBe 1 + checkStatus(graphExecutor, root, FailedExecution) + } + + it should "run two tasks end-to-end with success" in { + val graphExecutor = this.graphExecutor + val root = successfulTask withName "p1" + val child = successfulTask withName "p2" + + root ==> child + + graphExecutor.execute(root) shouldBe 0 + checkStatus(graphExecutor, root, SucceededExecution) + checkStatus(graphExecutor, child, SucceededExecution) + } + + it should "run a few tasks end-to-end with success" in { + val graphExecutor = this.graphExecutor + val root = successfulTask withName "root" + val left = successfulTask withName "left" + val right = successfulTask withName "right" + val leaf = successfulTask withName "leaf" + + root ==> (left :: right) ==> leaf + + graphExecutor.execute(root) shouldBe 0 + Seq(root, left, right, leaf).foreach { t => checkStatus(graphExecutor, t, SucceededExecution) } + } + + it should "run a pipeline end-to-end with success" in { + val graphExecutor = this.graphExecutor + val pipeline = this.pipeline + graphExecutor.execute(pipeline) shouldBe 0 + checkStatus(graphExecutor, pipeline, SucceededExecution) + } + + it should "run a pipeline end-to-end with failure" in { + val graphExecutor = this.graphExecutor + val pipeline = this.pipelineFailure // pipeline ==> (okTasks(0) ==> (failTask :: okTasks(1)) ==> okTasks(2)) + graphExecutor.execute(pipeline) shouldBe 3 + checkPipelineFailure(pipeline, graphExecutor) + } + + it should "fails when the task executor does not support the task" in { + val graphExecutor = this.graphExecutor + val root = new Task { + final def getTasks: Traversable[_ <: this.type] = List(this) + } + graphExecutor.execute(root) shouldBe 1 + checkStatus(graphExecutor, root, FailedSubmission) + } + + it should "fails when a task cannot be scheduled" in { + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) + val graphExecutor = new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + val root = infiniteResourcesTask + + val execute = Future { graphExecutor.execute(root) } + whenReady(execute) { result => + result shouldBe 1 + checkStatus(graphExecutor, root, FailedSubmission) + } + } + + it should "fails when a task is executed that has dependencies" in { + val graphExecutor = this.graphExecutor + val root = successfulTask + val child = successfulTask + root ==> child + + graphExecutor.execute(child) shouldBe 1 + //an[IllegalArgumentException] should be thrownBy graphExecutor.execute(child) + } + + it should "not depend on the order tasks are enqueued" in { + val graphExecutor = this.graphExecutor + val task = new Task { + override def getTasks: Traversable[_ <: Task] = { + pipeline.getTasks.toList.reverse + } + } + + graphExecutor.execute(task) shouldBe 0 + checkInfo(graphExecutor, task=task, statuses=Seq(Pending, Queued, Running, SucceededExecution)) + } + + //////////////////////////////////////////////////////////////////////////////// + // Checking various routes for a single task + // * FailedUnknown + // * Pending => Queued => Submitted => Running => SucceededExecution (for executable tasks) + // * Pending => Queued => Running => SucceededExecution (for pipelines) + // * Pending => FailedUnknown + // * Pending => Queued => FailedToBuild + // * Pending => Queued => Submitted => FailedSubmission + // * Pending => Queued => Submitted => Running => FailedExecution + // * Pending => Queued => Submitted => Running => FailedOnComplete + // *** Stopped and ManuallySucceeded not tested *** + //////////////////////////////////////////////////////////////////////////////// + + it should "route through FailedUnknown" in { + val graphExecutor = this.graphExecutor + // fails when we look at dependencies + val root = new PromiseTask(Duration.Zero, ResourceSet.empty) with UnitTask { + var count = true + override def tasksDependedOn: Traversable[Task] = throw new IllegalArgumentException + } withName "root" + graphExecutor.execute(root) shouldBe 1 + + checkInfo(graphExecutor, task=root, statuses=Seq(FailedUnknown)) + } + + it should "route through Pending => Queued => Submitted => Running => SucceededExecution (for executable tasks)" in { + val graphExecutor = this.graphExecutor + val root = successfulTask + graphExecutor.execute(root) shouldBe 0 + + checkInfo(graphExecutor, task=root, statuses=Seq(Pending, Queued, Submitted, Running, SucceededExecution)) + } + + it should "route through Pending => Queued => Running => SucceededExecution (for pipelines)" in { + val graphExecutor = this.graphExecutor + val task = pipeline + graphExecutor.execute(task) shouldBe 0 + + checkInfo(graphExecutor, task=task, statuses=Seq(Pending, Queued, Running, SucceededExecution)) + } + + // This test is disabled since it relies on how many times `tasksDependedOn` gets called, which is a bad idea. I + // don't see another way to make it blow up :/ + /* + it should "route through Pending => FailedUnknown" in { + val graphExecutor = this.graphExecutor + // fails when we look at dependencies the second time! + val root = new PromiseTask(Duration.Zero, ResourceSet.empty) with UnitTask { + var count = true + override def tasksDependedOn: Traversable[Task] = { + if (first) { + first = false + Traversable.empty + } + else { + throw new IllegalArgumentException + } + } + } withName "root" + graphExecutor.execute(root) shouldBe 1 + checkInfo(graphExecutor, task=root, statuses=Seq(Pending, FailedUnknown)) + } + */ + + it should "route through Pending => Queued => FailedToBuild" in { + val graphExecutor = this.graphExecutor + // fails when building + val root = new Task { + withName("root") + override def getTasks = Nil + } + + graphExecutor.execute(root) shouldBe 1 + checkInfo(graphExecutor, task=root, statuses=Seq(Pending, Queued, FailedToBuild)) + } + + it should "route through Pending => Queued => Submitted => FailedSubmission" in { + val graphExecutor = this.graphExecutor + val root = new Task { + final def getTasks: Traversable[_ <: this.type] = List(this) + } + + graphExecutor.execute(root) shouldBe 1 + checkInfo(graphExecutor, task=root, statuses=Seq(Pending, Queued, Submitted, FailedSubmission)) + } + + it should "route through Pending => Queued => Submitted => Running => FailedExecution" in { + val graphExecutor = this.graphExecutor + val root = new ShellCommand("exit", "1") withName "exit 1" + + graphExecutor.execute(root) shouldBe 1 + checkInfo(graphExecutor, task=root, statuses=Seq(Pending, Queued, Submitted, Running, FailedExecution)) + } + + it should "route through Pending => Queued => Submitted => Running => FailedOnComplete" in { + val graphExecutor = this.graphExecutor + // scheduled and run immediately, onComplete throws + val root = new PromiseTask(Duration.Zero, ResourceSet.empty) with UnitTask { + override def onComplete(exitCode: Int): Boolean = false + } + + graphExecutor.execute(root) shouldBe 1 + checkInfo(graphExecutor, task=root, statuses=Seq(Pending, Queued, Submitted, Running, FailedOnComplete)) + } + + //////////////////////////////////////////////////////////////////////////////// + // More complicated sets of tasks + //////////////////////////////////////////////////////////////////////////////// + + it should "execute a chain of one-task pipelines to successful execution" in { + val graphExecutor = this.graphExecutor + val p1 = pipelineOneTask withName "p1" + val p2 = pipelineOneTask withName "p2" + val p3 = pipelineOneTask withName "p3" + p1 ==> p2 ==> p3 + + graphExecutor.execute(p1) shouldBe 0 + + Seq(p1, p2, p3).foreach { p => + checkInfo(graphExecutor, task=p, statuses=Seq(Pending, Queued, Running, SucceededExecution)) + } + + val times = Seq(p1, p2, p3).map(_.taskInfo(SucceededExecution).value) + checkInstants(times) + } + + it should "execute a chain of pipelines, each with multiple tasks, to successful execution" in { + val graphExecutor = this.graphExecutor + val p1 = pipeline withName "p1" + val p2 = pipeline withName "p2" + val p3 = pipeline withName "p3" + p1 ==> p2 ==> p3 + + graphExecutor.execute(p1) shouldBe 0 + + Seq(p1, p2, p3).foreach { p => + checkInfo(graphExecutor, p, statuses=Seq(Pending, Queued, Running, SucceededExecution)) + } + val times = Seq(p1, p2, p3).map(_.taskInfo(SucceededExecution).value) + checkInstants(times) + } + + it should "succeed a pipeline task that has a dependent task that will fail" in { + val graphExecutor = this.graphExecutor + val p1 = this.pipelineOneTask withName "p1" + val fail = failureTask + + p1 ==> fail + graphExecutor.execute(p1) shouldBe 1 // fail + + checkInfo(graphExecutor, p1, statuses=Seq(Pending, Queued, Running, SucceededExecution)) + checkInfo(graphExecutor, fail, statuses=Seq(Pending, Queued, Submitted, FailedSubmission)) + } + + it should "execute a chain of pipelines where an intermediate pipeline fails" in { + val graphExecutor = this.graphExecutor + val p1 = pipeline withName "p1" + val p2 = pipelineFailure withName "p2" + val p3 = pipeline withName "p3" + p1 ==> p2 ==> p3 + + val result = graphExecutor.execute(p1) + + result should be >= 4 // p1, p2, p2-1, p2-fail, p3 + result should be <= 5 // p1, p2, p2-1, p2-2, p2-fail, p3 + + // success + checkInfo(graphExecutor, p1, statuses=Seq(Pending, Queued, Running, SucceededExecution)) + + // failure + checkPipelineFailure(p2, graphExecutor) + + // failure + checkStatus(graphExecutor, p3, Pending) + } + + + //////////////////////////////////////////////////////////////////////////////// + // Retries + //////////////////////////////////////////////////////////////////////////////// + + it should "retry a task that succeeds on its second attempt" in { + val graphExecutor = this.graphExecutor + val task = new SimpleInJvmTask with Retry { + var attempt = 0 + def run(): Unit = { attempt += 1; require(attempt > 1) } + override def retry(resources: SystemResources, taskInfo: RootTaskInfo): Boolean = attempt < 2 + } withName "retry-task" + + graphExecutor.execute(task) shouldBe 0 + val statuses = Seq(Pending, Queued, Submitted, Running, FailedExecution, Queued, Submitted, Running, SucceededExecution) + checkInfo(graphExecutor, task, statuses=statuses, attempts=2) + } + + it should "retry a task that fails on its second and final attempt" in { + val graphExecutor = this.graphExecutor + val task = new SimpleInJvmTask with Retry { + var attempt = 0 + def run(): Unit = { attempt += 1; throw new IllegalArgumentException("this task should never succeed") } + override def retry(resources: SystemResources, taskInfo: RootTaskInfo): Boolean = attempt < 2 + } withName "retry-failure-task" + + graphExecutor.execute(task) shouldBe 1 + val statuses = Seq(Pending, Queued, Submitted, Running, FailedExecution, Queued, Submitted, Running, FailedExecution) + checkInfo(graphExecutor, task, statuses=statuses, attempts=2) + } + + it should "retry a task that succeeds on its second attempt, after which a dependent task runs" in { + val graphExecutor = this.graphExecutor + val root = new SimpleInJvmTask with Retry { + var attempt = 0 + def run(): Unit = { attempt += 1; require(attempt > 1) } + override def retry(resources: SystemResources, taskInfo: RootTaskInfo): Boolean = attempt < 2 + } withName "retry-task" + val child = successfulTask + + root ==> child + + graphExecutor.execute(root) shouldBe 0 + Seq(root, child).foreach { task => + val attempts = if (root == task) 2 else 1 + val statuses = if (root == task) { + Seq(Pending, Queued, Submitted, Running, FailedExecution, Queued, Submitted, Running, SucceededExecution) + } + else { + Seq(Pending, Queued, Submitted, Running, SucceededExecution) + } + checkInfo(graphExecutor, task, statuses=statuses, attempts=attempts) + } + } + + //////////////////////////////////////////////////////////////////////////////// + // Long running tests + //////////////////////////////////////////////////////////////////////////////// + + { + val numTasks = 10000 + val dependencyProbability = 0.1 + + trait ZTask extends UnitTask { + override def pickResources(availableResources: ResourceSet): Option[ResourceSet] = { + val mem = Memory("1g") + (8 to 1 by -1).map(c => ResourceSet(Cores(c), mem)).find(rs => availableResources.subset(rs).isDefined) + } + } + + class ATask extends ProcessTask with ZTask { + override def args = "exit" :: "0" :: Nil + } + + class BTask extends SimpleInJvmTask with ZTask { + def run(): Unit = Unit + } + + def toATask: () => ATask = () => new ATask + def toBTask: () => BTask = () => new BTask + + + Seq(true, false).foreach { inJvm: Boolean => + + val toTask = if (inJvm) toBTask else toATask + val taskType = if (inJvm) "JVM" else "Shell" + + it should s"handle a few thousand $taskType tasks" taggedAs TestTags.LongRunningTest in { + + // create the tasks + val root = successfulTask withName "root" + val tasks: Seq[Task] = for (i <- 1 to numTasks) yield (toTask() withName s"task-$i") + + // make them depend on previous tasks + val randomNumberGenerator = scala.util.Random + for (i <- 1 until numTasks) { + for (j <- 1 until i) { + if (randomNumberGenerator.nextFloat < dependencyProbability) tasks(j) ==> tasks(i) + } + } + + val rootTasks: Seq[Task] = tasks.last +: tasks.filter(_.tasksDependedOn.isEmpty) + rootTasks.foreach { task => + root ==> task + } + + //val systemResources = SystemResources(cores=Cores(16), systemMemory=Memory("16g"), jvmMemory=Memory("16g")) + val systemResources = SystemResources.infinite + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory), systemResources=systemResources) + val graphExecutor = new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + + whenReady(Future { graphExecutor.execute(root) }, timeout=Timeout(Span(120, Seconds))) { t => + t shouldBe 0 + (Seq(root) ++ tasks).foreach { task => + //println(s"checking status for ${task.name}") + checkStatus(graphExecutor, task, SucceededExecution) + } + } + } + } + } + + // TODO: add relevant tests from TaskManagerTest + // Execution + // - handle a few thousand tasks + // - set the submission, start, and end dates correctly for Pipelines + // - get the submission, start, and end dates correctly for a Pipeline within a Pipeline + // Cycles + // - detect a task that has cycles in in its dependencies + // - detect a cycle in the graph introduced by pipeline.build() + // Retries + // - retry a task once and its attempt index is updated; the task succeeds the second time + // - retry a task N times and it succeeds on attempt 1 < M <= N + // - retry a task N times and it fails all attempts + // - the onComplete method should always be run!!! + // - run a task that fails its onComplete method, is retried, where it modifies the onComplete method return value, and succeeds + // - run a task that fails its onComplete method, whereby it changes its args to empty, and succeeds + // - run a task, that its onComplete method mutates its args and return value based on the attempt index + // + +} diff --git a/core/src/test/scala/dagr/core/execsystem2/GraphExecutorUnitSpec.scala b/core/src/test/scala/dagr/core/execsystem2/GraphExecutorUnitSpec.scala new file mode 100644 index 00000000..cf392534 --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/GraphExecutorUnitSpec.scala @@ -0,0 +1,200 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import java.nio.file.Files +import java.time.Instant + +import com.fulcrumgenomics.commons.CommonsDef.DirPath +import dagr.core.FutureUnitSpec +import dagr.core.exec.ResourceSet +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.execsystem2.local.LocalTaskExecutor +import dagr.core.tasksystem.Task.{TimePoint, TaskInfo => RootTaskInfo} +import dagr.core.tasksystem.{Pipeline, ShellCommand, Task, UnitTask} + +import scala.concurrent.Future +import scala.concurrent.duration.Duration + +/** Various methods to help test [[GraphExecutor]]. The various methods use [[UnitTask]] and [[Pipeline]] and were + * written with a [[LocalTaskExecutor]] in mind. */ +private[execsystem2] trait GraphExecutorUnitSpec extends FutureUnitSpec { + + protected def scriptsDirectory: DirPath = { + val dir = Files.createTempDirectory("GraphExecutorTest.scripts") + dir.toFile.deleteOnExit() + dir + } + + protected def logDirectory: DirPath = { + val dir = Files.createTempDirectory("GraphExecutorTest.scripts") + dir.toFile.deleteOnExit() + dir + } + + protected def taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) { + //override def checkTaskCanBeScheduled(task: UnitTask): Option[Future[Future[UnitTask]]] = None + } + + protected def graphExecutor: GraphExecutorImpl[UnitTask] = { + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + } + + protected def graphExecutorSubmissionException: GraphExecutorImpl[UnitTask] = { + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) { + override def execute(task: UnitTask): Future[Future[UnitTask]] = throw new IllegalArgumentException + } + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + } + + protected def graphExecutorFailedSubmission: GraphExecutorImpl[UnitTask] = { + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) { + override def execute(task: UnitTask): Future[Future[UnitTask]] = Future.failed(new IllegalArgumentException) + } + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + } + + protected def graphExecutorFailedExecution: GraphExecutorImpl[UnitTask] = { + val taskExecutor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) { + override def execute(task: UnitTask): Future[Future[UnitTask]] = Future { Future.failed(new IllegalArgumentException) } + } + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=DependencyGraph()) + } + + protected def graphExecutorWithDependencyGraph(dependencyGraph: DependencyGraph): GraphExecutorImpl[UnitTask] = { + new GraphExecutorImpl(taskExecutor=taskExecutor, dependencyGraph=dependencyGraph) + } + + protected def graphAndExecutor(task: Task): (DependencyGraph, GraphExecutorImpl[UnitTask]) = { + val dependencyGraph = DependencyGraph() + val graphExecutor = graphExecutorWithDependencyGraph(dependencyGraph=dependencyGraph) + require(dependencyGraph.add(task), "Task was already added: WTF") + GraphExecutorImplTest.updateMetadata(graphExecutor, task, Pending) + (dependencyGraph, graphExecutor) + } + + /** Creates a task that requires no resources and completes immediately. */ + protected def successfulTask: UnitTask = new PromiseTask(Duration.Zero, ResourceSet.empty) with UnitTask withName "successful-task" + + /** Creates a task that has an infinite amount of resources. */ + protected def infiniteResourcesTask: UnitTask = new PromiseTask(Duration.Inf, ResourceSet.Inf) with UnitTask withName "infinite-resources-task" + + /** Creates a task that requires no resources but never halts. */ + protected def infiniteDurationTask: UnitTask = new PromiseTask(Duration.Inf, ResourceSet.empty) with UnitTask withName "infinite-duration-task" + + /** Creates a task that has an infinite amount of resources and never halts. */ + protected def failureTask: UnitTask = new PromiseTask(Duration.Inf, ResourceSet.Inf) with UnitTask withName "failure-task" + + /** A simple pipeline with 1 ==> (2 :: 3) ==> 4. */ + protected def pipeline: Pipeline = new Pipeline() { + withName("pipeline") + def task(i: Int): UnitTask = successfulTask withName s"$name-$i" + override def build(): Unit = { + root ==> task(0) ==> (task(1) :: task(2)) ==> task(3) + } + } + + /** A pipeline with one task. */ + protected def pipelineOneTask: Pipeline = new Pipeline() { + withName("pipeline") + override def build(): Unit = { + root ==> (successfulTask withName s"${this.name}-task") + } + } + + /** A pipeline that fails: OK ==> (FAIL :: OK) ==> OK. */ + protected trait PipelineFailureTrait extends Pipeline { + def failTask: UnitTask + def okTasks: Seq[UnitTask] + } + protected def pipelineFailure: PipelineFailureTrait = new PipelineFailureTrait() { + withName("pipeline-failure") + def task(i: Int): UnitTask = successfulTask withName s"ok-task-$i" + val failTask: UnitTask = new ShellCommand("exit", "1") withName "fail-task" + val okTasks: Seq[UnitTask] = Seq(task(0), task(1), task(2)) + override def build(): Unit = { + root ==> okTasks.head ==> (failTask :: okTasks(1)) ==> okTasks.last + } + } + + protected def checkStatus(graphExecutor: GraphExecutor[UnitTask], task: Task, status: TaskStatus): RootTaskInfo = { + graphExecutor.contains(task) shouldBe true + val info = task.taskInfo + info.status shouldBe status + info + } + + protected def checkInfo(graphExecutor: GraphExecutor[UnitTask], task: Task, statuses: Seq[TaskStatus], attempts: Int = 1): Unit = { + val info = checkStatus(graphExecutor, task, statuses.last) + checkTimePoints(timePoints=info.timePoints, statuses=statuses) + info.attempts shouldBe attempts + } + + protected def checkPipelineFailure(pipeline: PipelineFailureTrait, graphExecutor: GraphExecutor[UnitTask]): Unit = { + + // root + { + checkInfo(graphExecutor=graphExecutor, task=pipeline, statuses=Seq(Pending, Queued, Running, FailedExecution)) + } + + // task1 + { + val task = pipeline.okTasks.head + checkInfo(graphExecutor=graphExecutor, task=task, statuses=Seq(Pending, Queued, Submitted, Running, SucceededExecution)) + } + + // task2 + { + val task = pipeline.okTasks(1) + checkInfo(graphExecutor=graphExecutor, task=task, statuses=Seq(Pending, Queued, Submitted, Running, SucceededExecution)) + } + + // task3 + { + val task = pipeline.okTasks.last + checkInfo(graphExecutor=graphExecutor, task=task, statuses=Seq(Pending)) + } + + // failTask + { + val task = pipeline.failTask + checkInfo(graphExecutor=graphExecutor, task=task, statuses=Seq(Pending, Queued, Submitted, Running, FailedExecution)) + } + } + + /** Checks that the instants are in increasing order. */ + protected def checkInstants(instants: Traversable[Instant]): Unit = { + instants.toSeq should contain theSameElementsInOrderAs instants.toSeq.sorted + } + + /** Checks that the time points contains the same elements in order for the given statuses, and that the instants + * are in increasing order of time ordered by the input list statuses .*/ + protected def checkTimePoints(timePoints: Traversable[TimePoint], statuses: Seq[TaskStatus]): Unit = { + timePoints.map(_.status) should contain theSameElementsAs statuses + checkInstants(timePoints.map(_.instant)) + } + +} diff --git a/core/src/test/scala/dagr/core/execsystem2/PromiseTask.scala b/core/src/test/scala/dagr/core/execsystem2/PromiseTask.scala new file mode 100644 index 00000000..a045189e --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/PromiseTask.scala @@ -0,0 +1,54 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import dagr.core.exec.ResourceSet +import dagr.core.tasksystem.SimpleInJvmTask + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Promise, TimeoutException} + +object PromiseTask { + def apply(duration: Duration = Duration.Inf, + resourceSet: ResourceSet = ResourceSet.Inf) + (implicit ex: ExecutionContext): PromiseTask = { + new PromiseTask(duration=duration, resourceSet=resourceSet) + } +} + +/** A task that does not complete until the promise is completed. */ +class PromiseTask(duration: Duration = Duration.Inf, + resourceSet: ResourceSet = ResourceSet.Inf) + (implicit ex: ExecutionContext) extends SimpleInJvmTask { + val promise: Promise[Int] = Promise[Int]() + requires(resourceSet) + override def run() = try { + Await.result(promise.future, duration) + } catch { + case e: TimeoutException => this.promise.success(-1) + } +} + diff --git a/core/src/test/scala/dagr/core/execsystem2/TaskInfoTest.scala b/core/src/test/scala/dagr/core/execsystem2/TaskInfoTest.scala new file mode 100644 index 00000000..a388920e --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/TaskInfoTest.scala @@ -0,0 +1,58 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import java.time.Instant + +import dagr.core.UnitSpec +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.tasksystem.{NoOpInJvmTask, Task} +import org.scalatest.OptionValues + +class TaskInfoTest extends UnitSpec with OptionValues { + private def task: Task = new NoOpInJvmTask("name") + + "TaskInfo.submissionDate" should "be the latest instant of Pending" in { + val info = new TaskInfo(task=task, initStatus=Queued) + val instant = Instant.now() + info.update(Pending, instant) + info.submissionDate.value shouldBe instant + } + + "TaskInfo.startDate" should "be the latest instant of Running" in { + val info = new TaskInfo(task=task, initStatus=Queued) + val instant = Instant.now() + info.update(Running, instant) + info.startDate.value shouldBe instant + } + + "TaskInfo.endDate" should "be the latest instant of Completed" in { + val info = new TaskInfo(task=task, initStatus=Queued) + val instant = Instant.now() + info.update(SucceededExecution, instant) + info.endDate.value shouldBe instant + } +} diff --git a/core/src/test/scala/dagr/core/execsystem2/TopLikeStatusReporterTest.scala b/core/src/test/scala/dagr/core/execsystem2/TopLikeStatusReporterTest.scala new file mode 100644 index 00000000..686fc756 --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/TopLikeStatusReporterTest.scala @@ -0,0 +1,103 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2 + +import java.io.{ByteArrayOutputStream, PrintStream} + +import com.fulcrumgenomics.commons.util.Logger +import dagr.core.FutureUnitSpec +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.execsystem2.local.LocalTaskExecutor +import dagr.core.tasksystem.NoOpInJvmTask + +class TopLikeStatusReporterTest extends FutureUnitSpec { + + private def toLoggerOutputStream: ByteArrayOutputStream = { + val loggerOutputStream = new ByteArrayOutputStream() + val loggerPrintStream = new PrintStream(loggerOutputStream) + Logger.out = loggerPrintStream + loggerOutputStream + } + + "TopLikeStatusReporter" should "have predicates for the task statuses" in { + val task = new NoOpInJvmTask("name") + val taskExecutor = new LocalTaskExecutor() + val exeecutor = GraphExecutor(taskExecutor) + val reporter = new TopLikeStatusReporter( + systemResources = taskExecutor.resources, + loggerOut = Some(toLoggerOutputStream), + print = s => Unit + ) + + exeecutor.withLogger(reporter) + + new TaskInfo(task, Pending) + + // running + task.taskInfo.status = Running + reporter.running(task) shouldBe true + task.taskInfo.status = FailedExecution + reporter.running(task) shouldBe false + + // queued + Seq(Queued, Submitted).foreach { status => + task.taskInfo.status = status + reporter.queued(task) shouldBe true + } + task.taskInfo.status = FailedExecution + reporter.running(task) shouldBe false + + // failed + completed + Seq(FailedToBuild, FailedSubmission, FailedExecution, FailedOnComplete, FailedUnknown).foreach { status => + task.taskInfo.status = status + reporter.failed(task) shouldBe true + reporter.completed(task) shouldBe true + } + task.taskInfo.status = Pending + reporter.failed(task) shouldBe false + reporter.completed(task) shouldBe false + + // succeeded + task.taskInfo.status = SucceededExecution + reporter.succeeded(task) shouldBe true + task.taskInfo.status = Pending + reporter.succeeded(task) shouldBe false + + // completed + Seq(SucceededExecution, ManuallySucceeded).foreach { status => + task.taskInfo.status = status + reporter.completed(task) shouldBe true + } + task.taskInfo.status = Pending + reporter.completed(task) shouldBe false + + // pending + task.taskInfo.status = Pending + reporter.pending(task) shouldBe true + task.taskInfo.status = SucceededExecution + reporter.pending(task) shouldBe false + } +} diff --git a/core/src/test/scala/dagr/core/execsystem2/local/LocalTaskExecutorTest.scala b/core/src/test/scala/dagr/core/execsystem2/local/LocalTaskExecutorTest.scala new file mode 100644 index 00000000..008ad706 --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/local/LocalTaskExecutorTest.scala @@ -0,0 +1,271 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2.local + +import java.nio.file.Files + +import com.fulcrumgenomics.commons.CommonsDef.DirPath +import dagr.core.FutureUnitSpec +import dagr.core.exec.{Cores, ResourceSet} +import dagr.core.execsystem2.TaskStatus._ +import dagr.core.execsystem2._ +import dagr.core.tasksystem._ + +import scala.concurrent.duration.Duration +import scala.concurrent.{CancellationException, Future} + +class LocalTaskExecutorTest extends FutureUnitSpec { + + private def scriptsDirectory: DirPath = { + val dir = Files.createTempDirectory("LocalTaskExecutorTest.scripts") + dir.toFile.deleteOnExit() + dir + } + + private def logDirectory: DirPath = { + val dir = Files.createTempDirectory("LocalTaskExecutorTest.scripts") + dir.toFile.deleteOnExit() + dir + } + + private def executor = new LocalTaskExecutor(scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) + + private def info(task: UnitTask): task.type = { + new TaskInfo(task=task, initStatus=Queued) + task + } + + "LocalTaskExecutor.execute" should "return a failure if the task cannot be scheduled (not enough resources)" in { + val executor = this.executor + val task = info(PromiseTask()) + val future: Future[Future[UnitTask]] = executor.execute(task) + + whenReady(future.failed) { t => + executor.contains(task) shouldBe false + } + } + + it should "run a task to completion" in { + val executor = this.executor + val task: PromiseTask = info(PromiseTask(resourceSet=ResourceSet.empty)) + val future: Future[Future[UnitTask]] = executor.execute(task) + + whenReady(future) { execFuture: Future[UnitTask] => + // executing but waiting on the promise to complete + executor.contains(task) shouldBe true + executor.waiting(task) shouldBe false + executor.running(task) shouldBe true + + // complete the promise + task.promise.success(42) + + whenReady(execFuture) { t => + executor.contains(t) shouldBe false + } + } + } + + it should "run a task to completion a second time" in { + val executor = this.executor + val task: PromiseTask = info(PromiseTask(duration=Duration.Zero, resourceSet=ResourceSet.empty)) + val future: Future[Future[UnitTask]] = executor.execute(task) + + whenReady(future) { execFuture: Future[UnitTask] => + whenReady(execFuture) { t => + executor.contains(t) shouldBe false + } + } + } + + "LocalTaskExecutor.kill" should "kill task that is not tracked and return None" in { + this.executor.kill(PromiseTask()) shouldBe None + } + + it should "kill task that has not been scheduled and return Some(false)" in { + val executor = this.executor + + val allTheMemory = ResourceSet(Cores(1), LocalTaskExecutorDefaults.defaultSystemResources.jvmMemory) + + // this task takes all the resources + val blockingTask: PromiseTask = info(PromiseTask( + duration=Duration.Inf, + resourceSet=allTheMemory + )) withName "blockingTask" + + // this task takes one core, but depends on the first + val child: PromiseTask = info(PromiseTask( + duration=Duration.Zero, + resourceSet=allTheMemory + )) withName "child" + + val blockingFuture = executor.execute(blockingTask) + val childFuture = executor.execute(child) + + whenReady(blockingFuture) { execFuture => + + // executing but waiting on the promise to complete + executor.contains(blockingTask) shouldBe true + executor.waiting(blockingTask) shouldBe false + executor.running(blockingTask) shouldBe true + + // not yet scheduled + childFuture.isCompleted shouldBe false + executor.contains(child) shouldBe true + executor.waiting(child) shouldBe true + executor.running(child) shouldBe false + + // kill the child task + executor.kill(child).value shouldBe false + executor.contains(child) shouldBe false + + whenReady(childFuture.failed) { thr: Throwable => + thr shouldBe a[CancellationException] + } + + // complete the promise + blockingTask.promise.success(42) + + whenReady(execFuture) { t => + executor.contains(t) shouldBe false + } + } + } + + it should "kill a running task and return Some(true)" in { + val executor = this.executor + val task: PromiseTask = info(PromiseTask(resourceSet=ResourceSet.empty)) + val future: Future[Future[UnitTask]] = executor.execute(task) + + whenReady(future) { _ => + // executing but waiting on the promise to complete + executor.contains(task) shouldBe true + executor.waiting(task) shouldBe false + executor.running(task) shouldBe true + + executor.kill(task).value shouldBe true + } + } + + it should "kill a completed task and return None" in { + val executor = this.executor + val task: PromiseTask = info(PromiseTask(resourceSet=ResourceSet.empty)) + val future: Future[Future[UnitTask]] = executor.execute(task) + + whenReady(future) { execFuture: Future[UnitTask] => + // executing but waiting on the promise to complete + executor.contains(task) shouldBe true + executor.waiting(task) shouldBe false + executor.running(task) shouldBe true + + // complete the promise + task.promise.success(42) + + whenReady(execFuture) { t: UnitTask => + executor.kill(task) shouldBe None + executor.contains(t) shouldBe false + } + } + } + + it should "kill a completed task and return None a second time" in { + val executor = this.executor + val task: PromiseTask = info(PromiseTask(duration=Duration.Zero, resourceSet=ResourceSet.empty)) + val future: Future[Future[UnitTask]] = executor.execute(task) + + whenReady(future) { execFuture: Future[UnitTask] => + whenReady(execFuture) { t: UnitTask => + executor.kill(task) shouldBe None + executor.contains(t) shouldBe false + } + } + } + + // NB: needs the LocalTaskExecutor to execute the onComplete method prior to scheduling another task! + /* + it should "not run tasks concurrently with more Cores than are defined in the system" in { + import scala.concurrent.blocking + + val systemCores = 4 + var allocatedCores = 0 + var maxAllocatedCores = 0 + val lock = new Object + var nameIndex = 0 + + // A task that would like 1-8 cores each + class HungryTask extends ProcessTask { + var coresGiven = 0 + override def args: Seq[Any] = "exit" :: "0" :: Nil + + name = lock.synchronized { yieldAndThen(s"$name-$nameIndex")(nameIndex + 1) } + + info(this) + + override def pickResources(availableResources: ResourceSet): Option[ResourceSet] = { + (systemCores*2 to 1 by -1) + .map{ c => ResourceSet(Cores(c), Memory("1g")) } + .find(rs => availableResources.subset(rs).isDefined) + } + + override def applyResources(resources: ResourceSet): Unit = { + coresGiven = resources.cores.toInt + blocking { + lock.synchronized { + allocatedCores += coresGiven + maxAllocatedCores = Math.max(maxAllocatedCores, allocatedCores) + } + } + } + + override def onComplete(exitCode: Int): Boolean = { + blocking { + lock.synchronized { + allocatedCores -= coresGiven + } + } + super.onComplete(exitCode) + } + } + + val resources = SystemResources(systemCores, Resource.parseSizeToBytes("8g").toLong, 0.toLong) + + val executor = new LocalTaskExecutor(systemResources=resources, scriptsDirectory=Some(scriptsDirectory), logDirectory=Some(logDirectory)) + + Seq(new HungryTask, new HungryTask, new HungryTask).map { task => + executor.execute(task.asInstanceOf[UnitTask]) + } foreach { future => + whenReady(future.flatMap(identity)) { task: UnitTask => + executor.contains(task) shouldBe false + task.onComplete(0) + // TODO: check the amount of resources it ran with + } + } + + maxAllocatedCores should be <= systemCores + } + */ + + // TODO: test that both tasks that fail or succeed are removed +} diff --git a/core/src/test/scala/dagr/core/execsystem2/local/LocalTaskRunnerTest.scala b/core/src/test/scala/dagr/core/execsystem2/local/LocalTaskRunnerTest.scala new file mode 100644 index 00000000..8b05afe3 --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/local/LocalTaskRunnerTest.scala @@ -0,0 +1,211 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2.local + +import java.nio.file.{Files, Path} + +import com.fulcrumgenomics.commons.CommonsDef.FilePath +import dagr.core.FutureUnitSpec +import dagr.core.exec.ResourceSet +import dagr.core.execsystem2.TaskInfo +import dagr.core.execsystem2.TaskStatus.Running +import dagr.core.tasksystem._ + +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, CancellationException, TimeoutException} + + +class LocalTaskRunnerTest extends FutureUnitSpec { + + /** A task that exits zero and that's it. */ + private class TrivialInJvmTask(exitCode: Int, sleep: Int = 0) extends InJvmTask with FixedResources { + override def inJvmMethod(): Int = { + Thread.sleep(sleep) + exitCode + } + } + + /** A [[dagr.core.tasksystem.ProcessTask]] that throws an exception when trying to build a process builder. */ + private class ProcessBuilderExceptionTask extends ProcessTask with FixedResources { + override def processBuilder(script: Path, logFile: Path, setPipefail: Boolean = true): scala.sys.process.ProcessBuilder = { + throw new IllegalStateException("I failed creating my process builder") + } + override def onComplete(exitCode: Int): Boolean = false + override def args: Seq[Any] = List.empty + } + + private class ProperShellCommand(commands: String*) extends ShellCommand(commands:_*) { + this.quoteIfNecessary = false + } + + private trait FailOnCompleteTask extends UnitTask { + override def onComplete(exitCode: Int): Boolean = false + } + + private trait OnCompleteIsOppositeExitCodeTask extends UnitTask { + override def onComplete(exitCode: Int): Boolean = 0 != exitCode + } + + private class InJvmExceptionTask extends SimpleInJvmTask { + override def run() = throw new IllegalStateException("I throw exceptions") + } + + /** A path to a script file. */ + private def script: FilePath = Files.createTempFile("TaskRunnerTest", ".sh") + + /** A path to a log file. */ + private def log: FilePath = Files.createTempFile("TaskRunnerTest", ".log") + + /** Creates a task runner. */ + private def taskRunner(task: UnitTask): LocalTaskRunner = { + new TaskInfo(task=task, initStatus=Running) + task.taskInfo.script = Some(script) + task.taskInfo.log = Some(log) + LocalTaskRunner(task) + } + + /** Create a trivial in JVM task and build it. */ + private def trivialInJvmTask(exitCode: Int): InJvmTask = new TrivialInJvmTask(exitCode = exitCode).getTasks.head + + /** Create a trivial in process task and build it. */ + private def proccessTask(argv: List[String]): ProcessTask = new ShellCommand(argv:_*).withName(argv.mkString(" ")).getTasks.head + + /** Create a in JVM or process task that will exit with zero or one, and then build it. */ + private def testTask(doInJvmTask: Boolean=true, exitOk: Boolean=true, onCompleteSuccessful: Boolean=true, sleep: Int=0): UnitTask = { + val exitCode = if (exitOk) 0 else 1 + val task = { + if (doInJvmTask) { + if (onCompleteSuccessful) new TrivialInJvmTask(exitCode=exitCode, sleep=sleep) + else new TrivialInJvmTask(exitCode=exitCode, sleep=sleep) with FailOnCompleteTask + } + else { + val argv = Seq("sleep", sleep, ";", "exit", exitCode).map(_.toString) + if (onCompleteSuccessful) new ProperShellCommand(argv:_*) + else new ProperShellCommand(argv:_*) with FailOnCompleteTask + } + } + task.withName(s"Exit $exitCode").getTasks.head + } + + /** Build a task and task runner. */ + private def taskRunner(doInJvmTask: Boolean, exitOk: Boolean=true, onCompleteSuccessful: Boolean=true, sleep: Int=0): LocalTaskRunner = { + val task = testTask(doInJvmTask=doInJvmTask, exitOk=exitOk, onCompleteSuccessful=onCompleteSuccessful, sleep=sleep) + taskRunner(task=task) + } + + /** Call the execute() method for at most the duration, and return a future. */ + private def executeRunner(runner: LocalTaskRunner, duration: Duration = Duration("60s")): UnitTask = { + Await.result(runner.execute(), duration) + } + + private def requireRunner(runner: LocalTaskRunner, + exitCode: Int= 0, + onCompleteSuccessful: Option[Boolean] = Some(true), + thrown: Boolean = false): Unit = { + runner.exitCode.value shouldBe exitCode + runner.throwable.nonEmpty shouldBe thrown + } + + Seq(true, false).foreach { doInJvmTask => + val doInJvmTaskMsg = if (doInJvmTask) "in the JVM" else "in a Process" + + s"LocalTaskRunner ($doInJvmTaskMsg)" should "run with exit 0 and succeed" in { + val runner = taskRunner(doInJvmTask=doInJvmTask) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner) + } + + it should "run with exit 1 and fail" in { + val runner = taskRunner(doInJvmTask=doInJvmTask, exitOk=false) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner, exitCode=1) + } + + it should "have a task fail its onComplete method" in { + val runner = taskRunner(doInJvmTask=doInJvmTask, onCompleteSuccessful=false) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner, onCompleteSuccessful=Some(false)) + } + + it should "have a task that fail on its onComplete method only if the exit code is zero" in { + { + val runner = taskRunner(doInJvmTask=doInJvmTask, onCompleteSuccessful=false) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner, onCompleteSuccessful=Some(false)) + } + { + val runner = taskRunner(doInJvmTask=doInJvmTask, exitOk=false) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner, exitCode=1) + } + } + + it should "have a special exit code when a task gets interrupted" in { + val runner = taskRunner(doInJvmTask=doInJvmTask, sleep=100000) // sleep forever + val future = runner.execute() + future.isCompleted shouldBe false + runner.interrupt() shouldBe None + future.isCompleted shouldBe true + runner.join(Duration("1s")) shouldBe None // interrupted! + runner.interrupted() shouldBe true + requireRunner(runner=runner, exitCode=LocalTaskRunner.InterruptedExitCode, thrown=true) + runner.throwable.value shouldBe a[CancellationException] + whenReady(future.failed) { thr => + thr shouldBe a[CancellationException] + } + } + + it should "have a special exit code when a task does not join in the specified time" in { + val runner = taskRunner(doInJvmTask=doInJvmTask, sleep=100000) // sleep forever + val future = runner.execute() + future.isCompleted shouldBe false + runner.join(Duration("1s")) shouldBe None // timeout! + requireRunner(runner=runner, exitCode=LocalTaskRunner.InterruptedExitCode, thrown=true) + runner.throwable.value shouldBe a[TimeoutException] + } + } + + "LocalTaskRunner" should "have an non-zero exit code when an InJvmTask that throws an Exception" in { + val task = new InJvmExceptionTask + val runner = taskRunner(task) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner, exitCode=1) + } + + it should "get a non-zero exit code for a task that fails during getProcessBuilder by throwing an exception" in { + val task = new ProcessBuilderExceptionTask + val runner = taskRunner(task) + executeRunner(runner) shouldBe runner.task + requireRunner(runner=runner, exitCode=1, onCompleteSuccessful=Some(false), thrown=true) + } + + it should "fail to run a UnitTask that is not either an InJvmTask or a ProcessTask task" in { + val task = new UnitTask { + override def pickResources(availableResources: ResourceSet) = None + } + an[RuntimeException] should be thrownBy taskRunner(task) + } +} diff --git a/core/src/test/scala/dagr/core/execsystem2/util/InterruptableFutureTest.scala b/core/src/test/scala/dagr/core/execsystem2/util/InterruptableFutureTest.scala new file mode 100644 index 00000000..774e9e8c --- /dev/null +++ b/core/src/test/scala/dagr/core/execsystem2/util/InterruptableFutureTest.scala @@ -0,0 +1,165 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.core.execsystem2.util + +import java.util.concurrent.CancellationException + +import dagr.core.FutureUnitSpec +import dagr.core.execsystem2.util.InterruptableFuture.Interruptable + +import scala.concurrent.Future +import scala.util.Success + + +class InterruptableFutureTest extends FutureUnitSpec { + + + // NB: cannot ensure that the code block executes after interrupt() to test this + /* + "InterruptableFuture.apply" should "not invoke the work block of code if it was interrupted before it was invoked" in { + } + */ + + // NB: cannot ensure that the code block executes after interrupt() to test this + /* + "InterruptableFuture" should "ignore the result of the future an execute anyway" in { + } + */ + + "InterruptableFuture" should "support being interrupted before the work body is completed" in { + val future: InterruptableFuture[Int] = new InterruptableFuture[Int](fun = _ => { Thread.sleep(1000); 2 }) + + future.interrupt() shouldBe None + + future.future + + whenReady(future.future.failed) { result: Throwable => + result shouldBe a [CancellationException] + future.interrupted shouldBe true + } + } + + it should "complete successfully if it is never interrupted" in { + val future: InterruptableFuture[Int] = new InterruptableFuture[Int](fun = _ => 2) + whenReady(future.future) { _ shouldBe 2 } + } + + it should "support returning a Future[T]" in { + val future: InterruptableFuture[Future[Int]] = new InterruptableFuture[Future[Int]](fun = _ => Future { + 2 + }) + whenReady(future.future) { result: Future[Int] => + whenReady(result) { _ shouldBe 2 } + } + } + + it should "support interrupting before constructing a future type" in { + var internalFuture: Option[Future[Int]] = None + val fun: () => Future[Int] = () => { + Thread.sleep(100000) + val future = Future { 2 } + internalFuture = Some(future) + future + } + + val future: InterruptableFuture[Future[Int]] = InterruptableFuture[Future[Int]](work=fun()) + + future.interrupt() shouldBe None + + whenReady(future.future.failed) { result: Throwable => + result shouldBe a [CancellationException] + future.interrupted shouldBe true + } + + internalFuture shouldBe None + } + + it should "support interrupting when given a future to wrap" in { + val futureToWrap: Future[Int] = Future { Thread.sleep(100000); 2 } + val interruptableFuture: InterruptableFuture[Int] = InterruptableFuture(futureToWrap) + val futureToUse: Future[Int] = interruptableFuture.future + + interruptableFuture.interrupt() shouldBe None + + whenReady(interruptableFuture.future.failed) { result: Throwable => + result shouldBe a [CancellationException] + interruptableFuture.interrupted shouldBe true + } + + futureToWrap.isCompleted shouldBe false + + whenReady(futureToUse.failed) { result: Throwable => + result shouldBe a [CancellationException] + } + } + + "InterruptableFuture.apply" should "complete successfully if it is never interrupted" in { + val work: () => Int = () => 2 + val future = InterruptableFuture[Int] { work() } + whenReady(future.future) { _ shouldBe 2 } + } + + "InterruptableFuture.Interruptable" should "convert a Future[T] to an InterruptableFuture[T]" in { + val interruptableFuture = Future { Thread.sleep(100000); 2 } interruptable() + + interruptableFuture.interrupt() shouldBe None + + whenReady(interruptableFuture.future.failed) { result: Throwable => + result shouldBe a [CancellationException] + interruptableFuture.interrupted shouldBe true + } + + whenReady(interruptableFuture.future.failed) { result: Throwable => + result shouldBe a [CancellationException] + } + } + + "InterruptableFuture.interrupted" should "be false if the work body completes before being interrupted" in { + val future: InterruptableFuture[Int] = new InterruptableFuture[Int](fun = _ => 2 ) + + whenReady(future.future) { result: Int => + future.interrupt().value shouldBe 2 + future.interrupted shouldBe false + result shouldBe 2 + } + } + + it should "true even if it is interrupted multiple times" in { + val future: InterruptableFuture[Int] = new InterruptableFuture[Int](fun = _ => 2 ) + + whenReady(future.future) { result => + Range(1, 10).foreach { _ => + future.interrupt().value shouldBe 2 + future.interrupted shouldBe false + + future.future.isCompleted shouldBe true + future.future.value.isDefined shouldBe true + future.future.value.value shouldBe Success(2) + } + result shouldBe 2 + } + } +} diff --git a/pipelines/src/main/scala/dagr/pipelines/SleepyPipeline.scala b/pipelines/src/main/scala/dagr/pipelines/SleepyPipeline.scala new file mode 100644 index 00000000..94c3fa08 --- /dev/null +++ b/pipelines/src/main/scala/dagr/pipelines/SleepyPipeline.scala @@ -0,0 +1,89 @@ +/* + * The MIT License + * + * Copyright (c) 2017 Fulcrum Genomics LLC + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in + * all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + * THE SOFTWARE. + * + */ + +package dagr.pipelines + +import dagr.core.cmdline.Pipelines +import dagr.core.tasksystem._ +import com.fulcrumgenomics.sopt.{arg, clp} +import dagr.core.exec.{Cores, Memory, ResourceSet} + +private trait GreedyResourcePicking extends UnitTask { + override def pickResources(availableResources: ResourceSet): Option[ResourceSet] = { + val mem = Memory("1g") + (8 to 1 by -1).map(c => ResourceSet(Cores(c), mem)).find(rs => availableResources.subset(rs).isDefined) + } +} + +private class SleepProcessTask(seconds: Int = 1) extends ProcessTask with GreedyResourcePicking { + override def args: Seq[Any] = "sleep" :: s"$seconds" :: Nil +} + +private class SleepInJvmTask(seconds: Int = 1) extends SimpleInJvmTask with GreedyResourcePicking { + def run(): Unit = Thread.sleep(seconds * 1000) +} + +/** + * Very simple example pipeline that creates random tasks and dependencies + */ +@clp(description="A bunch of sleep tasks.", group = classOf[Pipelines]) +class SleepyPipeline +( @arg(flag='j', doc="Use JVM tasks") val jvmTask: Boolean = false, + @arg(flag='n', doc="The number of tasks to create") val numTasks: Int = 100, + @arg(flag='p', doc="The probability of creating a dependency") val dependencyProbability: Double = 0.1, + @arg(flag='s', doc="The seed for the random number generator") val seed: Option[Long] = None +) extends Pipeline { + + private def toATask: () => SleepProcessTask = () => new SleepProcessTask + private def toBTask: () => SleepInJvmTask = () => new SleepInJvmTask + private val toTask = if (jvmTask) toBTask else toATask + private val taskType = if (jvmTask) "JVM" else "Shell" + + override def build(): Unit = { + // create the tasks + val tasks: Seq[Task] = for (i <- 0 to numTasks) yield toTask() withName s"task-$i" + + // make them depend on previous tasks + var rootTasks = Seq.range(start=0, numTasks).toSet + val randomNumberGenerator = seed match { + case Some(s) => new scala.util.Random(s) + case None => scala.util.Random + } + for (i <- 0 until numTasks) { + for (j <- 0 until i) { + if (randomNumberGenerator.nextFloat < dependencyProbability) { + logger.info(s"Task $i will depend on task $j") + tasks(j) ==> tasks(i) + rootTasks = rootTasks - i + } + } + } + + rootTasks.foreach { i => + root ==> tasks(i) + } + } +} +