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

[5/5] mesos git commit: Added volume secret isolator.

Added volume secret isolator.

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


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

Branch: refs/heads/master
Commit: 1c7ffbeb505b3f5ab759202195f0b946a20cb803
Parents: e9e9e73
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Wed May 3 03:05:17 2017 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Mon May 22 16:36:22 2017 -0400

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   5 +-
 src/slave/containerizer/mesos/containerizer.cpp |   6 +
 .../mesos/isolators/volume/secret.cpp           | 300 +++++++++++++++++++
 .../mesos/isolators/volume/secret.hpp           |  67 +++++
 src/tests/CMakeLists.txt                        |   1 +
 .../environment_secret_isolator_tests.cpp       |   4 +-
 src/tests/containerizer/rootfs.cpp              |   1 +
 .../volume_secret_isolator_tests.cpp            | 263 ++++++++++++++++
 9 files changed, 645 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index fd154e2..ca7d538 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -321,6 +321,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp
   slave/containerizer/mesos/isolators/volume/image.cpp
+  slave/containerizer/mesos/isolators/volume/secret.cpp
   slave/containerizer/mesos/provisioner/backends/aufs.cpp
   slave/containerizer/mesos/provisioner/backends/bind.cpp
   slave/containerizer/mesos/provisioner/backends/overlay.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 78158a3..7e4ce85 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1204,6 +1204,7 @@ MESOS_LINUX_FILES =									\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp	\
   slave/containerizer/mesos/isolators/volume/image.cpp					\
+  slave/containerizer/mesos/isolators/volume/secret.cpp					\
   slave/containerizer/mesos/provisioner/backends/aufs.cpp				\
   slave/containerizer/mesos/provisioner/backends/bind.cpp				\
   slave/containerizer/mesos/provisioner/backends/overlay.cpp
@@ -1250,6 +1251,7 @@ MESOS_LINUX_FILES +=									\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp	\
   slave/containerizer/mesos/isolators/volume/image.hpp					\
+  slave/containerizer/mesos/isolators/volume/secret.hpp					\
   slave/containerizer/mesos/provisioner/backends/aufs.hpp				\
   slave/containerizer/mesos/provisioner/backends/bind.hpp				\
   slave/containerizer/mesos/provisioner/backends/overlay.hpp
@@ -2414,7 +2416,8 @@ mesos_tests_SOURCES +=						\
   tests/containerizer/runtime_isolator_tests.cpp		\
   tests/containerizer/sched_tests.cpp				\
   tests/containerizer/setns_test_helper.cpp			\
-  tests/containerizer/volume_image_isolator_tests.cpp
+  tests/containerizer/volume_image_isolator_tests.cpp		\
+  tests/containerizer/volume_secret_isolator_tests.cpp
 endif
 
 if ENABLE_PORT_MAPPING_ISOLATOR

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index b513e68..7695bce 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -71,6 +71,7 @@
 #include "slave/containerizer/mesos/isolators/posix/disk.hpp"
 #include "slave/containerizer/mesos/isolators/posix/rlimits.hpp"
 #include "slave/containerizer/mesos/isolators/volume/sandbox_path.hpp"
+#include "slave/containerizer/mesos/isolators/volume/secret.hpp"
 
 #include "slave/containerizer/mesos/provisioner/provisioner.hpp"
 
@@ -338,6 +339,11 @@ Try<MesosContainerizer*> MesosContainerizer::create(
         return VolumeImageIsolatorProcess::create(flags, provisioner);
       }},
 
