You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gr...@apache.org on 2018/04/06 06:32:59 UTC

[05/13] mesos git commit: Refactored agent task launch for better composition [1/2].

Refactored agent task launch for better composition [1/2].

This helps to encapsulate a task launch into a single
future which will be useful when enforcing the task
launch order.

This patch also consolidated the error handling code
in the task launch path.

Affected tests are also updated.

Review: https://reviews.apache.org/r/66126/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/adfdffdb
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/adfdffdb
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/adfdffdb

Branch: refs/heads/master
Commit: adfdffdbe0ade15b3e7290aba0e6d8454a9383a2
Parents: cc57765
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Thu Apr 5 17:44:12 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Thu Apr 5 17:56:04 2018 -0700

----------------------------------------------------------------------
 src/slave/slave.cpp       | 190 +++++++++++++++++++++--------------------
 src/slave/slave.hpp       |   9 +-
 src/tests/mock_slave.cpp  |   8 +-
 src/tests/mock_slave.hpp  |   6 +-
 src/tests/slave_tests.cpp |  98 ++++++++++-----------
 5 files changed, 156 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/adfdffdb/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index b178547..7d21ce8 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -2152,22 +2152,88 @@ void Slave::run(
     }
   }
 
+  auto onUnscheduleGCFailure =
+    [=](const Future<list<bool>>& unschedules) -> Future<list<bool>> {
+      LOG(ERROR) << "Failed to unschedule directories scheduled for gc: "
+                 << unschedules.failure();
+
+      Framework* _framework = getFramework(frameworkId);
+      if (_framework == nullptr) {
+        const string error =
+          "Cannot handle unschedule GC failure for " +
+          taskOrTaskGroup(task, taskGroup) + " because the framework " +
+          stringify(frameworkId) + " does not exist";
+
+        LOG(WARNING) << error;
+
+        return Failure(error);
+      }
+
+      // We report TASK_DROPPED to the framework because the task was
+      // never launched. For non-partition-aware frameworks, we report
+      // TASK_LOST for backward compatibility.
+      mesos::TaskState taskState = TASK_DROPPED;
+      if (!protobuf::frameworkHasCapability(
+              frameworkInfo, FrameworkInfo::Capability::PARTITION_AWARE)) {
+        taskState = TASK_LOST;
+      }
+
+      foreach (const TaskInfo& _task, tasks) {
+        _framework->removePendingTask(_task.task_id());
+
+        const StatusUpdate update = protobuf::createStatusUpdate(
+            frameworkId,
+            info.id(),
+            _task.task_id(),
+            taskState,
+            TaskStatus::SOURCE_SLAVE,
+            id::UUID::random(),
+            "Could not launch the task because we failed to unschedule"
+            " directories scheduled for gc",
+            TaskStatus::REASON_GC_ERROR);
+
+        // TODO(vinod): Ensure that the task status update manager
+        // reliably delivers this update. Currently, we don't guarantee
+        // this because removal of the framework causes the status
+        // update manager to stop retrying for its un-acked updates.
+        statusUpdate(update, UPID());
+      }
+
+      if (_framework->idle()) {
+        removeFramework(_framework);
+      }
+
+      return unschedules;
+  };
+
   // Run the task after the unschedules are done.
   collect(unschedules)
-    .onAny(defer(self(),
-                 &Self::_run,
-                 lambda::_1,
-                 frameworkInfo,
-                 executorInfo,
-                 task,
-                 taskGroup,
-                 resourceVersionUuids,
-                 launchExecutor));
+    .repair(defer(self(), onUnscheduleGCFailure))
+    .then(defer(
+        self(),
+        &Self::_run,
+        frameworkInfo,
+        executorInfo,
+        task,
+        taskGroup,
+        resourceVersionUuids,
+        launchExecutor))
+    .recover(defer(self(),
+      [=](const Future<Nothing>& future) -> Future<Nothing> {
+        if (launchExecutor.isSome() && launchExecutor.get()) {
+          // Master expects new executor to be launched for this task launch.
+          // To keep the master executor entries updated, the agent needs to
+          // send 'ExitedExecutorMessage' even though no executor launched.
+          sendExitedExecutorMessage(frameworkId, executorInfo.executor_id());
+        }
+
+        return future;
+      }
+    ));
 }
 
 
