You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2016/07/01 18:42:34 UTC

[1/2] mesos git commit: Updated operator APIs proto files for `GET_EXECUTORS` call.

Repository: mesos
Updated Branches:
  refs/heads/master 926e8d25c -> 53de5578c


Updated operator APIs proto files for `GET_EXECUTORS` call.

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


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

Branch: refs/heads/master
Commit: 42d0ebbabe9f50ef607f8a7838a8c4da1940c35d
Parents: 926e8d2
Author: haosdent huang <ha...@gmail.com>
Authored: Fri Jul 1 11:42:13 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 1 11:42:13 2016 -0700

----------------------------------------------------------------------
 include/mesos/master/master.proto    | 80 ++++++++++++++++++-------------
 include/mesos/v1/master/master.proto | 80 ++++++++++++++++++-------------
 2 files changed, 96 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/42d0ebba/include/mesos/master/master.proto
----------------------------------------------------------------------
diff --git a/include/mesos/master/master.proto b/include/mesos/master/master.proto
index e34414b..d06258e 100644
--- a/include/mesos/master/master.proto
+++ b/include/mesos/master/master.proto
@@ -58,33 +58,34 @@ message Call {
 
     GET_AGENTS = 11;
     GET_FRAMEWORKS = 12;
-    GET_TASKS = 13;         // Retrieves the information about all known tasks.
-    GET_ROLES = 14;         // Retrieves the information about roles.
+    GET_EXECUTORS = 13;     // Retrieves the information about all executors.
+    GET_TASKS = 14;         // Retrieves the information about all known tasks.
+    GET_ROLES = 15;         // Retrieves the information about roles.
 
-    GET_WEIGHTS = 15;       // Retrieves the information about role weights.
-    UPDATE_WEIGHTS = 16;
+    GET_WEIGHTS = 16;       // Retrieves the information about role weights.
+    UPDATE_WEIGHTS = 17;
 
-    GET_LEADING_MASTER = 17; // Retrieves the leading master information.
+    GET_LEADING_MASTER = 18; // Retrieves the leading master information.
 
-    SUBSCRIBE = 18;          // Subscribes the master to receive events.
+    SUBSCRIBE = 19;          // Subscribes the master to receive events.
 
-    RESERVE_RESOURCES = 19;
-    UNRESERVE_RESOURCES = 20;
+    RESERVE_RESOURCES = 20;
+    UNRESERVE_RESOURCES = 21;
 
-    CREATE_VOLUMES = 21;     // See 'CreateVolumes' below.
-    DESTROY_VOLUMES = 22;    // See 'DestroyVolumes' below.
+    CREATE_VOLUMES = 22;     // See 'CreateVolumes' below.
+    DESTROY_VOLUMES = 23;    // See 'DestroyVolumes' below.
 
     // Retrieves the cluster's maintenance status.
-    GET_MAINTENANCE_STATUS = 23;
+    GET_MAINTENANCE_STATUS = 24;
     // Retrieves the cluster's maintenance schedule.
-    GET_MAINTENANCE_SCHEDULE = 24;
-    UPDATE_MAINTENANCE_SCHEDULE = 25; // See 'UpdateMaintenanceSchedule' below.
-    START_MAINTENANCE = 26;           // See 'StartMaintenance' below.
-    STOP_MAINTENANCE = 27;            // See 'StopMaintenance' below.
+    GET_MAINTENANCE_SCHEDULE = 25;
+    UPDATE_MAINTENANCE_SCHEDULE = 26; // See 'UpdateMaintenanceSchedule' below.
+    START_MAINTENANCE = 27;           // See 'StartMaintenance' below.
+    STOP_MAINTENANCE = 28;            // See 'StopMaintenance' below.
 
-    GET_QUOTA = 28;
-    SET_QUOTA = 29;          // See 'SetQuota' below.
-    REMOVE_QUOTA = 30;
+    GET_QUOTA = 29;
+    SET_QUOTA = 30;          // See 'SetQuota' below.
+    REMOVE_QUOTA = 31;
   }
 
   // Provides a snapshot of the current metrics tracked by the master.
@@ -220,17 +221,18 @@ message Response {
 
     GET_AGENTS = 10;
     GET_FRAMEWORKS = 11;
-    GET_TASKS = 12;                // See 'GetTasks' below.
-    GET_ROLES = 13;                // See 'GetRoles' below.
+    GET_EXECUTORS = 12;            // See 'GetExecutors' below.
+    GET_TASKS = 13;                // See 'GetTasks' below.
+    GET_ROLES = 14;                // See 'GetRoles' below.
 
-    GET_WEIGHTS = 14;              // See 'GetWeights' below.
+    GET_WEIGHTS = 15;              // See 'GetWeights' below.
 
-    GET_LEADING_MASTER = 15;       // See 'GetLeadingMaster' below.
+    GET_LEADING_MASTER = 16;       // See 'GetLeadingMaster' below.
 
-    GET_MAINTENANCE_STATUS = 16;   // See 'GetMaintenanceStatus' below.
-    GET_MAINTENANCE_SCHEDULE = 17; // See 'GetMaintenanceSchedule' below.
+    GET_MAINTENANCE_STATUS = 17;   // See 'GetMaintenanceStatus' below.
+    GET_MAINTENANCE_SCHEDULE = 18; // See 'GetMaintenanceSchedule' below.
 
-    GET_QUOTA = 18;
+    GET_QUOTA = 19;
   }
 
   // `healthy` would be true if the master is healthy. Delayed responses are
@@ -329,6 +331,19 @@ message Response {
     repeated FrameworkID unsubscribed_frameworks = 3;
   }
 
+  // Lists information about all the executors known to the master at the
+  // current time. Note that there might be executors unknown to the master
+  // running on partitioned or unsubscribed agents.
+  message GetExecutors {
+    message Executor {
+      required ExecutorInfo executor_info = 1;
+      required SlaveID slave_id = 2;
+    }
+
+    repeated Executor executors = 1;
+    repeated Executor orphan_executors = 2;
+  }
+
   // Lists information about all the tasks known to the master at the current
   // time. Note that there might be tasks unknown to the master running on
   // partitioned or unsubscribed agents.