+    {"volume/secret",
+      [secretResolver] (const Flags& flags) -> Try<Isolator*> {
+        return VolumeSecretIsolatorProcess::create(flags, secretResolver);
+      }},
+
     {"gpu/nvidia",
       [&nvidia] (const Flags& flags) -> Try<Isolator*> {
         if (!nvml::isAvailable()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/slave/containerizer/mesos/isolators/volume/secret.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.cpp b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
new file mode 100644
index 0000000..ee9366f
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
@@ -0,0 +1,300 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "slave/containerizer/mesos/isolators/volume/secret.hpp"
+
+#include <list>
+#include <string>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/collect.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/owned.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+
+#include <stout/os/mkdir.hpp>
+#include <stout/os/write.hpp>
+
+#ifdef __linux__
+#include "linux/ns.hpp"
+#endif // __linux__
+
+#include "common/validation.hpp"
+
+using std::list;
+using std::string;
+
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerClass;
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+constexpr char SECRET_DIR[] = ".secret";
+
+
+Try<Isolator*> VolumeSecretIsolatorProcess::create(
+    const Flags& flags,
+    SecretResolver* secretResolver)
+{
+  if (flags.launcher != "linux" ||
+      !strings::contains(flags.isolation, "filesystem/linux")) {
+    return Error("Volume secret isolation requires filesystem/linux isolator.");
+  }
+
+  const string hostSecretTmpDir = path::join(flags.runtime_dir, SECRET_DIR);
+
+  Try<Nothing> mkdir = os::mkdir(hostSecretTmpDir);
+  if (mkdir.isError()) {
+    return Error("Failed to create secret directory on the host tmpfs:" +
+                 mkdir.error());
+  }
+
+  Owned<MesosIsolatorProcess> process(new VolumeSecretIsolatorProcess(
+      flags,
+      secretResolver));
+
+  return new MesosIsolator(process);
+}
+
+
+VolumeSecretIsolatorProcess::VolumeSecretIsolatorProcess(
+    const Flags& _flags,
+    SecretResolver* secretResolver)
+  : ProcessBase(process::ID::generate("volume-secret-isolator")),
+    flags(_flags),
+    secretResolver(secretResolver) {}
+
+
+bool VolumeSecretIsolatorProcess::supportsNesting()
+{
+  return true;
+}
+
+
+Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  if (!containerConfig.has_container_info()) {
+    return None();
+  }
+
+  const ContainerInfo& containerInfo = containerConfig.container_info();
+
+  if (containerInfo.type() != ContainerInfo::MESOS) {
+    return Failure(
+        "Can only prepare the secret volume isolator for a MESOS container");
+  }
+
+  if (containerConfig.has_container_class() &&
+      containerConfig.container_class() == ContainerClass::DEBUG) {
+    return None();
+  }
+
+  ContainerLaunchInfo launchInfo;
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
+
+  const string sandboxSecretRootDir =
+    path::join(containerConfig.directory(),
+               SECRET_DIR + string("-") + stringify(UUID::random()));
+
+  // TODO(Kapil): Add some UUID suffix to the secret-root dir to avoid conflicts
+  // with user container_path.
+  Try<Nothing> mkdir = os::mkdir(sandboxSecretRootDir);
+  if (mkdir.isError()) {
+    return Failure("Failed to create sandbox secret root directory at '" +
+                   sandboxSecretRootDir + "': " + mkdir.error());
+  }
+
+  // Mount ramfs in the container.
+  CommandInfo* command = launchInfo.add_pre_exec_commands();
+  command->set_shell(false);
+  command->set_value("mount");
+  command->add_arguments("mount");
+  command->add_arguments("-n");
+  command->add_arguments("-t");
+  command->add_arguments("ramfs");
+  command->add_arguments("ramfs");
+  command->add_arguments(sandboxSecretRootDir);
+
+  list<Future<Nothing>> futures;
+  foreach (const Volume& volume, containerInfo.volumes()) {
+    if (!volume.has_source() ||
+        !volume.source().has_type() ||
+        volume.source().type() != Volume::Source::SECRET) {
+      continue;
+    }
+
+    if (!volume.source().has_secret()) {
+      return Failure("volume.source.secret is not specified");
+    }
+
+    if (secretResolver == nullptr) {
+      return Failure(
+          "Error: Volume has secret but no secret-resolver provided");
+    }
+
+    const Secret& secret = volume.source().secret();
+
+    Option<Error> error = common::validation::validateSecret(secret);
+    if (error.isSome()) {
+      return Failure("Invalid secret specified in volume: " + error->message);
+    }
+
+    string targetContainerPath;
+    if (path::absolute(volume.container_path())) {
+      if (containerConfig.has_rootfs()) {
+        targetContainerPath = path::join(
+            containerConfig.rootfs(),
+            volume.container_path());
+
+        Try<Nothing> mkdir = os::mkdir(Path(targetContainerPath).dirname());
+        if (mkdir.isError()) {
+          return Failure(
+              "Failed to create directory '" +
+              Path(targetContainerPath).dirname() + "' "
+              "for the target mount file: " + mkdir.error());
+        }
+
+        Try<Nothing> touch = os::touch(targetContainerPath);
+        if (touch.isError()) {
+          return Failure(
+              "Failed to create the target mount file at '" +
+              targetContainerPath + "': " + touch.error());
+        }
+      } else {
+        targetContainerPath = volume.container_path();
+
+        if (!os::exists(targetContainerPath)) {
+          return Failure(
+              "Absolute container path '" + targetContainerPath + "' "
+              "does not exist");
+        }
+      }
+    } else {
+      if (containerConfig.has_rootfs()) {
+        targetContainerPath = path::join(
+            containerConfig.rootfs(),
+            flags.sandbox_directory,
+            volume.container_path());
+      } else {
+        targetContainerPath = path::join(
+            containerConfig.directory(),
+            volume.container_path());
+      }
+
+      // Create the mount point if bind mount is used.
+      // NOTE: We cannot create the mount point at 'targetContainerPath' if
+      // container has rootfs defined. The bind mount of the sandbox
+      // will hide what's inside 'targetContainerPath'. So we should always
+      // create the mount point in the sandbox.
+      const string mountPoint = path::join(
+          containerConfig.directory(),
+          volume.container_path());
+
+      Try<Nothing> mkdir = os::mkdir(Path(mountPoint).dirname());
+      if (mkdir.isError()) {
+        return Failure(
+            "Failed to create the target mount file directory at '" +
+            Path(mountPoint).dirname() + "': " + mkdir.error());
+      }
+
+      Try<Nothing> touch = os::touch(mountPoint);
+      if (touch.isError()) {
+        return Failure(
+            "Failed to create the target mount file at '" +
+            targetContainerPath + "': " + touch.error());
+      }
+    }
+
+    const string hostSecretPath =
+      path::join(flags.runtime_dir, SECRET_DIR, stringify(UUID::random()));
+
+    const string sandboxSecretPath =
+      path::join(sandboxSecretRootDir,
+                 volume.container_path());
+
+    Try<Nothing> mkdir = os::mkdir(Path(sandboxSecretPath).dirname());
+    if (mkdir.isError()) {
+      return Failure(
+          "Failed to create the target mount file directory at '" +
+          Path(sandboxSecretPath).dirname() + "': " + mkdir.error());
+    }
+
+    // Create directory tree inside sandbox secret root dir.
+    command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mkdir");
+    command->add_arguments("mkdir");
+    command->add_arguments("-p");
+    command->add_arguments(Path(sandboxSecretPath).dirname());
+
+    // Move secret from hostSecretPath to sandboxSecretPath.
+    command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mv");
+    command->add_arguments("mv");
+    command->add_arguments("-f");
+    command->add_arguments(hostSecretPath);
+    command->add_arguments(sandboxSecretPath);
+
+    // Bind mount sandboxSecretPath to targetContainerPath
+    command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mount");
+    command->add_arguments("mount");
+    command->add_arguments("-n");
+    command->add_arguments("--rbind");
+    command->add_arguments(sandboxSecretPath);
+    command->add_arguments(targetContainerPath);
+
+    Future<Nothing> future = secretResolver->resolve(secret)
+      .then([hostSecretPath](const Secret::Value& value) -> Future<Nothing> {
+        Try<Nothing> writeSecret = os::write(hostSecretPath, value.data());
+        if (writeSecret.isError()) {
+          return Failure(
+              "Error writing secret to '" + hostSecretPath + "': " +
+              writeSecret.error());
+        }
+        return Nothing();
+      });
+
+    futures.push_back(future);
+  }
+
+  return collect(futures)
+    .then([launchInfo]() -> Future<Option<ContainerLaunchInfo>> {
+      return launchInfo;
+    });
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/slave/containerizer/mesos/isolators/volume/secret.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.hpp b/src/slave/containerizer/mesos/isolators/volume/secret.hpp
new file mode 100644
index 0000000..2680345
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.hpp
@@ -0,0 +1,67 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __VOLUME_SECRET_ISOLATOR_HPP__
+#define __VOLUME_SECRET_ISOLATOR_HPP__
+
+#include <list>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <mesos/slave/isolator.hpp>
+
+#include <process/future.hpp>
+
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class VolumeSecretIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(
+      const Flags& flags,
+      SecretResolver* secretResolver);
+
+  virtual ~VolumeSecretIsolatorProcess() {}
+
+  virtual bool supportsNesting();
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  VolumeSecretIsolatorProcess(
+      const Flags& flags,
+      SecretResolver* secretResolver);
+
+  const Flags flags;
+  SecretResolver* secretResolver;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __VOLUME_SECRET_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 8cecb72..29a46bd 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -231,6 +231,7 @@ if (LINUX)
     containerizer/runtime_isolator_tests.cpp
     containerizer/sched_tests.cpp
     containerizer/volume_image_isolator_tests.cpp
+    containerizer/volume_secret_isolator_tests.cpp
     )
 endif (LINUX)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/containerizer/environment_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/environment_secret_isolator_tests.cpp b/src/tests/containerizer/environment_secret_isolator_tests.cpp
index f8c7719..6190040 100644
--- a/src/tests/containerizer/environment_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/environment_secret_isolator_tests.cpp
@@ -85,8 +85,8 @@ TEST_F(EnvironmentSecretIsolatorTest, ResolveSecret)
 
   const string commandString = strings::format(
       "env; test \"$%s\" = \"%s\"",
-      string(SECRET_ENV_NAME),
-      string(SECRET_VALUE));
+      SECRET_ENV_NAME,
+      SECRET_VALUE).get();
 
   CommandInfo command;
   command.set_value(commandString);

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/containerizer/rootfs.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/rootfs.cpp b/src/tests/containerizer/rootfs.cpp
index fdfecc6..fe9252d 100644
--- a/src/tests/containerizer/rootfs.cpp
+++ b/src/tests/containerizer/rootfs.cpp
@@ -126,6 +126,7 @@ Try<process::Owned<Rootfs>> LinuxRootfs::create(const string& root)
   }
 
   const vector<string> programs = {
+    "/bin/cat",
     "/bin/echo",
     "/bin/ls",
     "/bin/ping",

http://git-wip-us.apache.org/repos/asf/mesos/blob/1c7ffbeb/src/tests/containerizer/volume_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_secret_isolator_tests.cpp b/src/tests/containerizer/volume_secret_isolator_tests.cpp
new file mode 100644
index 0000000..073c392
--- /dev/null
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -0,0 +1,263 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <map>
+#include <set>
+#include <string>
+
+#include <mesos/secret/resolver.hpp>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+
+#include <stout/gtest.hpp>
+
+#include "tests/mesos.hpp"
+
+#include "tests/containerizer/docker_archive.hpp"
+
+using process::Future;
+using process::Owned;
+
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::MesosContainerizer;
+
+using mesos::internal::slave::state::SlaveState;
+
+using mesos::slave::ContainerTermination;
+
+using std::map;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+const char SECRET_VALUE[] = "password";
+
+
+enum FS_TYPE {
+  WITH_ROOTFS,
+  WITHOUT_ROOTFS
+};
+
+
+enum CONTAINER_LAUNCH_STATUS {
+  CONTAINER_LAUNCH_FAILURE,
+  CONTAINER_LAUNCH_SUCCESS
+};
+
+
+class VolumeSecretIsolatorTest :
+  public MesosTest,
+  public ::testing::WithParamInterface<std::tr1::tuple<
+      const char*, const char*, enum FS_TYPE, enum CONTAINER_LAUNCH_STATUS>>
+
+{
+protected:
+  virtual void SetUp()
+  {
+    const char* prefix = std::tr1::get<0>(GetParam());
+    const char* path = std::tr1::get<1>(GetParam());
+    secretContainerPath = string(prefix) + string(path);
+
+    fsType = std::tr1::get<2>(GetParam());
+    expectedContainerLaunchStatus = std::tr1::get<3>(GetParam());
+
+    volume.set_mode(Volume::RW);
+    volume.set_container_path(secretContainerPath);
+
+    Volume::Source* source = volume.mutable_source();
+    source->set_type(Volume::Source::SECRET);
+
+    // Request a secret.
+    Secret* secret = source->mutable_secret();
+    secret->set_type(Secret::VALUE);
+    secret->mutable_value()->set_data(SECRET_VALUE);
+
+    MesosTest::SetUp();
+  }
+
+  string secretContainerPath;
+  bool expectedContainerLaunchStatus;
+  bool fsType;
+
+  Volume volume;
+};
+
+
+static const char* paths[] = {
+  "my_secret",
+  "some/my_secret",
+  "etc/my_secret",
+  "etc/some/my_secret"
+};
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithoutRootFSRelativePath,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values(""),
+                       ::testing::ValuesIn(paths),
+                       ::testing::Values(WITHOUT_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithoutRootFSNonExisitingAbsolutePath,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values("/"),
+                       ::testing::ValuesIn(paths),
+                       ::testing::Values(WITHOUT_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_FAILURE)));
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithoutRootFSExistingAbsolutePath,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values(""),
+                       ::testing::Values("/bin/touch"),
+                       ::testing::Values(WITHOUT_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+
+
+INSTANTIATE_TEST_CASE_P(
+    SecretTestTypeWithRootFS,
+    VolumeSecretIsolatorTest,
+    ::testing::Combine(::testing::Values("", "/"),
+                       ::testing::ValuesIn(paths),
+                       ::testing::Values(WITH_ROOTFS),
+                       ::testing::Values(CONTAINER_LAUNCH_SUCCESS)));
+
+
+TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,volume/secret";
+
+  if (fsType == WITH_ROOTFS) {
+    const string registry = path::join(sandbox.get(), "registry");
+    AWAIT_READY(DockerArchive::create(registry, "test_image_rootfs"));
+    AWAIT_READY(DockerArchive::create(registry, "test_image_volume"));
+
+    flags.isolation += ",volume/image,docker/runtime";
+    flags.docker_registry = registry;
+    flags.docker_store_dir = path::join(sandbox.get(), "store");
+    flags.image_providers = "docker";
+  }
+
+  Fetcher fetcher;
+
+  Try<SecretResolver*> secretResolver = SecretResolver::create();
+  EXPECT_SOME(secretResolver);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher,
+      secretResolver.get());
+
+  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());
+
+  ContainerInfo containerInfo;
+  if (fsType == WITH_ROOTFS) {
+    containerInfo = createContainerInfo(
+        "test_image_rootfs",
+        {createVolumeFromDockerImage(
+            "rootfs", "test_image_volume", Volume::RW)});
+  } else {
+    containerInfo.set_type(ContainerInfo::MESOS);
+  }
+
+  containerInfo.add_volumes()->CopyFrom(volume);
+
+  CommandInfo command = createCommandInfo(
+      "secret=$(cat " + secretContainerPath + "); "
+      "test \"$secret\" = \"" + string(SECRET_VALUE) + "\" && sleep 1000");
+
+  ExecutorInfo executor = createExecutorInfo("test_executor", command);
+  executor.mutable_container()->CopyFrom(containerInfo);
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      None(),
+      executor,
+      directory,
+      None(),
+      SlaveID(),
+      map<string, string>(),
+      false);
+
+  if (expectedContainerLaunchStatus == CONTAINER_LAUNCH_FAILURE) {
+    AWAIT_FAILED(launch);
+    return;
+  }
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(UUID::random().toString());
+
+  CommandInfo nestedCommand = createCommandInfo(
+      "secret=$(cat " + secretContainerPath + "); "
+      "test \"$secret\" = \"" + string(SECRET_VALUE) + "\"");
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      nestedCommand,
+      containerInfo,
+      None(),
+      state.id);
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  // Wait for nested container.
+  Future<Option<ContainerTermination>> wait = containerizer->wait(
+      nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Now wait for parent container.
+  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());
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {