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 2018/05/24 20:33:24 UTC

[1/2] mesos git commit: Adjusted the tests that use nobody.

Repository: mesos
Updated Branches:
  refs/heads/master 9780b214e -> 32d4305b8


Adjusted the tests that use nobody.

Used `$SUDO_USER` instead because `nobody` sometimes cannot access
direcotries under `$HOME` of the current user running the tests.

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


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

Branch: refs/heads/master
Commit: 32d4305b87e79ed02cc686e0c29b027e31c6b3a4
Parents: 902868f
Author: Jie Yu <yu...@gmail.com>
Authored: Thu May 24 10:05:17 2018 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu May 24 13:33:13 2018 -0700

----------------------------------------------------------------------
 src/tests/container_logger_tests.cpp            |  12 +-
 src/tests/containerizer/capabilities_tests.cpp  |  17 ++-
 .../containerizer/cgroups_isolator_tests.cpp    |  17 ++-
 .../docker_containerizer_tests.cpp              |   8 +-
 .../containerizer/mesos_containerizer_tests.cpp |  10 +-
 .../containerizer/provisioner_docker_tests.cpp  |  12 +-
 .../volume_sandbox_path_isolator_tests.cpp      |  22 +++-
 src/tests/environment.cpp                       |  38 ++++++
 src/tests/fetcher_tests.cpp                     |  10 +-
 src/tests/slave_tests.cpp                       | 127 ++++---------------
 10 files changed, 132 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/container_logger_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/container_logger_tests.cpp b/src/tests/container_logger_tests.cpp