@@ -398,13 +413,14 @@ message Response {
   optional GetStateSummary get_state_summary = 10;
   optional GetAgents get_agents = 11;
   optional GetFrameworks get_frameworks = 12;
-  optional GetTasks get_tasks = 13;
-  optional GetRoles get_roles = 14;
-  optional GetWeights get_weights = 15;
-  optional GetLeadingMaster get_leading_master = 16;
-  optional GetMaintenanceStatus get_maintenance_status = 17;
-  optional GetMaintenanceSchedule get_maintenance_schedule = 18;
-  optional GetQuota get_quota = 19;
+  optional GetExecutors get_executors = 13;
+  optional GetTasks get_tasks = 14;
+  optional GetRoles get_roles = 15;
+  optional GetWeights get_weights = 16;
+  optional GetLeadingMaster get_leading_master = 17;
+  optional GetMaintenanceStatus get_maintenance_status = 18;
+  optional GetMaintenanceSchedule get_maintenance_schedule = 19;
+  optional GetQuota get_quota = 20;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/42d0ebba/include/mesos/v1/master/master.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/master/master.proto b/include/mesos/v1/master/master.proto
index f6155df..b7cb6fd 100644
--- a/include/mesos/v1/master/master.proto
+++ b/include/mesos/v1/master/master.proto
@@ -58,33 +58,34 @@ message Call {
 
     GET_AGENTS = 11;
     GET_FRAMEWORKS = 12;
-    GET_TASKS = 13;         // Retrieves the information about all known tasks.
-    GET_ROLES = 14;         // Retrieves the information about roles.
+    GET_EXECUTORS = 13;     // Retrieves the information about all executors.
+    GET_TASKS = 14;         // Retrieves the information about all known tasks.
+    GET_ROLES = 15;         // Retrieves the information about roles.
 
-    GET_WEIGHTS = 15;       // Retrieves the information about role weights.
-    UPDATE_WEIGHTS = 16;
+    GET_WEIGHTS = 16;       // Retrieves the information about role weights.
+    UPDATE_WEIGHTS = 17;
 
-    GET_LEADING_MASTER = 17; // Retrieves the leading master information.
+    GET_LEADING_MASTER = 18; // Retrieves the leading master information.
 
-    SUBSCRIBE = 18;          // Subscribes the master to receive events.
+    SUBSCRIBE = 19;          // Subscribes the master to receive events.
 
-    RESERVE_RESOURCES = 19;
-    UNRESERVE_RESOURCES = 20;
+    RESERVE_RESOURCES = 20;
+    UNRESERVE_RESOURCES = 21;
 
-    CREATE_VOLUMES = 21;     // See 'CreateVolumes' below.
-    DESTROY_VOLUMES = 22;    // See 'DestroyVolumes' below.
+    CREATE_VOLUMES = 22;     // See 'CreateVolumes' below.
+    DESTROY_VOLUMES = 23;    // See 'DestroyVolumes' below.
 
     // Retrieves the cluster's maintenance status.
-    GET_MAINTENANCE_STATUS = 23;
+    GET_MAINTENANCE_STATUS = 24;
     // Retrieves the cluster's maintenance schedule.
-    GET_MAINTENANCE_SCHEDULE = 24;
-    UPDATE_MAINTENANCE_SCHEDULE = 25; // See 'UpdateMaintenanceSchedule' below.
-    START_MAINTENANCE = 26;           // See 'StartMaintenance' below.
-    STOP_MAINTENANCE = 27;            // See 'StopMaintenance' below.
+    GET_MAINTENANCE_SCHEDULE = 25;
+    UPDATE_MAINTENANCE_SCHEDULE = 26; // See 'UpdateMaintenanceSchedule' below.
+    START_MAINTENANCE = 27;           // See 'StartMaintenance' below.
+    STOP_MAINTENANCE = 28;            // See 'StopMaintenance' below.
 
-    GET_QUOTA = 28;
-    SET_QUOTA = 29;          // See 'SetQuota' below.
-    REMOVE_QUOTA = 30;
+    GET_QUOTA = 29;
+    SET_QUOTA = 30;          // See 'SetQuota' below.
+    REMOVE_QUOTA = 31;
   }
 
   // Provides a snapshot of the current metrics tracked by the master.
@@ -221,17 +222,18 @@ message Response {
 
     GET_AGENTS = 10;
     GET_FRAMEWORKS = 11;
-    GET_TASKS = 12;                // See 'GetTasks' below.
-    GET_ROLES = 13;                // See 'GetRoles' below.
+    GET_EXECUTORS = 12;            // See 'GetExecutors' below.
+    GET_TASKS = 13;                // See 'GetTasks' below.
+    GET_ROLES = 14;                // See 'GetRoles' below.
 
-    GET_WEIGHTS = 14;              // See 'GetWeights' below.
+    GET_WEIGHTS = 15;              // See 'GetWeights' below.
 
-    GET_LEADING_MASTER = 15;       // See 'GetLeadingMaster' below.
+    GET_LEADING_MASTER = 16;       // See 'GetLeadingMaster' below.
 
-    GET_MAINTENANCE_STATUS = 16;   // See 'GetMaintenanceStatus' below.
-    GET_MAINTENANCE_SCHEDULE = 17; // See 'GetMaintenanceSchedule' below.
+    GET_MAINTENANCE_STATUS = 17;   // See 'GetMaintenanceStatus' below.
+    GET_MAINTENANCE_SCHEDULE = 18; // See 'GetMaintenanceSchedule' below.
 
-    GET_QUOTA = 18;
+    GET_QUOTA = 19;
   }
 
   // `healthy` would be true if the master is healthy. Delayed responses are
@@ -330,6 +332,19 @@ message Response {
     repeated FrameworkID unsubscribed_frameworks = 3;
   }
 
+  // Lists information about all the executors known to the master at the
+  // current time. Note that there might be executors unknown to the master
+  // running on partitioned or unsubscribed agents.
+  message GetExecutors {
+    message Executor {
+      required ExecutorInfo executor_info = 1;
+      required AgentID agent_id = 2;
+    }
+
+    repeated Executor executors = 1;
+    repeated Executor orphan_executors = 2;
+  }
+
   // Lists information about all the tasks known to the master at the current
   // time. Note that there might be tasks unknown to the master running on
   // partitioned or unsubscribed agents.
@@ -399,13 +414,14 @@ message Response {
   optional GetStateSummary get_state_summary = 10;
   optional GetAgents get_agents = 11;
   optional GetFrameworks get_frameworks = 12;
-  optional GetTasks get_tasks = 13;
-  optional GetRoles get_roles = 14;
-  optional GetWeights get_weights = 15;
-  optional GetLeadingMaster get_leading_master = 16;
-  optional GetMaintenanceStatus get_maintenance_status = 17;
-  optional GetMaintenanceSchedule get_maintenance_schedule = 18;
-  optional GetQuota get_quota = 19;
+  optional GetExecutors get_executors = 13;
+  optional GetTasks get_tasks = 14;
+  optional GetRoles get_roles = 15;
+  optional GetWeights get_weights = 16;
+  optional GetLeadingMaster get_leading_master = 17;
+  optional GetMaintenanceStatus get_maintenance_status = 18;
+  optional GetMaintenanceSchedule get_maintenance_schedule = 19;
+  optional GetQuota get_quota = 20;
 }
 
 


[2/2] mesos git commit: Implemented GET_EXECUTORS Call in v1 master API.

Posted by vi...@apache.org.
Implemented GET_EXECUTORS Call in v1 master API.

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


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

Branch: refs/heads/master
Commit: 53de5578c6ffc418275ff801838befc7b3900504
Parents: 42d0ebb
Author: haosdent huang <ha...@gmail.com>
Authored: Fri Jul 1 11:42:19 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 1 11:42:19 2016 -0700

----------------------------------------------------------------------
 src/master/http.cpp       | 115 +++++++++++++++++++++++++++++++++++++++++
 src/master/master.hpp     |   5 ++
 src/master/validation.cpp |   3 ++
 src/tests/api_tests.cpp   |  90 ++++++++++++++++++++++++++++++++
 4 files changed, 213 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/53de5578/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 528f01f..7b2f77b 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -526,6 +526,9 @@ Future<Response> Master::Http::api(
     case mesos::master::Call::GET_FRAMEWORKS:
       return getFrameworks(call, principal, acceptType);
 
+    case mesos::master::Call::GET_EXECUTORS:
+      return getExecutors(call, principal, acceptType);
+
     case mesos::master::Call::GET_TASKS:
       return getTasks(call, principal, acceptType);
 
@@ -1390,6 +1393,118 @@ Future<Response> Master::Http::getFrameworks(
 }
 
 
+Future<Response> Master::Http::getExecutors(
+    const mesos::master::Call& call,
+    const Option<string>& principal,
+    ContentType contentType) const
+{
+  CHECK_EQ(mesos::master::Call::GET_EXECUTORS, call.type());
+
+  // Retrieve `ObjectApprover`s for authorizing frameworks and executors.
+  Future<Owned<ObjectApprover>> frameworksApprover;
+  Future<Owned<ObjectApprover>> executorsApprover;
+  if (master->authorizer.isSome()) {
+    authorization::Subject subject;
+    if (principal.isSome()) {
+      subject.set_value(principal.get());
+    }
+
+    frameworksApprover = master->authorizer.get()->getObjectApprover(
+        subject, authorization::VIEW_FRAMEWORK);
+
+    executorsApprover = master->authorizer.get()->getObjectApprover(
+        subject, authorization::VIEW_EXECUTOR);
+  } else {
+    frameworksApprover = Owned<ObjectApprover>(new AcceptingObjectApprover());
+    executorsApprover = Owned<ObjectApprover>(new AcceptingObjectApprover());
+  }
+
+  return collect(frameworksApprover, executorsApprover)
+    .then(defer(master->self(),
+        [=](const tuple<Owned<ObjectApprover>,
+                        Owned<ObjectApprover>>& approvers)
+          -> Response {
+      // Get approver from tuple.
+      Owned<ObjectApprover> frameworksApprover;
+      Owned<ObjectApprover> executorsApprover;
+      tie(frameworksApprover, executorsApprover) = approvers;
+
+      // Construct framework list with both active and completed frameworks.
+      vector<const Framework*> frameworks;
+      foreachvalue (Framework* framework, master->frameworks.registered) {
+        // Skip unauthorized frameworks.
+        if (!approveViewFrameworkInfo(frameworksApprover, framework->info)) {
+          continue;
+        }
+
+        frameworks.push_back(framework);
+      }
+
+      foreach (const std::shared_ptr<Framework>& framework,
+               master->frameworks.completed) {
+        // Skip unauthorized frameworks.
+        if (!approveViewFrameworkInfo(frameworksApprover, framework->info)) {
+          continue;
+        }
+
+        frameworks.push_back(framework.get());
+      }
+
+      mesos::master::Response response;
+      response.set_type(mesos::master::Response::GET_EXECUTORS);
+
+      mesos::master::Response::GetExecutors* getExecutors =
+        response.mutable_get_executors();
+
+      foreach (const Framework* framework, frameworks) {
+        foreachpair (const SlaveID& slaveId,
+                     const auto& executorsMap,
+                     framework->executors) {
+          foreachvalue (const ExecutorInfo& info, executorsMap) {
+            // Skip unauthorized executors.
+            if (!approveViewExecutorInfo(executorsApprover,
+                                         info,
+                                         framework->info)) {
+              continue;
+            }
+
+            mesos::master::Response::GetExecutors::Executor* executor =
+              getExecutors->add_executors();
+
+            executor->mutable_executor_info()->CopyFrom(info);
+            executor->mutable_slave_id()->CopyFrom(slaveId);
+          }
+        }
+      }
+
+      // Orphan executors.
+      // TODO(haosdent): Need to filter these executors based on authorization!
+      // This is currently not possible because we don't have `FrameworkInfo`
+      // for these executors. We need to either store `FrameworkInfo` for orphan
+      // executors or persist FrameworkInfo of all frameworks in the registry.
+      foreachvalue (const Slave* slave, master->slaves.registered) {
+        typedef hashmap<ExecutorID, ExecutorInfo> ExecutorMap;
+        foreachpair (const FrameworkID& frameworkId,
+                     const ExecutorMap& executors,
+                     slave->executors) {
+          foreachvalue (const ExecutorInfo& info, executors) {
+            if (!master->frameworks.registered.contains(frameworkId)) {
+              mesos::master::Response::GetExecutors::Executor* executor =
+                getExecutors->add_orphan_executors();
+
+              executor->mutable_executor_info()->CopyFrom(info);
+              executor->mutable_slave_id()->CopyFrom(slave->id);
+            }
+          }
+        }
+      }
+
+      return OK(serialize(contentType, evolve(response)),
+                stringify(contentType));
+    }));
+}
+
+
 class Master::Http::FlagsError : public Error
 {
 public:

http://git-wip-us.apache.org/repos/asf/mesos/blob/53de5578/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 7388a3e..be7cd23 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1477,6 +1477,11 @@ private:
         const Option<std::string>& principal,
         ContentType contentType) const;
 
+    process::Future<process::http::Response> getExecutors(
+        const mesos::master::Call& call,
+        const Option<std::string>& principal,
+        ContentType contentType) const;
+
     Master* master;
 
     // NOTE: The quota specific pieces of the Operator API are factored

http://git-wip-us.apache.org/repos/asf/mesos/blob/53de5578/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 5026afc..50ba372 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -117,6 +117,9 @@ Option<Error> validate(
     case mesos::master::Call::GET_FRAMEWORKS:
       return None();
 
+    case mesos::master::Call::GET_EXECUTORS:
+      return None();
+
     case mesos::master::Call::GET_TASKS:
       return None();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/53de5578/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index a7f0751..e2d8bf5 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -335,6 +335,96 @@ TEST_P(MasterAPITest, GetMetrics)
 }
 
 
+TEST_P(MasterAPITest, GetExecutors)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  // For capturing the SlaveID so we can use it to verify GET_EXECUTORS API
+  // call.
+  Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+    FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
+  TestContainerizer containerizer(&exec);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), &containerizer);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(slaveRegisteredMessage);
+  SlaveID slaveId = slaveRegisteredMessage.get().slave_id();
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  TaskInfo task;
+  task.set_name("test");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->MergeFrom(offers.get()[0].slave_id());
+  task.mutable_resources()->MergeFrom(offers.get()[0].resources());
+  task.mutable_executor()->MergeFrom(DEFAULT_EXECUTOR_INFO);
+
+  EXPECT_CALL(exec, registered(_, _, _, _))
+    .Times(1);
+
+  EXPECT_CALL(exec, launchTask(_, _))
+    .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
+
+  Future<TaskStatus> status;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&status));
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(status);
+  EXPECT_EQ(TASK_RUNNING, status.get().state());
+  EXPECT_TRUE(status.get().has_executor_id());
+  EXPECT_EQ(exec.id, status.get().executor_id());
+
+  v1::master::Call v1Call;
+  v1Call.set_type(v1::master::Call::GET_EXECUTORS);
+
+  ContentType contentType = GetParam();
+
+  Future<v1::master::Response> v1Response =
+    post(master.get()->pid, v1Call, contentType);
+
+  AWAIT_READY(v1Response);
+  ASSERT_TRUE(v1Response.get().IsInitialized());
+  ASSERT_EQ(v1::master::Response::GET_EXECUTORS, v1Response.get().type());
+  ASSERT_EQ(1, v1Response.get().get_executors().executors_size());
+
+  ASSERT_EQ(evolve<v1::AgentID>(slaveId),
+            v1Response.get().get_executors().executors(0).agent_id());
+
+  v1::ExecutorInfo executorInfo =
+    v1Response.get().get_executors().executors(0).executor_info();
+
+  ASSERT_EQ(evolve<v1::ExecutorID>(exec.id), executorInfo.executor_id());
+
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
+  driver.stop();
+  driver.join();
+}
+
+
 TEST_P(MasterAPITest, GetTasksNoRunningTask)
 {
   Try<Owned<cluster::Master>> master = this->StartMaster();