-void Slave::_run(
-    const Future<list<bool>>& unschedules,
+Future<Nothing> Slave::_run(
     const FrameworkInfo& frameworkInfo,
     const ExecutorInfo& executorInfo,
     const Option<TaskInfo>& task,
@@ -2192,26 +2258,24 @@ void Slave::_run(
   const FrameworkID& frameworkId = frameworkInfo.id();
   Framework* framework = getFramework(frameworkId);
   if (framework == nullptr) {
-    LOG(WARNING) << "Ignoring running " << taskOrTaskGroup(task, taskGroup)
-                 << " because the framework " << frameworkId
-                 << " does not exist";
+    const string error =
+      "Ignoring running " + taskOrTaskGroup(task, taskGroup) +
+      " because the framework " + stringify(frameworkId) + " does not exist";
 
-    if (launchExecutor.isSome() && launchExecutor.get()) {
-      // Master expects new executor to be launched for this task(s) launch.
-      // To keep the master executor entries updated, the agent needs to send
-      // 'ExitedExecutorMessage' even though no executor launched.
-      sendExitedExecutorMessage(frameworkId, executorInfo.executor_id());
-    }
+    LOG(WARNING) << error;
 
-    return;
+    return Failure(error);
   }
 
   // We don't send a status update here because a terminating
   // framework cannot send acknowledgements.
   if (framework->state == Framework::TERMINATING) {
-    LOG(WARNING) << "Ignoring running " << taskOrTaskGroup(task, taskGroup)
-                 << " of framework " << frameworkId
-                 << " because the framework is terminating";
+    const string error = "Ignoring running " +
+                         taskOrTaskGroup(task, taskGroup) + " of framework " +
+                         stringify(frameworkId) +
+                         " because the framework is terminating";
+
+    LOG(WARNING) << error;
 
     // Although we cannot send a status update in this case, we remove
     // the affected tasks from the pending tasks.
@@ -2223,14 +2287,7 @@ void Slave::_run(
       removeFramework(framework);
     }
 
-    if (launchExecutor.isSome() && launchExecutor.get()) {
-      // Master expects new executor to be launched for this task(s) launch.
-      // To keep the master executor entries updated, the agent needs to send
-      // 'ExitedExecutorMessage' even though no executor launched.
-      sendExitedExecutorMessage(frameworkId, executorInfo.executor_id());
-    }
-
-    return;
+    return Failure(error);
   }
 
   // Ignore the launch if killed in the interim. The invariant here
@@ -2251,69 +2308,14 @@ void Slave::_run(
     << " was partially killed";
 
   if (allRemoved) {
-    LOG(WARNING) << "Ignoring running " << taskOrTaskGroup(task, taskGroup)
-                 << " of framework " << frameworkId
-                 << " because it has been killed in the meantime";
+    const string error = "Ignoring running " +
+                         taskOrTaskGroup(task, taskGroup) + " of framework " +
+                         stringify(frameworkId) +
+                         " because it has been killed in the meantime";
 
-    if (launchExecutor.isSome() && launchExecutor.get()) {
-      // Master expects new executor to be launched for this task(s) launch.
-      // To keep the master executor entries updated, the agent needs to send
-      // 'ExitedExecutorMessage' even though no executor launched.
-      sendExitedExecutorMessage(frameworkId, executorInfo.executor_id());
-    }
+    LOG(WARNING) << error;
 
-    return;
-  }
-
-  CHECK(!unschedules.isDiscarded());
-
-  if (!unschedules.isReady()) {
-    LOG(ERROR) << "Failed to unschedule directories scheduled for gc: "
-               << (unschedules.isFailed() ?
-                   unschedules.failure() : "future discarded");
-
-    // We report TASK_DROPPED to the framework because the task was
-    // never launched. For non-partition-aware frameworks, we report
-    // TASK_LOST for backward compatibility.
-    mesos::TaskState taskState = TASK_DROPPED;
-    if (!protobuf::frameworkHasCapability(
-            frameworkInfo, FrameworkInfo::Capability::PARTITION_AWARE)) {
-      taskState = TASK_LOST;
-    }
-
-    foreach (const TaskInfo& _task, tasks) {
-      framework->removePendingTask(_task.task_id());
-
-      const StatusUpdate update = protobuf::createStatusUpdate(
-          frameworkId,
-          info.id(),
-          _task.task_id(),
-          taskState,
-          TaskStatus::SOURCE_SLAVE,
-          id::UUID::random(),
-          "Could not launch the task because we failed to unschedule"
-          " directories scheduled for gc",
-          TaskStatus::REASON_GC_ERROR);
-
-      // TODO(vinod): Ensure that the task status update manager
-      // reliably delivers this update. Currently, we don't guarantee
-      // this because removal of the framework causes the status
-      // update manager to stop retrying for its un-acked updates.
-      statusUpdate(update, UPID());
-    }
-
-    if (framework->idle()) {
-      removeFramework(framework);
-    }
-
-    if (launchExecutor.isSome() && launchExecutor.get()) {
-      // Master expects new executor to be launched for this task(s) launch.
-      // To keep the master executor entries updated, the agent needs to send
-      // 'ExitedExecutorMessage' even though no executor launched.
-      sendExitedExecutorMessage(frameworkId, executorInfo.executor_id());
-    }
-
-    return;
+    return Failure(error);
   }
 
   // Authorize the task or tasks (as in a task group) to ensure that the
@@ -2338,6 +2340,8 @@ void Slave::_run(
                  taskGroup,
                  resourceVersionUuids,
                  launchExecutor));
+
+  return Nothing();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/adfdffdb/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 37f0361..8159838 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -170,8 +170,13 @@ public:
       const Option<bool>& launchExecutor);
 
   // Made 'virtual' for Slave mocking.
-  virtual void _run(
-      const process::Future<std::list<bool>>& unschedules,
+  //
+  // This function returns a future so that we can encapsulate a task(group)
+  // launch operation (from agent receiving the run message to the completion
+  // of `_run()`) into a single future. This includes all the asynchronous
+  // steps (currently two: unschedule GC and task authorization) prior to the
+  // executor launch.
+  virtual process::Future<Nothing> _run(
       const FrameworkInfo& frameworkInfo,
       const ExecutorInfo& executorInfo,
       const Option<TaskInfo>& task,

http://git-wip-us.apache.org/repos/asf/mesos/blob/adfdffdb/src/tests/mock_slave.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_slave.cpp b/src/tests/mock_slave.cpp
index 8ec55b6..a20b8f5 100644
--- a/src/tests/mock_slave.cpp
+++ b/src/tests/mock_slave.cpp
@@ -128,7 +128,7 @@ MockSlave::MockSlave(
     .WillRepeatedly(Invoke(this, &MockSlave::unmocked____run));
   EXPECT_CALL(*this, runTask(_, _, _, _, _, _, _))
     .WillRepeatedly(Invoke(this, &MockSlave::unmocked_runTask));
-  EXPECT_CALL(*this, _run(_, _, _, _, _, _, _))
+  EXPECT_CALL(*this, _run(_, _, _, _, _, _))
     .WillRepeatedly(Invoke(this, &MockSlave::unmocked__run));
   EXPECT_CALL(*this, __run(_, _, _, _, _, _, _))
     .WillRepeatedly(Invoke(this, &MockSlave::unmocked___run));
@@ -191,8 +191,7 @@ void MockSlave::unmocked_runTask(
 }
 
 
-void MockSlave::unmocked__run(
-    const Future<list<bool>>& unschedules,
+Future<Nothing> MockSlave::unmocked__run(
     const FrameworkInfo& frameworkInfo,
     const ExecutorInfo& executorInfo,
     const Option<TaskInfo>& taskInfo,
@@ -200,8 +199,7 @@ void MockSlave::unmocked__run(
     const std::vector<ResourceVersionUUID>& resourceVersionUuids,
     const Option<bool>& launchExecutor)
 {
-  slave::Slave::_run(
-      unschedules,
+  return slave::Slave::_run(
       frameworkInfo,
       executorInfo,
       taskInfo,

http://git-wip-us.apache.org/repos/asf/mesos/blob/adfdffdb/src/tests/mock_slave.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_slave.hpp b/src/tests/mock_slave.hpp
index 8ec2b65..25a414f 100644
--- a/src/tests/mock_slave.hpp
+++ b/src/tests/mock_slave.hpp
@@ -135,8 +135,7 @@ public:
       const std::vector<ResourceVersionUUID>& resourceVersionUuids,
       const Option<bool>& launchExecutor);
 
-  MOCK_METHOD7(_run, void(
-      const process::Future<std::list<bool>>& unschedules,
+  MOCK_METHOD6(_run, process::Future<Nothing>(
       const FrameworkInfo& frameworkInfo,
       const ExecutorInfo& executorInfo,
       const Option<TaskInfo>& task,
@@ -144,8 +143,7 @@ public:
       const std::vector<ResourceVersionUUID>& resourceVersionUuids,
       const Option<bool>& launchExecutor));
 
-  void unmocked__run(
-      const process::Future<std::list<bool>>& unschedules,
+  process::Future<Nothing> unmocked__run(
       const FrameworkInfo& frameworkInfo,
       const ExecutorInfo& executorInfo,
       const Option<TaskInfo>& task,

http://git-wip-us.apache.org/repos/asf/mesos/blob/adfdffdb/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index caf165e..7626438 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -1847,8 +1847,9 @@ TEST_F(SlaveTest, GetStateTaskGroupPending)
   // unmocked `_run()` method. Instead, we want to do nothing so that tasks
   // remain in the framework's 'pending' list.
   Future<Nothing> _run;
-  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _, _))
-    .WillOnce(FutureSatisfy(&_run));
+  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _))
+    .WillOnce(DoAll(FutureSatisfy(&_run),
+                    Return(Nothing())));
 
   // The executor should not be launched.
   EXPECT_CALL(*executor, connected(_))
@@ -4131,7 +4132,6 @@ TEST_F(SlaveTest, KillTaskBetweenRunTaskParts)
     .WillOnce(Invoke(slave.get()->mock(), &MockSlave::unmocked_runTask));
 
   // Saved arguments from Slave::_run().
-  Future<list<bool>> unschedules;
   FrameworkInfo frameworkInfo;
   ExecutorInfo executorInfo;
   Option<TaskGroupInfo> taskGroup;
@@ -4142,15 +4142,15 @@ TEST_F(SlaveTest, KillTaskBetweenRunTaskParts)
   // later, tie reaching the critical moment when to kill the task to
   // a future.
   Future<Nothing> _run;
-  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _, _))
+  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&_run),
-                    SaveArg<0>(&unschedules),
-                    SaveArg<1>(&frameworkInfo),
-                    SaveArg<2>(&executorInfo),
-                    SaveArg<3>(&task_),
-                    SaveArg<4>(&taskGroup),
-                    SaveArg<5>(&resourceVersionUuids),
-                    SaveArg<6>(&launchExecutor)));
+                    SaveArg<0>(&frameworkInfo),
+                    SaveArg<1>(&executorInfo),
+                    SaveArg<2>(&task_),
+                    SaveArg<3>(&taskGroup),
+                    SaveArg<4>(&resourceVersionUuids),
+                    SaveArg<5>(&launchExecutor),
+                    Return(Nothing())));
 
   driver.launchTasks(offers.get()[0].id(), {task});
 
@@ -4181,7 +4181,6 @@ TEST_F(SlaveTest, KillTaskBetweenRunTaskParts)
 
   Future<Nothing> unmocked__run = process::dispatch(slave.get()->pid, [=] {
     slave.get()->mock()->unmocked__run(
-        unschedules,
         frameworkInfo,
         executorInfo,
         task_,
@@ -4271,7 +4270,6 @@ TEST_F(SlaveTest, KillMultiplePendingTasks)
   // Skip what Slave::_run() normally does, save its arguments for
   // later, tie reaching the critical moment when to kill the task to
   // a future.
-  Future<list<bool>> unschedules1, unschedules2;
   FrameworkInfo frameworkInfo1, frameworkInfo2;
   ExecutorInfo executorInfo1, executorInfo2;
   Option<TaskGroupInfo> taskGroup1, taskGroup2;
@@ -4280,23 +4278,23 @@ TEST_F(SlaveTest, KillMultiplePendingTasks)
   Option<bool> launchExecutor1, launchExecutor2;
 
   Future<Nothing> _run1, _run2;
-  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _, _))
+  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&_run1),
-                    SaveArg<0>(&unschedules1),
-                    SaveArg<1>(&frameworkInfo1),
-                    SaveArg<2>(&executorInfo1),
-                    SaveArg<3>(&task_1),
-                    SaveArg<4>(&taskGroup1),
-                    SaveArg<5>(&resourceVersionUuids1),
-                    SaveArg<6>(&launchExecutor1)))
+                    SaveArg<0>(&frameworkInfo1),
+                    SaveArg<1>(&executorInfo1),
+                    SaveArg<2>(&task_1),
+                    SaveArg<3>(&taskGroup1),
+                    SaveArg<4>(&resourceVersionUuids1),
+                    SaveArg<5>(&launchExecutor1),
+                    Return(Nothing())))
     .WillOnce(DoAll(FutureSatisfy(&_run2),
-                    SaveArg<0>(&unschedules2),
-                    SaveArg<1>(&frameworkInfo2),
-                    SaveArg<2>(&executorInfo2),
-                    SaveArg<3>(&task_2),
-                    SaveArg<4>(&taskGroup2),
-                    SaveArg<5>(&resourceVersionUuids2),
-                    SaveArg<6>(&launchExecutor2)));
+                    SaveArg<0>(&frameworkInfo2),
+                    SaveArg<1>(&executorInfo2),
+                    SaveArg<2>(&task_2),
+                    SaveArg<3>(&taskGroup2),
+                    SaveArg<4>(&resourceVersionUuids2),
+                    SaveArg<5>(&launchExecutor2),
+                    Return(Nothing())));
 
   driver.launchTasks(offers.get()[0].id(), {task1, task2});
 
@@ -4334,7 +4332,6 @@ TEST_F(SlaveTest, KillMultiplePendingTasks)
   // The `__run` continuations should have no effect.
   process::dispatch(slave.get()->pid, [=] {
     slave.get()->mock()->unmocked__run(
-        unschedules1,
         frameworkInfo1,
         executorInfo1,
         task_1,
@@ -4345,7 +4342,6 @@ TEST_F(SlaveTest, KillMultiplePendingTasks)
 
   process::dispatch(slave.get()->pid, [=] {
     slave.get()->mock()->unmocked__run(
-        unschedules2,
         frameworkInfo2,
         executorInfo2,
         task_2,
@@ -4663,7 +4659,6 @@ TEST_F(SlaveTest, RemoveExecutorUponFailedLaunch)
     FUTURE_PROTOBUF(ExitedExecutorMessage(), _, _);
 
   // Saved arguments from `Slave::_run()`.
-  Future<list<bool>> unschedules;
   FrameworkInfo frameworkInfo;
   ExecutorInfo executorInfo_;
   Option<TaskGroupInfo> taskGroup_;
@@ -4671,18 +4666,21 @@ TEST_F(SlaveTest, RemoveExecutorUponFailedLaunch)
   vector<ResourceVersionUUID> resourceVersionUuids;
   Option<bool> launchExecutor;
 
-  // Before launching the executor in `__run`, we kill the task, so that
-  // no executor is launched.
+  // Before launching the executor in `__run`, we pause the continuation
+  // by returning a pending future. We then kill the task and re-dispatch
+  // `_run`. We use its return result to fulfill the pending future and
+  // resume the continuation.
+  Promise<Nothing> promise;
   Future<Nothing> _run;
-  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _, _))
+  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&_run),
-                  SaveArg<0>(&unschedules),
-                  SaveArg<1>(&frameworkInfo),
-                  SaveArg<2>(&executorInfo_),
-                  SaveArg<3>(&task_),
-                  SaveArg<4>(&taskGroup_),
-                  SaveArg<5>(&resourceVersionUuids),
-                  SaveArg<6>(&launchExecutor)));
+                  SaveArg<0>(&frameworkInfo),
+                  SaveArg<1>(&executorInfo_),
+                  SaveArg<2>(&task_),
+                  SaveArg<3>(&taskGroup_),
+                  SaveArg<4>(&resourceVersionUuids),
+                  SaveArg<5>(&launchExecutor),
+                  Return(promise.future())));
 
   Future<Nothing> executorLost;
   EXPECT_CALL(sched, executorLost(&driver, DEFAULT_EXECUTOR_ID, _, _))