index 7ec6f87..6c6b315 100644
--- a/src/tests/container_logger_tests.cpp
+++ b/src/tests/container_logger_tests.cpp
@@ -599,7 +599,8 @@ INSTANTIATE_TEST_CASE_P(
 //    launch subprocesses with the same user as the executor.
 // 2. When `--switch_user` is false on the agent, the logger module should
 //    inherit the user of the agent.
-TEST_P(UserContainerLoggerTest, ROOT_LOGROTATE_RotateWithSwitchUserTrueOrFalse)
+TEST_P(UserContainerLoggerTest,
+       ROOT_LOGROTATE_UNPRIVILEGED_USER_RotateWithSwitchUserTrueOrFalse)
 {
   // Create a master, agent, and framework.
   Try<Owned<cluster::Master>> master = StartMaster();
@@ -671,8 +672,11 @@ TEST_P(UserContainerLoggerTest, ROOT_LOGROTATE_RotateWithSwitchUserTrueOrFalse)
       "i=0; while [ $i -lt 3072 ]; "
       "do printf '%-1024d\\n' $i; i=$((i+1)); done");
 
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
   // Start the task as a non-root user.
-  task.mutable_command()->set_user("nobody");
+  task.mutable_command()->set_user(user.get());
 
   Future<TaskStatus> statusStarting;
   Future<TaskStatus> statusRunning;
@@ -744,10 +748,10 @@ TEST_P(UserContainerLoggerTest, ROOT_LOGROTATE_RotateWithSwitchUserTrueOrFalse)
   ASSERT_GE(::stat(stdoutPath.c_str(), &stdoutStat), 0);
 
   // Depending on the `--switch_user`, the expected user is either
-  // "nobody" or "root".
+  // "$SUDO_USER" or "root".
   Result<string> stdoutUser = os::user(stdoutStat.st_uid);
   if (GetParam()) {
-    ASSERT_SOME_EQ("nobody", stdoutUser);
+    ASSERT_SOME_EQ(user.get(), stdoutUser);
   } else {
     ASSERT_SOME_EQ("root", stdoutUser);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/containerizer/capabilities_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/capabilities_tests.cpp b/src/tests/containerizer/capabilities_tests.cpp
index 734aa21..8d7f6e9 100644
--- a/src/tests/containerizer/capabilities_tests.cpp
+++ b/src/tests/containerizer/capabilities_tests.cpp
@@ -48,9 +48,6 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-constexpr char CAPS_TEST_UNPRIVILEGED_USER[] = "nobody";
-
-
 class CapabilitiesTest : public ::testing::Test
 {
 public:
@@ -103,7 +100,7 @@ TEST_F(CapabilitiesTest, ROOT_PingWithNoNetRawCaps)
 // be controlled after `setuid` system call. An operation ('ping')
 // that needs `NET_RAW` capability does not succeed if the capability
 // `NET_RAW` is dropped.
-TEST_F(CapabilitiesTest, ROOT_PingWithNoNetRawCapsChangeUser)
+TEST_F(CapabilitiesTest, ROOT_UNPRIVILEGED_USER_PingWithNoNetRawCapsChangeUser)
 {
   Try<Capabilities> manager = Capabilities::create();
   ASSERT_SOME(manager);
@@ -113,9 +110,12 @@ TEST_F(CapabilitiesTest, ROOT_PingWithNoNetRawCapsChangeUser)
 
   capabilities->drop(capabilities::PERMITTED, capabilities::NET_RAW);
 
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
   Try<Subprocess> s = ping(
       capabilities->get(capabilities::PERMITTED),
-      CAPS_TEST_UNPRIVILEGED_USER);
+      user.get());
 
   ASSERT_SOME(s);
 
@@ -138,14 +138,17 @@ TEST_F(CapabilitiesTest, ROOT_PingWithNoNetRawCapsChangeUser)
 // modified to understand capabilities. For such applications, the
 // kernel checks if the process obtained all permitted capabilities
 // that were specified in the file permitted set during 'exec'.
-TEST_F(CapabilitiesTest, ROOT_PingWithJustNetRawSysAdminCap)
+TEST_F(CapabilitiesTest, ROOT_UNPRIVILEGED_USER_PingWithJustNetRawSysAdminCap)
 {
   set<Capability> capabilities = {
     capabilities::NET_RAW,
     capabilities::NET_ADMIN
   };
 
-  Try<Subprocess> s = ping(capabilities, CAPS_TEST_UNPRIVILEGED_USER);
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
+  Try<Subprocess> s = ping(capabilities, user.get());
   ASSERT_SOME(s);
 
   AWAIT_EXPECT_WEXITSTATUS_EQ(0, s->status());

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/containerizer/cgroups_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cgroups_isolator_tests.cpp b/src/tests/containerizer/cgroups_isolator_tests.cpp
index 40c18a1..231e958 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -95,7 +95,8 @@ class CgroupsIsolatorTest
 // task with an unprivileged user. Then verifies that the unprivileged
 // user has write permission under the corresponding cgroups which are
 // prepared for the container to run the task.
-TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_NET_CLS_UserCgroup)
+TEST_F(CgroupsIsolatorTest,
+       ROOT_CGROUPS_PERF_NET_CLS_UNPRIVILEGED_USER_UserCgroup)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -158,13 +159,16 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_NET_CLS_UserCgroup)
   AWAIT_READY(offers);
   ASSERT_FALSE(offers->empty());
 
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
   // Launch a task with the command executor.
   CommandInfo command;
   command.set_shell(false);
   command.set_value("/bin/sleep");
   command.add_arguments("sleep");
   command.add_arguments("120");
-  command.set_user("nobody");
+  command.set_user(user.get());
 
   TaskInfo task = createTask(
       offers.get()[0].slave_id(),
@@ -218,7 +222,8 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_NET_CLS_UserCgroup)
     // Verify that the user cannot manipulate the container's cgroup
     // control files as their owner is root.
     EXPECT_SOME_NE(0, os::system(strings::format(
-        "su - nobody -s /bin/sh -c 'echo $$ > %s'",
+        "su - %s -s /bin/sh -c 'echo $$ > %s'",
+        user.get(),
         path::join(hierarchy.get(), cgroup, "cgroup.procs")).get()));
 
     // Verify that the user can create a cgroup under the container's
@@ -226,13 +231,15 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_NET_CLS_UserCgroup)
     string userCgroup = path::join(cgroup, "user");
 
     EXPECT_SOME_EQ(0, os::system(strings::format(
-        "su - nobody -s /bin/sh -c 'mkdir %s'",
+        "su - %s -s /bin/sh -c 'mkdir %s'",
+        user.get(),
         path::join(hierarchy.get(), userCgroup)).get()));
 
     // Verify that the user can manipulate control files in the
     // created cgroup as it's owned by the user.
     EXPECT_SOME_EQ(0, os::system(strings::format(
-        "su - nobody -s /bin/sh -c 'echo $$ > %s'",
+        "su - %s -s /bin/sh -c 'echo $$ > %s'",
+        user.get(),
         path::join(hierarchy.get(), userCgroup, "cgroup.procs")).get()));
 
     // Clear up the folder.

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/containerizer/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index e37a9c1..f474c71 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -4196,7 +4196,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_CGROUPS_CFS_CgroupsEnableCFS)
 // Run a task as non root while inheriting this ownership from the
 // framework supplied default user. Tests if the sandbox "stdout"
 // is correctly owned and writeable by the tasks user.
