You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2019/08/15 18:49:41 UTC

[mesos] branch master updated (ea95348 -> 244ce66)

This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from ea95348  Made sure we are tracking ephemeral quota before getting the usage.
     new 8498a9b  Implemented `cleanup` method for `volume/secret` isolator.
     new 34330fb  Moved const string `.secret` to paths.hpp.
     new 244ce66  Added a test `VolumeSecretIsolatorCleanupTest.ROOT_FailInPreparing`.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../mesos/isolators/volume/secret.cpp              | 50 +++++++++--
 .../mesos/isolators/volume/secret.hpp              |  3 +
 src/slave/containerizer/mesos/paths.hpp            |  1 +
 .../containerizer/volume_secret_isolator_tests.cpp | 97 ++++++++++++++++++++++
 4 files changed, 144 insertions(+), 7 deletions(-)


[mesos] 01/03: Implemented `cleanup` method for `volume/secret` isolator.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 8498a9b262cd145fd4966f621b91353bb162b56c
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Thu Aug 15 11:49:22 2019 -0700

    Implemented `cleanup` method for `volume/secret` isolator.
    
    Previously, after `volume/secret` isolator resolves a secret and write
    it into a path (i.e., <runtime_dir>/.secret/<UUID>) on agent host for a
    container, if the container fails to launch somehow (e.g., fails in
    another isolator's `prepare` method), that path on the host will never
    be cleaned up. In this patch, `volume/secret` isolator is improved to
    write all the resolved secrets for a container into a single directory
    (i.e., <runtime_dir>/.secret/<containerID>) on agent host, and the
    `cleanup` method of the `volume/secret` isolator is implemented to
    remove that directory when the container is destroyed.
    
    Review: https://reviews.apache.org/r/71201/
---
 .../mesos/isolators/volume/secret.cpp              | 38 ++++++++++++++++++++--
 .../mesos/isolators/volume/secret.hpp              |  3 ++
 2 files changed, 39 insertions(+), 2 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.cpp b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
index 4bbcc7a..5131ecb 100644
--- a/src/slave/containerizer/mesos/isolators/volume/secret.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
@@ -33,6 +33,7 @@
 #include <stout/strings.hpp>
 
 #include <stout/os/mkdir.hpp>
+#include <stout/os/rmdir.hpp>
 #include <stout/os/touch.hpp>
 #include <stout/os/write.hpp>
 
@@ -128,6 +129,18 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
     return None();
   }
 
+  const string containerDir = path::join(
+      flags.runtime_dir,
+      SECRET_DIR,
+      stringify(containerId));
+
+  Try<Nothing> mkdir = os::mkdir(containerDir);
+  if (mkdir.isError()) {
+    return Failure(
+        "Failed to create container directory at '" +
+        containerDir + "': " + mkdir.error());
+  }
+
   ContainerLaunchInfo launchInfo;
   launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
@@ -137,7 +150,7 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
 
   // TODO(Kapil): Add some UUID suffix to the secret-root dir to avoid conflicts
   // with user container_path.
-  Try<Nothing> mkdir = os::mkdir(sandboxSecretRootDir);
+  mkdir = os::mkdir(sandboxSecretRootDir);
   if (mkdir.isError()) {
     return Failure("Failed to create sandbox secret root directory at '" +
                    sandboxSecretRootDir + "': " + mkdir.error());
@@ -238,7 +251,7 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
     }
 
     const string hostSecretPath =
-      path::join(flags.runtime_dir, SECRET_DIR, stringify(id::UUID::random()));
+      path::join(containerDir, stringify(id::UUID::random()));
 
     const string sandboxSecretPath =
       path::join(sandboxSecretRootDir,
@@ -290,6 +303,27 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
     });
 }
 
+
+Future<Nothing> VolumeSecretIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  const string containerDir = path::join(
+      flags.runtime_dir,
+      SECRET_DIR,
+      stringify(containerId));
+
+  if (os::exists(containerDir)) {
+    Try<Nothing> rmdir = os::rmdir(containerDir);
+    if (rmdir.isError()) {
+      return Failure(
+          "Failed to remove the container directory '" +
+          containerDir + "': " + rmdir.error());
+    }
+  }
+
+  return Nothing();
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {
diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.hpp b/src/slave/containerizer/mesos/isolators/volume/secret.hpp
index a166491..e3cf713 100644
--- a/src/slave/containerizer/mesos/isolators/volume/secret.hpp
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.hpp
@@ -51,6 +51,9 @@ public:
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig) override;
 
+  process::Future<Nothing> cleanup(
+      const ContainerID& containerId) override;
+
 private:
   VolumeSecretIsolatorProcess(
       const Flags& flags,


[mesos] 02/03: Moved const string `.secret` to paths.hpp.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 34330fb08466116c8483ce6de234126a6089a683
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Thu Aug 15 11:49:23 2019 -0700

    Moved const string `.secret` to paths.hpp.
    
    Review: https://reviews.apache.org/r/71221/
---
 .../containerizer/mesos/isolators/volume/secret.cpp      | 16 +++++++++-------
 src/slave/containerizer/mesos/paths.hpp                  |  1 +
 2 files changed, 10 insertions(+), 7 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/volume/secret.cpp b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
index 5131ecb..91d7013 100644
--- a/src/slave/containerizer/mesos/isolators/volume/secret.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/secret.cpp
@@ -14,6 +14,8 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include "slave/containerizer/mesos/paths.hpp"
+
 #include "slave/containerizer/mesos/isolators/volume/secret.hpp"
 
 #include <sys/mount.h>
@@ -56,6 +58,8 @@ using mesos::internal::protobuf::slave::containerMountOperation;
 using mesos::internal::protobuf::slave::containerRenameOperation;
 using mesos::internal::protobuf::slave::createContainerMount;
 
+using mesos::internal::slave::containerizer::paths::SECRET_DIRECTORY;
+
 using mesos::slave::ContainerClass;
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
@@ -67,9 +71,6 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
-constexpr char SECRET_DIR[] = ".secret";
-
-
 Try<Isolator*> VolumeSecretIsolatorProcess::create(
     const Flags& flags,
     SecretResolver* secretResolver)
@@ -79,7 +80,8 @@ Try<Isolator*> VolumeSecretIsolatorProcess::create(
     return Error("Volume secret isolation requires filesystem/linux isolator.");
   }
 
-  const string hostSecretTmpDir = path::join(flags.runtime_dir, SECRET_DIR);
+  const string hostSecretTmpDir =
+    path::join(flags.runtime_dir, SECRET_DIRECTORY);
 
   Try<Nothing> mkdir = os::mkdir(hostSecretTmpDir);
   if (mkdir.isError()) {
@@ -131,7 +133,7 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
 
   const string containerDir = path::join(
       flags.runtime_dir,
-      SECRET_DIR,
+      SECRET_DIRECTORY,
       stringify(containerId));
 
   Try<Nothing> mkdir = os::mkdir(containerDir);
@@ -146,7 +148,7 @@ Future<Option<ContainerLaunchInfo>> VolumeSecretIsolatorProcess::prepare(
 
   const string sandboxSecretRootDir =
     path::join(containerConfig.directory(),
-               SECRET_DIR + string("-") + stringify(id::UUID::random()));
+               SECRET_DIRECTORY + string("-") + stringify(id::UUID::random()));
 
   // TODO(Kapil): Add some UUID suffix to the secret-root dir to avoid conflicts
   // with user container_path.
@@ -309,7 +311,7 @@ Future<Nothing> VolumeSecretIsolatorProcess::cleanup(
 {
   const string containerDir = path::join(
       flags.runtime_dir,
-      SECRET_DIR,
+      SECRET_DIRECTORY,
       stringify(containerId));
 
   if (os::exists(containerDir)) {
diff --git a/src/slave/containerizer/mesos/paths.hpp b/src/slave/containerizer/mesos/paths.hpp
index c003335..e35d380 100644
--- a/src/slave/containerizer/mesos/paths.hpp
+++ b/src/slave/containerizer/mesos/paths.hpp
@@ -84,6 +84,7 @@ constexpr char CONTAINER_LAUNCH_INFO_FILE[] = "launch_info";
 constexpr char STANDALONE_MARKER_FILE[] = "standalone.marker";
 constexpr char CONTAINER_SHM_DIRECTORY[] = "shm";
 constexpr char AGENT_SHM_DIRECTORY[] = "/dev/shm";
+constexpr char SECRET_DIRECTORY[] = ".secret";
 
 
 enum Mode


[mesos] 03/03: Added a test `VolumeSecretIsolatorCleanupTest.ROOT_FailInPreparing`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 244ce660ba70abb9bd772abd6b0605ae1eb9366b
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Thu Aug 15 11:49:24 2019 -0700

    Added a test `VolumeSecretIsolatorCleanupTest.ROOT_FailInPreparing`.
    
    Review: https://reviews.apache.org/r/71222/
---
 .../containerizer/volume_secret_isolator_tests.cpp | 97 ++++++++++++++++++++++
 1 file changed, 97 insertions(+)

diff --git a/src/tests/containerizer/volume_secret_isolator_tests.cpp b/src/tests/containerizer/volume_secret_isolator_tests.cpp
index b6c43c3..e68d12f 100644
--- a/src/tests/containerizer/volume_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -25,6 +25,8 @@
 
 #include <stout/gtest.hpp>
 
+#include "slave/containerizer/mesos/paths.hpp"
+
 #include "tests/mesos.hpp"
 
 #include "tests/containerizer/docker_archive.hpp"
@@ -38,8 +40,11 @@ using mesos::internal::slave::MesosContainerizer;
 
 using mesos::internal::slave::state::SlaveState;
 
+using mesos::internal::slave::containerizer::paths::SECRET_DIRECTORY;
+
 using mesos::slave::ContainerTermination;
 
+using std::list;
 using std::map;
 using std::string;
 
@@ -270,6 +275,98 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
   EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
 
+
+class VolumeSecretIsolatorCleanupTest : public MesosTest {};
+
+
+// This test verifies that container directory created by `volume/secret`
+// isolator can be cleaned up when the container is destroyed.
+TEST_F(VolumeSecretIsolatorCleanupTest, ROOT_FailInPreparing)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,volume/secret,network/cni";
+
+  Fetcher fetcher(flags);
+
+  Try<SecretResolver*> secretResolver = SecretResolver::create();
+  EXPECT_SOME(secretResolver);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher,
+      nullptr,
+      secretResolver.get());
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  Volume volume;
+  volume.set_mode(Volume::RW);
+  volume.set_container_path("my_secret");
+
+  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);
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.add_volumes()->CopyFrom(volume);
+
+  // Specify a nonexistent CNI network to make container fails to launch.
+  NetworkInfo* networkInfo = containerInfo.add_network_infos();
+  networkInfo->set_name("nonexistent_network");
+
+  ExecutorInfo executor = createExecutorInfo("test_executor", "sleep 1000");
+  executor.mutable_container()->CopyFrom(containerInfo);
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(None(), executor, directory),
+      map<string, string>(),
+      None());
+
+  AWAIT_FAILED(launch);
+
+  // Check the container directory is created.
+  const string containerDir = path::join(
+      flags.runtime_dir,
+      SECRET_DIRECTORY,
+      stringify(containerId));
+
+  ASSERT_TRUE(os::exists(containerDir));
+
+  // Check there is one secret resolved and written to the container directory.
+  Try<list<string>> secretFiles = os::ls(containerDir);
+  ASSERT_SOME(secretFiles);
+  ASSERT_EQ(secretFiles->size(), 1u);
+
+  // Destroy the container.
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
+
+  AWAIT_READY(termination);
+
+  // Check the container directory is removed.
+  ASSERT_FALSE(os::exists(containerDir));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {