You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2016/10/06 02:01:01 UTC

[4/4] mesos git commit: Updated the nested container launch to correctly determine the user.

Updated the nested container launch to correctly determine the user.

This locates the executor that corresponds to the parent container,
and uses the user that was used to launch the executor.

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


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

Branch: refs/heads/master
Commit: 558613cc72248b633bb5e26ef93708abca8ccbf0
Parents: 797d722
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Oct 5 18:40:13 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Wed Oct 5 18:50:56 2016 -0700

----------------------------------------------------------------------
 src/slave/http.cpp      |  33 +++++++++++-
 src/tests/api_tests.cpp | 125 +++++++++++++++++++++++++++----------------
 2 files changed, 112 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/558613cc/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index bdafe1d..79061c3 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -1930,6 +1930,37 @@ Future<Response> Slave::Http::launchNestedContainer(
         " but 'launch_nested_container.container_id.parent.parent' is set");
   }
 
+  // Locate the executor (for now we just loop since we don't
+  // index based on container id and this likely won't have a
+  // significant performance impact due to the low number of
+  // executors per-agent).
+  Executor* executor = nullptr;
+  foreachvalue (Framework* framework, slave->frameworks) {
+    foreachvalue (Executor* executor_, framework->executors) {
+      if (executor_->containerId == containerId.parent()) {
+        executor = executor_;
+        break;
+      }
+    }
+  }
+
+  // Return a "Bad Request" here rather than "Not Found" since
+  // the executor needs to set parent to its container id.
+  if (executor == nullptr) {
+    return BadRequest("Unable to locate executor for parent container"
+                      " " + stringify(containerId.parent()));
+  }
+
+  // By default, we use the executor's user.
+  // The command user overrides it if specified.
+  Option<string> user = executor->user;
+
+#ifndef __WINDOWS__
+  if (call.launch_nested_container().command().has_user()) {
+    user = call.launch_nested_container().command().user();
+  }
+#endif
+
   // TODO(gilbert): The sandbox directory and user are incorrect,
   // Please update it.
   Future<bool> launched = slave->containerizer->launch(
@@ -1938,7 +1969,7 @@ Future<Response> Slave::Http::launchNestedContainer(
       call.launch_nested_container().has_container()
         ? call.launch_nested_container().container()
         : Option<ContainerInfo>::none(),
-      None(),
+      user,
       slave->info.id());
 
   // TODO(bmahler): The containerizers currently require that

http://git-wip-us.apache.org/repos/asf/mesos/blob/558613cc/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 7b0ad3c..24fefbd 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -3330,31 +3330,54 @@ TEST_P(AgentAPITest, NestedContainerLaunchFalse)
 
   Clock::pause();
 
-  StandaloneMasterDetector detector;
-  MockContainerizer mockContainerizer;
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
 
-  EXPECT_CALL(mockContainerizer, recover(_))
-    .WillOnce(Return(Future<Nothing>(Nothing())));
+  Owned<MasterDetector> detector = master.get()->createDetector();
 
-  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
+  TestContainerizer containerizer(&exec);
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(&detector, &mockContainerizer);
+    StartSlave(detector.get(), &containerizer);
 
   ASSERT_SOME(slave);
 
-  // Wait for the agent to finish recovery.
-  AWAIT_READY(__recover);
-  Clock::settle();
+  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(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 0.1, 32, "*"))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  Future<Nothing> executorRegistered;
+  EXPECT_CALL(exec, registered(_, _, _, _))
+    .WillOnce(FutureSatisfy(&executorRegistered));
+
+  EXPECT_CALL(exec, launchTask(_, _))
+    .Times(1);
+
+  driver.start();
+
+  AWAIT_READY(executorRegistered);
+
+  Future<hashset<ContainerID>> containerIds = containerizer.containers();
+  AWAIT_READY(containerIds);
+  EXPECT_EQ(1u, containerIds->size());
 
   // Try to launch an "unsupported" container.
   v1::ContainerID containerId;
   containerId.set_value(UUID::random().toString());