-TEST_F(DockerContainerizerTest, ROOT_DOCKER_Non_Root_Sandbox)
+TEST_F(DockerContainerizerTest,
+       ROOT_DOCKER_UNPRIVILEGED_USER_NonRootSandbox)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -4231,9 +4232,12 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Non_Root_Sandbox)
   AWAIT_READY(slaveRegisteredMessage);
   SlaveID slaveId = slaveRegisteredMessage->slave_id();
 
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
   FrameworkInfo framework;
   framework.set_name("default");
-  framework.set_user("nobody");
+  framework.set_user(user.get());
   framework.set_principal(DEFAULT_CREDENTIAL.principal());
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 01f2b38..78dbdea 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -657,7 +657,8 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
 
 // This test verified that the stdout and stderr files in the task's sandbox
 // are owned by the task user.
-TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
+TEST_F(MesosContainerizerExecuteTest,
+       ROOT_UNPRIVILEGED_USER_SandboxFileOwnership)
 {
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
@@ -673,10 +674,11 @@ TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
   ContainerID containerId;
   containerId.set_value(id::UUID::random().toString());
 
-  const string user = "nobody";
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
 
   ExecutorInfo executor = createExecutorInfo("executor", "exit 0");
-  executor.mutable_command()->set_user(user);
+  executor.mutable_command()->set_user(user.get());
 
   Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
@@ -687,7 +689,7 @@ TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
   // Wait for the launch to complete.
   AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
-  Result<uid_t> uid = os::getuid(user);
+  Result<uid_t> uid = os::getuid(user.get());
   ASSERT_SOME(uid);
 
   // Verify that stdout is owned by the task user.

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/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 c664ff8..3cb1a7e 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -954,10 +954,11 @@ TEST_F(ProvisionerDockerTest, ROOT_INTERNET_CURL_ImageDigest)
 
 
 // This test verifies that if a container image is specified, the
-// command runs as the specified user 'nobody' and the sandbox of
+// command runs as the specified user "$SUDO_USER" and the sandbox of
 // the command task is writeable by the specified user. It also
 // verifies that stdout/stderr are owned by the specified user.
-TEST_F(ProvisionerDockerTest, ROOT_INTERNET_CURL_CommandTaskUser)
+TEST_F(ProvisionerDockerTest,
+       ROOT_INTERNET_CURL_UNPRIVILEGED_USER_CommandTaskUser)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -988,11 +989,14 @@ TEST_F(ProvisionerDockerTest, ROOT_INTERNET_CURL_CommandTaskUser)
 
   const Offer& offer = offers.get()[0];
 
-  Result<uid_t> uid = os::getuid("nobody");
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
+  Result<uid_t> uid = os::getuid(user.get());
   ASSERT_SOME(uid);
 
   CommandInfo command;
-  command.set_user("nobody");
+  command.set_user(user.get());
   command.set_value(strings::format(
       "#!/bin/sh\n"
       "touch $MESOS_SANDBOX/file\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
index 0400052..7aa181b 100644
--- a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
@@ -219,7 +219,8 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedParentTypeVolume)
 // simulate the scenario that the framework user is non-root while
 // the agent process is root, to make sure that non-root user can
 // still have the permission to write to the volume as expected.
-TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfTypeOwnership)
+TEST_F(VolumeSandboxPathIsolatorTest,
+       ROOT_UNPRIVILEGED_USER_SelfTypeOwnership)
 {
   string registry = path::join(sandbox.get(), "registry");
   AWAIT_READY(DockerArchive::create(registry, "test_image"));
@@ -255,11 +256,14 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfTypeOwnership)
 
   // Simulate the executor sandbox ownership as the user
   // from FrameworkInfo.
-  ASSERT_SOME(os::chown("nobody", directory));
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
+  ASSERT_SOME(os::chown(user.get(), directory));
 
   Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
-      createContainerConfig(None(), executor, directory, "nobody"),
+      createContainerConfig(None(), executor, directory, user.get()),
       map<string, string>(),
       None());
 
@@ -281,7 +285,8 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfTypeOwnership)
 // simulate the scenario that the framework user is non-root while
 // the agent process is root, to make sure that non-root user can
 // still have the permission to write to the volume as expected.
-TEST_F(VolumeSandboxPathIsolatorTest, ROOT_ParentTypeOwnership)
+TEST_F(VolumeSandboxPathIsolatorTest,
+       ROOT_UNPRIVILEGED_USER_ParentTypeOwnership)
 {
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "volume/sandbox_path";
@@ -312,11 +317,14 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_ParentTypeOwnership)
 
   // Simulate the executor sandbox ownership as the user
   // from FrameworkInfo.
-  ASSERT_SOME(os::chown("nobody", directory.get()));
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
+  ASSERT_SOME(os::chown(user.get(), directory.get()));
 
   Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
-      createContainerConfig(None(), executor, directory.get(), "nobody"),
+      createContainerConfig(None(), executor, directory.get(), user.get()),
       map<string, string>(),
       None());
 
@@ -346,7 +354,7 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_ParentTypeOwnership)
           createCommandInfo("echo 'hello' > parent/file"),
           containerInfo,
           None(),
-          "nobody"),
+          user.get()),
       map<string, string>(),
       None());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/environment.cpp
----------------------------------------------------------------------
diff --git a/src/tests/environment.cpp b/src/tests/environment.cpp
index a5a255f..3b84c0a 100644
--- a/src/tests/environment.cpp
+++ b/src/tests/environment.cpp
@@ -860,6 +860,43 @@ public:
 };
 
 
+class UnprivilegedUserFilter : public TestFilter
+{
+public:
+  UnprivilegedUserFilter()
+  {
+#ifdef __WINDOWS__
+    unprivilegedUserFound = false;
+#else
+    Option<string> user = os::getenv("SUDO_USER");
+    if (user.isNone() || user.get() == "root") {
+      unprivilegedUserFound = false;
+    } else {
+      unprivilegedUserFound = true;
+    }
+
+    if (!unprivilegedUserFound) {
+      std::cerr
+        << "-------------------------------------------------------------\n"
+        << "No usable unprivileged user found from the 'SUDO_USER'\n"
+        << "environment variable. So tests that rely on an unprivileged\n"
+        << "user will not run\n"
+        << "-------------------------------------------------------------"
+        << std::endl;
+    }
+#endif
+  }
+
+  bool disable(const ::testing::TestInfo* test) const
+  {
+    return matches(test, "UNPRIVILEGED_USER_") && !unprivilegedUserFound;
+  }
+
+private:
+  bool unprivilegedUserFound;
+};
+
+
 class UnzipFilter : public TestFilter
 {
 public:
@@ -905,6 +942,7 @@ Environment::Environment(const Flags& _flags)
             std::make_shared<PerfCPUCyclesFilter>(),
             std::make_shared<PerfFilter>(),
             std::make_shared<RootFilter>(),
+            std::make_shared<UnprivilegedUserFilter>(),
             std::make_shared<UnzipFilter>(),
             std::make_shared<XfsFilter>()}),
     flags(_flags)

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/fetcher_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_tests.cpp b/src/tests/fetcher_tests.cpp
index 95359ec..8c353f2 100644
--- a/src/tests/fetcher_tests.cpp
+++ b/src/tests/fetcher_tests.cpp
@@ -222,10 +222,12 @@ TEST_F(FetcherTest, LogFailureToStderr)
 #ifndef __WINDOWS__
 // Tests that non-root users are unable to fetch root-protected files on the
 // local filesystem.
-TEST_F(FetcherTest, ROOT_RootProtectedFileURI)
+TEST_F(FetcherTest, ROOT_UNPRIVILEGED_USER_RootProtectedFileURI)
 {
-  const string user = "nobody";
-  ASSERT_SOME(os::getuid(user));
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
+
+  ASSERT_SOME(os::getuid(user.get()));
 
   string fromDir = path::join(os::getcwd(), "from");
   ASSERT_SOME(os::mkdir(fromDir));
@@ -240,7 +242,7 @@ TEST_F(FetcherTest, ROOT_RootProtectedFileURI)
   containerId.set_value(id::UUID::random().toString());
 
   CommandInfo commandInfo;
-  commandInfo.set_user(user);
+  commandInfo.set_user(user.get());
 
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(uri::from_path(testFile));

http://git-wip-us.apache.org/repos/asf/mesos/blob/32d4305b/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 5d3abd2..068eb5e 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -1031,53 +1031,26 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 #ifndef __WINDOWS__
 // This test runs a command _with_ the command user field set. The
 // command will verify the assumption that the command is run as the
-// specified user. We use (and assume the presence) of the
-// unprivileged 'nobody' user which should be available on both Linux
-// and Mac OS X.
-//
-// TODO(alexr): Enable after MESOS-2199 is resolved.
-TEST_F(SlaveTest, DISABLED_ROOT_RunTaskWithCommandInfoWithUser)
+// specified user.
+TEST_F(SlaveTest, ROOT_UNPRIVILEGED_USER_RunTaskWithCommandInfoWithUser)
 {
-  // TODO(nnielsen): Introduce STOUT abstraction for user verification
-  // instead of flat getpwnam call.
-  const string testUser = "nobody";
-  if (::getpwnam(testUser.c_str()) == nullptr) {
-    LOG(WARNING) << "Cannot run ROOT_RunTaskWithCommandInfoWithUser test:"
-                 << " user '" << testUser << "' is not present";
-    return;
-  }
-
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  // Need flags for 'executor_registration_timeout'.
-  slave::Flags flags = CreateSlaveFlags();
-  flags.isolation = "posix/cpu,posix/mem";
-
-  Fetcher fetcher(flags);
-
-  Try<MesosContainerizer*> _containerizer =
-    MesosContainerizer::create(flags, false, &fetcher);
-
-  ASSERT_SOME(_containerizer);
-  Owned<MesosContainerizer> containerizer(_containerizer.get());
-
   Owned<MasterDetector> detector = master.get()->createDetector();
 
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), containerizer.get());
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
   ASSERT_SOME(slave);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      DEFAULT_CREDENTIAL);
 
   EXPECT_CALL(sched, registered(&driver, _, _));
 
-  Future<TaskStatus> statusRunning;
-  Future<TaskStatus> statusFinished;
-  const string helper = getTestHelperPath("test-helper");
-
   Future<vector<Offer>> offers;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers))
@@ -1088,90 +1061,36 @@ TEST_F(SlaveTest, DISABLED_ROOT_RunTaskWithCommandInfoWithUser)
   AWAIT_READY(offers);
   ASSERT_FALSE(offers->empty());
 