@@ -8170,7 +8168,6 @@ TEST_F(SlaveTest, KillTaskGroupBetweenRunTaskParts)
                      &MockSlave::unmocked_runTaskGroup));
 
   // Saved arguments from `Slave::_run()`.
-  Future<list<bool>> unschedules;
   FrameworkInfo frameworkInfo;
   ExecutorInfo executorInfo_;
   Option<TaskGroupInfo> taskGroup_;
@@ -8182,15 +8179,15 @@ TEST_F(SlaveTest, KillTaskGroupBetweenRunTaskParts)
   // later, till reaching the critical moment when to kill the task
   // in the future.
   Future<Nothing> _run;
-  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _, _))
+  EXPECT_CALL(*slave.get()->mock(), _run(_, _, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&_run),
-                    SaveArg<0>(&unschedules),
-                    SaveArg<1>(&frameworkInfo),
-                    SaveArg<2>(&executorInfo_),
-                    SaveArg<3>(&task_),
-                    SaveArg<4>(&taskGroup_),
-                    SaveArg<5>(&resourceVersionUuids),
-                    SaveArg<6>(&launchExecutor)));
+                    SaveArg<0>(&frameworkInfo),
+                    SaveArg<1>(&executorInfo_),
+                    SaveArg<2>(&task_),
+                    SaveArg<3>(&taskGroup_),
+                    SaveArg<4>(&resourceVersionUuids),
+                    SaveArg<5>(&launchExecutor),
+                    Return(Nothing())));
 
   const v1::Offer& offer = offers->offers(0);
   const SlaveID slaveId = devolve(offer.agent_id());
@@ -8252,7 +8249,6 @@ TEST_F(SlaveTest, KillTaskGroupBetweenRunTaskParts)
 
   Future<Nothing> unmocked__run = process::dispatch(slave.get()->pid, [=] {
     slave.get()->mock()->unmocked__run(
-        unschedules,
         frameworkInfo,
         executorInfo_,
         task_,