You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2017/03/09 22:28:16 UTC

[1/3] mesos git commit: Fixed command task with container image 'root' user issue.

Repository: mesos
Updated Branches:
  refs/heads/1.2.x f575c586b -> 9f035e1d1


Fixed command task with container image 'root' user issue.

This issue is command task with container image provided specific.
We used to set user as 'root' explicitly for command task with
container image. However, this would break operators who set 'user'
for FrameworkInfo/CommandInfo to any user other than 'root' because
the task cannot access all other contents owned by 'root', e.g.,
persistent volumes, stdout/stderr or any other directories/files
written by modules.

Instead of relying on each isolator/module to explicitly chown,
Mesos should set user to 'root' right before launching the command
executor, because the root privilege is only necessary for 'chroot'
in command executor launch, which should not impact on other
components.

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


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

Branch: refs/heads/1.2.x
Commit: 0f3a68d10f21cb2d16b2bd51214dc86c393f7d1b
Parents: 9c0bbf1
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Mar 9 12:42:44 2017 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 9 13:50:22 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp |  9 +++++++++
 src/slave/slave.cpp                             | 10 +---------
 2 files changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3a68d1/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index a23a6fa..7676a4d 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1449,6 +1449,15 @@ Future<bool> MesosContainerizerProcess::_launch(
     launchInfo.set_user(container->config.user());
   }
 
+  // TODO(gilbert): Remove this once we no longer support command
+  // task in favor of default executor.
+  if (container->config.has_task_info() &&
+      container->config.has_rootfs()) {
+    // We need to set the executor user as root as it needs to
+    // perform chroot (even when switch_user is set to false).
+    launchInfo.set_user("root");
+  }
+
   // Use a pipe to block the child until it's been isolated.
   // The `pipes` array is captured later in a lambda.
   Try<std::array<int_fd, 2>> pipes_ = os::pipe();

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3a68d1/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 7564e8d..db48726 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -4462,10 +4462,6 @@ ExecutorInfo Slave::getExecutorInfo(
     // task. For this reason, we need to strip the image in
     // `executor.container.mesos`.
     container->mutable_mesos()->clear_image();
-
-    // We need to set the executor user as root as it needs to
-    // perform chroot (even when switch_user is set to false).
-    executor.mutable_command()->set_user("root");
   }
 
   // Prepare an executor name which includes information on the
@@ -4544,11 +4540,7 @@ ExecutorInfo Slave::getExecutorInfo(
         gracePeriod.ns());
   }
 
-  // We skip setting the user for the command executor that has
-  // a rootfs image since we need root permissions to chroot.
-  // We assume command executor will change to the correct user
-  // later on.
-  if (!hasRootfs && task.command().has_user()) {
+  if (task.command().has_user()) {
     executor.mutable_command()->set_user(task.command().user());
   }
 


[3/3] mesos git commit: Added unit test for verifying user in command task with image specified.

Posted by ji...@apache.org.
Added unit test for verifying user in command task with image specified.

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


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

Branch: refs/heads/1.2.x
Commit: 9f035e1d1371a513c224735b4f91f84aa6d3e9bf
Parents: 0f3a68d
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Mar 9 12:42:48 2017 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 9 13:52:11 2017 -0800

----------------------------------------------------------------------
 .../containerizer/provisioner_docker_tests.cpp  | 84 ++++++++++++++++++++
 1 file changed, 84 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9f035e1d/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index ce57c06..82fcfca 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -865,6 +865,90 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_ImageDigest)
   driver.join();
 }
 
+
+// This test verifies that if a container image is specified, the
+// command runs as the specified user 'nobody' and the sandbox of
+// the command task is writtable by the specified user. It also
+// verifies that stdout/stderr are owned by the specified user.
+TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_CommandTaskUser)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "docker/runtime,filesystem/linux";
+  flags.image_providers = "docker";
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_EQ(1u, offers->size());
+
+  const Offer& offer = offers.get()[0];
+
+  Result<uid_t> uid = os::getuid("nobody");
+  ASSERT_SOME(uid);
+
+  CommandInfo command;
+  command.set_user("nobody");
+  command.set_value(strings::format(
+      "#!/bin/sh\n"
+      "touch $MESOS_SANDBOX/file\n"
+      "FILE_UID=`stat -c %%u $MESOS_SANDBOX/file`\n"
+      "test $FILE_UID = %d\n"
+      "STDOUT_UID=`stat -c %%u $MESOS_SANDBOX/stdout`\n"
+      "test $STDOUT_UID = %d\n"
+      "STDERR_UID=`stat -c %%u $MESOS_SANDBOX/stderr`\n"
+      "test $STDERR_UID = %d\n",
+      uid.get(), uid.get(), uid.get()).get());
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      Resources::parse("cpus:1;mem:128").get(),
+      command);
+
+  Image image;
+  image.set_type(Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  ContainerInfo* container = task.mutable_container();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFinished;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFinished));
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY_FOR(statusRunning, Seconds(60));
+  EXPECT_EQ(task.task_id(), statusRunning->task_id());
+  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
+
+  AWAIT_READY(statusFinished);
+  EXPECT_EQ(task.task_id(), statusFinished->task_id());
+  EXPECT_EQ(TASK_FINISHED, statusFinished->state());
+
+  driver.stop();
+  driver.join();
+}
 #endif
 
 } // namespace tests {


[2/3] mesos git commit: Removed redundant 'root' user set in containerizer::launch().

Posted by ji...@apache.org.
Removed redundant 'root' user set in containerizer::launch().

The 'user' in launch command is ignored, so it is not necessary to
explicitly set 'root' user to 'CommandInfo' in the case of command
task.

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


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

Branch: refs/heads/1.2.x
Commit: 9c0bbf1ad6bf649323877776a25e09fde7ec04f7
Parents: f575c58
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Mar 9 12:42:40 2017 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 9 13:50:22 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9c0bbf1a/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index b001d02..a23a6fa 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1011,13 +1011,6 @@ Future<bool> MesosContainerizerProcess::launch(
     if (taskInfo->has_container()) {
       ContainerInfo* containerInfo = containerConfig.mutable_container_info();
       containerInfo->CopyFrom(taskInfo->container());
-
-      if (taskInfo->container().mesos().has_image()) {
-        // For command tasks, we need to set the command executor user
-        // as root as it needs to perform chroot (even when
-        // switch_user is set to false).
-        containerConfig.mutable_command_info()->set_user("root");
-      }
     }
   } else {
     // Other cases.