-  // HACK: Launch a prepare task as root to prepare the binaries.
-  // This task creates the lt-mesos-executor binary in the build dir.
-  // Because the real task is run as a test user (nobody), it does not
-  // have permission to create files in the build directory.
-  TaskInfo prepareTask;
-  prepareTask.set_name("prepare task");
-  prepareTask.mutable_task_id()->set_value("1");
-  prepareTask.mutable_slave_id()->CopyFrom(offers.get()[0].slave_id());
-  prepareTask.mutable_resources()->CopyFrom(
-      offers.get()[0].resources());
-
-  Result<string> user = os::user();
-  ASSERT_SOME(user) << "Failed to get current user name"
-                    << (user.isError() ? ": " + user.error() : "");
-  // Current user should be root.
-  EXPECT_EQ("root", user.get());
-
-  // This prepare command executor will run as the current user
-  // running the tests (root). After this command executor finishes,
-  // we know that the lt-mesos-executor binary file exists.
-  CommandInfo prepareCommand;
-  prepareCommand.set_shell(false);
-  prepareCommand.set_value(helper);
-  prepareCommand.add_arguments(helper);
-  prepareCommand.add_arguments(ActiveUserTestHelper::NAME);
-  prepareCommand.add_arguments("--user=" + user.get());
-  prepareTask.mutable_command()->CopyFrom(prepareCommand);
-
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&statusRunning))
-    .WillOnce(FutureArg<1>(&statusFinished));
-
-  driver.launchTasks(offers.get()[0].id(), {prepareTask});
+  Option<string> user = os::getenv("SUDO_USER");
+  ASSERT_SOME(user);
 
-  // Scheduler should first receive TASK_RUNNING followed by the
-  // TASK_FINISHED from the executor.
-  AWAIT_READY(statusRunning);
-  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
-  EXPECT_EQ(TaskStatus::SOURCE_EXECUTOR, statusRunning->source());
-
-  AWAIT_READY(statusFinished);
-  EXPECT_EQ(TASK_FINISHED, statusFinished->state());
-  EXPECT_EQ(TaskStatus::SOURCE_EXECUTOR, statusFinished->source());
-
-  // Start to launch a task with different user.
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
-
-  AWAIT_READY(offers);
-  ASSERT_FALSE(offers->empty());
-
-  // Launch a task with the command executor.
-  TaskInfo task;
-  task.set_name("");
-  task.mutable_task_id()->set_value("2");
-  task.mutable_slave_id()->CopyFrom(offers.get()[0].slave_id());
-  task.mutable_resources()->CopyFrom(offers.get()[0].resources());
+  Result<uid_t> uid = os::getuid(user.get());
+  ASSERT_SOME(uid);
 
-  CommandInfo command;
-  command.set_user(testUser);
-  command.set_shell(false);
-  command.set_value(helper);
-  command.add_arguments(helper);
-  command.add_arguments(ActiveUserTestHelper::NAME);
-  command.add_arguments("--user=" + testUser);
+  TaskInfo task = createTask(
+      offers->at(0),
+      "test `id -u` == " + stringify(uid.get()));
 
-  task.mutable_command()->CopyFrom(command);
+  task.mutable_command()->set_user(user.get());
 
+  Future<TaskStatus> statusStarting;
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFinished;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusStarting))
     .WillOnce(FutureArg<1>(&statusRunning))
     .WillOnce(FutureArg<1>(&statusFinished));
 
-  driver.launchTasks(offers.get()[0].id(), {task});
+  driver.launchTasks(offers->at(0).id(), {task});
+
+  AWAIT_READY(statusStarting);
+  EXPECT_EQ(TASK_STARTING, statusStarting->state());
 
-  // Scheduler should first receive TASK_RUNNING followed by the
-  // TASK_FINISHED from the executor.
   AWAIT_READY(statusRunning);
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
-  EXPECT_EQ(TaskStatus::SOURCE_EXECUTOR, statusRunning->source());
 
   AWAIT_READY(statusFinished);
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
-  EXPECT_EQ(TaskStatus::SOURCE_EXECUTOR, statusFinished->source());
 
   driver.stop();
   driver.join();


[2/2] mesos git commit: Removed an unneeded test.

Posted by ji...@apache.org.
Removed an unneeded test.