-  containerId.mutable_parent()->set_value(UUID::random().toString());
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
 
   {
     // Return false here to indicate "unsupported".
-    EXPECT_CALL(mockContainerizer, launch(_, _, _, _, _))
+    EXPECT_CALL(containerizer, launch(_, _, _, _, _))
       .WillOnce(Return(Future<bool>(false)));
 
     v1::agent::Call call;
@@ -3373,10 +3396,11 @@ TEST_P(AgentAPITest, NestedContainerLaunchFalse)
     AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
   }
 
-  // The destructor of `cluster::Slave` will try to clean up any
-  // remaining containers by inspecting the result of `containers()`.
-  EXPECT_CALL(mockContainerizer, containers())
-    .WillRepeatedly(Return(hashset<ContainerID>()));
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
+  driver.stop();
+  driver.join();
 }
 
 
@@ -3429,32 +3453,52 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
 
   Clock::pause();
 
-  StandaloneMasterDetector detector;
-  MockContainerizer mockContainerizer;
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
 
-  EXPECT_CALL(mockContainerizer, recover(_))
-    .WillOnce(Return(Future<Nothing>(Nothing())));
+  Owned<MasterDetector> detector = master.get()->createDetector();
 
-  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
+  TestContainerizer containerizer(&exec);
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(&detector, &mockContainerizer);
+    StartSlave(detector.get(), &containerizer);
 
   ASSERT_SOME(slave);
 
-  // Wait for the agent to finish recovery.
-  AWAIT_READY(__recover);
-  Clock::settle();
+  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(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 0.1, 32, "*"))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  Future<Nothing> executorRegistered;
+  EXPECT_CALL(exec, registered(_, _, _, _))
+    .WillOnce(FutureSatisfy(&executorRegistered));
+
+  EXPECT_CALL(exec, launchTask(_, _))
+    .Times(1);
+
+  driver.start();
+
+  AWAIT_READY(executorRegistered);
+
+  Future<hashset<ContainerID>> containerIds = containerizer.containers();
+  AWAIT_READY(containerIds);
+  EXPECT_EQ(1u, containerIds->size());
 
   // Launch a nested container and wait for it to finish.
   v1::ContainerID containerId;
   containerId.set_value(UUID::random().toString());
-  containerId.mutable_parent()->set_value(UUID::random().toString());
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
 
   {
-    EXPECT_CALL(mockContainerizer, launch(_, _, _, _, _))
-      .WillOnce(Return(Future<bool>(true)));
-
     v1::agent::Call call;
     call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
 
@@ -3471,13 +3515,9 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
     AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
   }
 
-  Promise<Option<ContainerTermination>> waitPromise;
   Future<v1::agent::Response> wait;
 
   {
-    EXPECT_CALL(mockContainerizer, wait(_))
-      .WillOnce(Return(waitPromise.future()));
-
     v1::agent::Call call;
     call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
 
@@ -3493,9 +3533,6 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
 
   // Now kill the nested container.
   {
-    EXPECT_CALL(mockContainerizer, destroy(_))
-      .WillOnce(Return(Future<bool>(true)));
-
     v1::agent::Call call;
     call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
 
@@ -3512,19 +3549,17 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
     AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
   }
 
-  ContainerTermination termination;
-  termination.set_status(1);
-
-  waitPromise.set(Option<ContainerTermination>::some(termination));
-
   AWAIT_READY(wait);
   ASSERT_EQ(v1::agent::Response::WAIT_NESTED_CONTAINER, wait->type());
-  EXPECT_EQ(1, wait->wait_nested_container().exit_status());
 
-  // The destructor of `cluster::Slave` will try to clean up any
-  // remaining containers by inspecting the result of `containers()`.
-  EXPECT_CALL(mockContainerizer, containers())
-    .WillRepeatedly(Return(hashset<ContainerID>()));
+  // The test containerizer sets exit status to 0 when destroyed.
+  EXPECT_EQ(0, wait->wait_nested_container().exit_status());
+
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
+  driver.stop();
+  driver.join();
 }
 
 } // namespace tests {