You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2017/05/11 10:16:46 UTC

[05/11] mesos git commit: Inherited environment from parent when launching a DEBUG container.

Inherited environment from parent when launching a DEBUG container.

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


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

Branch: refs/heads/master
Commit: e4940a5af27660332cbacb9fbeddd5ec5655d98a
Parents: 37f212e
Author: Alexander Rukletsov <al...@apache.org>
Authored: Fri Apr 28 17:09:51 2017 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Thu May 11 12:15:55 2017 +0200

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp |  54 ++++--
 .../nested_mesos_containerizer_tests.cpp        | 181 ++++++++++++++++++-
 2 files changed, 218 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e4940a5a/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 3ff0b0d..4c24f0b 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1420,24 +1420,39 @@ Future<bool> MesosContainerizerProcess::_launch(
   launchInfo.mutable_command()->clear_environment();
   launchInfo.mutable_command()->clear_user();
 
-  // Determine the environment for the command to be launched. The
-  // priority of the environment should be:
+  // Determine the environment for the command to be launched.
+  //
+  // For non-DEBUG containers, the priority of the environment is:
   //  1) User specified environment in CommandInfo.
   //  2) Environment returned by isolators (i.e., in 'launchInfo').
   //  3) Environment passed from agent (e.g., executor environment).
+  //
+  // DEBUG containers inherit parent's environment,
+  // hence the priority of the environment is:
+  //
+  // 1) User specified environment in CommandInfo.
+  // 2) Environment returned by isolators (i.e., in 'launchInfo').
+  // 3) Environment passed from agent (e.g., executor environment).
+  // 4) Environment inherited from the parent container.
+  //
+  // TODO(alexr): Consider using `hashmap` for merging environments to
+  // avoid duplicates, because `MergeFrom()` appends to the list.
+  Environment containerEnvironment;
 
-  // Save a copy of the environment returned by isolators because
-  // earlier entries in 'launchInfo.environment' will be overwritten
-  // by later entries. 'launchInfo.environment' will later be used as
-  // the environment for the container, and the container will not
-  // inherit the agent environment.
-  Environment isolatorEnvironment = launchInfo.environment();
-
-  Environment* containerEnvironment = launchInfo.mutable_environment();
-  containerEnvironment->Clear();
+  // Inherit environment from the parent container for DEBUG containers.
+  if (container->config.has_container_class() &&
+      container->config.container_class() == ContainerClass::DEBUG) {
+    // DEBUG containers must have a parent.
+    CHECK(containerId.has_parent());
+    if (containers_[containerId.parent()]->launchInfo.isSome()) {
+      containerEnvironment.CopyFrom(
+          containers_[containerId.parent()]->launchInfo->environment());
+    }
+  }
 
+  // Include environment passed from agent.
   foreachpair (const string& key, const string& value, environment) {
-    Environment::Variable* variable = containerEnvironment->add_variables();
+    Environment::Variable* variable = containerEnvironment.add_variables();
     variable->set_name(key);
     variable->set_value(value);
   }
@@ -1452,21 +1467,28 @@ Future<bool> MesosContainerizerProcess::_launch(
     // filesystem for itself, we still set 'MESOS_SANDBOX' according to
     // the root filesystem of the task (if specified). Command executor
     // itself does not use this environment variable.
-    Environment::Variable* variable = containerEnvironment->add_variables();
+    Environment::Variable* variable = containerEnvironment.add_variables();
     variable->set_name("MESOS_SANDBOX");
     variable->set_value(container->config.has_rootfs()
       ? flags.sandbox_directory
       : container->config.directory());
   }
 
-  containerEnvironment->MergeFrom(isolatorEnvironment);
+  // `launchInfo.environment` contains the environment returned by
+  // isolators. `launchInfo.environment` will later be overwritten
+  // by `containerEnvironment`, hence isolator environment will
+  // contribute to the resulting container environment.
+  containerEnvironment.MergeFrom(launchInfo.environment());
 
-  // Include any user specified environment variables.
+  // Include user specified environment.
   if (container->config.command_info().has_environment()) {
-    containerEnvironment->MergeFrom(
+    containerEnvironment.MergeFrom(
         container->config.command_info().environment());
   }
 
+  // Set the aggregated environment of the launch command.
+  launchInfo.mutable_environment()->CopyFrom(containerEnvironment);
+
   // Determine the rootfs for the container to be launched.
   //
   // NOTE: Command task is a special case. Even if the container

http://git-wip-us.apache.org/repos/asf/mesos/blob/e4940a5a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index c163b88..3f06df2 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -240,6 +240,185 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
 }
 
 
+// This test verifies that a debug container inherits the
+// environment of its parent even after agent failover.
+TEST_F(NestedMesosContainerizerTest,
+       ROOT_CGROUPS_DebugNestedContainerInheritsEnvironment)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher;
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  const string envKey = "MESOS_NESTED_INHERITS_ENVIRONMENT";
+  const int32_t envValue = 42;
+  mesos::Environment env = createEnvironment({{envKey, stringify(envValue)}});
+
+  CommandInfo command = createCommandInfo("sleep 1000");
+  command.mutable_environment()->CopyFrom(env);
+
+  ExecutorInfo executor = createExecutorInfo("executor", command, "cpus:1");
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory.get(),
+      None(),
+      state.id,
+      map<string, string>(),
+      true); // TODO(benh): Ever want to test not checkpointing?
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<ContainerStatus> status = containerizer->status(containerId);
+  AWAIT_READY(status);
+  ASSERT_TRUE(status->has_executor_pid());
+
+  pid_t pid = status->executor_pid();
+
+  // Launch a nested debug container that accesses the
+  // environment variable specified for its parent.
+  {
+    ContainerID nestedContainerId;
+    nestedContainerId.mutable_parent()->CopyFrom(containerId);
+    nestedContainerId.set_value(UUID::random().toString());
+
+    Future<bool> launchNested = containerizer->launch(
+        nestedContainerId,
+        createCommandInfo("exit $" + envKey),
+        None(),
+        None(),
+        state.id,
+        ContainerClass::DEBUG);
+
+    AWAIT_ASSERT_TRUE(launchNested);
+
+    Future<Option<ContainerTermination>> waitNested = containerizer->wait(
+        nestedContainerId);
+
+    AWAIT_READY(waitNested);
+    ASSERT_SOME(waitNested.get());
+    ASSERT_TRUE(waitNested.get()->has_status());
+    EXPECT_WEXITSTATUS_EQ(envValue, waitNested.get()->status());
+  }
+
+  // Launch a nested debug container that overwrites the
+  // environment variable specified for its parent.
+  {
+    const int32_t envOverrideValue = 99;
+    mesos::Environment env = createEnvironment(
+        {{envKey, stringify(envOverrideValue)}});
+
+    CommandInfo nestedCommand = createCommandInfo("exit $" + envKey);
+    nestedCommand.mutable_environment()->CopyFrom(env);
+
+    ContainerID nestedContainerId;
+    nestedContainerId.mutable_parent()->CopyFrom(containerId);
+    nestedContainerId.set_value(UUID::random().toString());
+
+    Future<bool> launchNested = containerizer->launch(
+        nestedContainerId,
+        nestedCommand,
+        None(),
+        None(),
+        state.id,
+        ContainerClass::DEBUG);
+
+    AWAIT_ASSERT_TRUE(launchNested);
+
+    Future<Option<ContainerTermination>> waitNested = containerizer->wait(
+        nestedContainerId);
+
+    AWAIT_READY(waitNested);
+    ASSERT_SOME(waitNested.get());
+    ASSERT_TRUE(waitNested.get()->has_status());
+    EXPECT_WEXITSTATUS_EQ(envOverrideValue, waitNested.get()->status());
+  }
+
+  // Force a delete on the containerizer to emulate recovery.
+  containerizer.reset();
+
+  create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  containerizer.reset(create.get());
+
+  Try<SlaveState> slaveState = createSlaveState(
+      containerId,
+      pid,
+      executor,
+      state.id,
+      flags.work_dir);
+
+  ASSERT_SOME(slaveState);
+
+  state = slaveState.get();
+  AWAIT_READY(containerizer->recover(state));
+
+  // Launch a nested debug container that access the
+  // environment variable specified for its parent.
+  {
+    ContainerID nestedContainerId;
+    nestedContainerId.mutable_parent()->CopyFrom(containerId);
+    nestedContainerId.set_value(UUID::random().toString());
+
+    Future<bool> launchNested = containerizer->launch(
+        nestedContainerId,
+        createCommandInfo("exit $" + envKey),
+        None(),
+        None(),
+        state.id,
+        ContainerClass::DEBUG);
+
+    AWAIT_ASSERT_TRUE(launchNested);
+
+    Future<Option<ContainerTermination>> waitNested = containerizer->wait(
+        nestedContainerId);
+
+    AWAIT_READY(waitNested);
+    ASSERT_SOME(waitNested.get());
+    ASSERT_TRUE(waitNested.get()->has_status());
+    EXPECT_WEXITSTATUS_EQ(envValue, waitNested.get()->status());
+  }
+
+  // Destroy the containerizer with all associated containers.
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  containerizer->destroy(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
+}
+
+
 TEST_F(NestedMesosContainerizerTest,
        ROOT_CGROUPS_LaunchNestedDebugCheckPidNamespace)
 {
@@ -1814,7 +1993,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
 // This test verifies that agent environment variables are not leaked
 // to the nested container, and the environment variables specified in
 // the command for the nested container will be honored.
-TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Environment)
+TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
 {
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "linux";