This test is covered by many other tests, and is testing for a
deprecated feature. Thus remove this test.

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


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

Branch: refs/heads/master
Commit: 902868fac0785b160c6cf0f13e655451bc0aa7ce
Parents: 9780b21
Author: Jie Yu <yu...@gmail.com>
Authored: Thu May 24 10:03:52 2018 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu May 24 13:33:13 2018 -0700

----------------------------------------------------------------------
 src/tests/slave_tests.cpp | 86 ------------------------------------------
 1 file changed, 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/902868fa/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 65d8605..5d3abd2 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -931,92 +931,6 @@ TEST_F(SlaveTest, GetExecutorInfoForTaskWithContainer)
 }
 
 
-// This tests ensures that MesosContainerizer will launch a command
-// executor even if it contains a ContainerInfo in the TaskInfo.
-// Prior to 0.26.0, this was only used to launch Docker containers, so
-// MesosContainerizer would fail the launch.
-//
-// TODO(jieyu): Move this test to the mesos containerizer tests.
-TEST_F(SlaveTest, ROOT_LaunchTaskInfoWithContainerInfo)
-{
-  Try<Owned<cluster::Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  // Need flags for 'executor_registration_timeout'.
-  slave::Flags flags = CreateSlaveFlags();
-  flags.isolation = defaultIsolators;
-
-  Fetcher fetcher(flags);
-
-  Try<MesosContainerizer*> _containerizer =
-    MesosContainerizer::create(flags, false, &fetcher);
-
-  ASSERT_SOME(_containerizer);
-  Owned<MesosContainerizer> containerizer(_containerizer.get());
-
-  StandaloneMasterDetector detector;
-
-  Try<Owned<cluster::Slave>> slave = StartSlave(
-      &detector,
-      containerizer.get(),
-      flags,
-      true);
-
-  ASSERT_SOME(slave);
-  ASSERT_NE(nullptr, slave.get()->mock());
-
-  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.mutable_id()->set_value(
-      "20141010-221431-251662764-60288-12345-0000");
-
-  Resources taskResources = Resources::parse("cpus:0.1;mem:32").get();
-  taskResources.allocate(frameworkInfo.roles(0));
-
-  // Launch a task with the command executor and ContainerInfo with
-  // NetworkInfo.
-  TaskInfo task;
-  task.set_name("task");
-  task.mutable_task_id()->set_value("1");
-  task.mutable_slave_id()->set_value(
-      "20141010-221431-251662764-60288-12345-0001");
-  task.mutable_resources()->MergeFrom(taskResources);
-  task.mutable_command()->MergeFrom(echoAuthorCommand());
-
-  ContainerID containerId;
-  containerId.set_value(id::UUID::random().toString());
-
-  ContainerInfo* container = task.mutable_container();
-  container->set_type(ContainerInfo::MESOS);
-
-  NetworkInfo* network = container->add_network_infos();
-  network->add_ip_addresses()->set_ip_address("4.3.2.1");
-  network->add_groups("public");
-
-  const ExecutorInfo& executor =
-    slave.get()->mock()->getExecutorInfo(frameworkInfo, task);
-
-  Try<string> sandbox = environment->mkdtemp();
-  ASSERT_SOME(sandbox);
-
-  SlaveID slaveID;
-  slaveID.set_value(id::UUID::random().toString());
-  Future<Containerizer::LaunchResult> launch = containerizer->launch(
-      containerId,
-      createContainerConfig(task, executor, sandbox.get(), "nobody"),
-      map<string, string>(),
-      None());
-
-  // TODO(spikecurtis): With agent capabilities (MESOS-3362), the
-  // Containerizer should fail this request since none of the listed
-  // isolators can handle NetworkInfo, which implies
-  // IP-per-container.
-  AWAIT_EXPECT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
-
-  // Wait for the container to terminate before shutting down.
-  AWAIT_READY(containerizer->wait(containerId));
-}
-
-
 // This test runs a command without the command user field set. The
 // command will verify the assumption that the command is run as the
 // slave user (in this case, root).