You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by qi...@apache.org on 2020/08/18 07:43:10 UTC

[mesos] branch master updated (014431e -> 1fb79fe)

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

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


    from 014431e  Updated volume manager to support user specified target path root.
     new a3e8fd8  Implemented the framework and `create` method of `volume/csi` isolator.
     new 83ae449  Implemented the `prepare` method of `volume/csi` isolator.
     new 17db3a9  Implemented the `cleanup` method of `volume/csi` isolator.
     new 64cd6b8  Implemented the `recover` method of `volume/csi` isolator.
     new 1fb79fe  Enabled the `volume/csi` isolator in `MesosContainerizer`.

The 5 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:
 include/mesos/mesos.proto                          |   8 +-
 include/mesos/v1/mesos.proto                       |   8 +-
 src/CMakeLists.txt                                 |   3 +
 src/Makefile.am                                    |   8 +
 src/common/validation.cpp                          |  12 +
 src/slave/containerizer/containerizer.cpp          |   7 +-
 src/slave/containerizer/containerizer.hpp          |   4 +-
 src/slave/containerizer/mesos/containerizer.cpp    |  10 +-
 src/slave/containerizer/mesos/containerizer.hpp    |   4 +-
 .../mesos/isolators/volume/csi/isolator.cpp        | 546 +++++++++++++++++++++
 .../posix.hpp => volume/csi/isolator.hpp}          |  86 ++--
 .../{docker/volume => volume/csi}/paths.cpp        |  17 +-
 .../{docker/volume => volume/csi}/paths.hpp        |  24 +-
 .../{docker/volume => volume/csi}/state.hpp        |  22 +-
 .../{docker/volume => volume/csi}/state.proto      |  12 +-
 15 files changed, 695 insertions(+), 76 deletions(-)
 create mode 100644 src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
 copy src/slave/containerizer/mesos/isolators/{filesystem/posix.hpp => volume/csi/isolator.hpp} (50%)
 copy src/slave/containerizer/mesos/isolators/{docker/volume => volume/csi}/paths.cpp (74%)
 copy src/slave/containerizer/mesos/isolators/{docker/volume => volume/csi}/paths.hpp (75%)
 copy src/slave/containerizer/mesos/isolators/{docker/volume => volume/csi}/state.hpp (65%)
 copy src/slave/containerizer/mesos/isolators/{docker/volume => volume/csi}/state.proto (83%)


[mesos] 03/05: Implemented the `cleanup` method of `volume/csi` isolator.

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

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

commit 17db3a90c644b5044e2b101cb431e37c2d23fed5
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Wed Aug 5 17:14:46 2020 +0800

    Implemented the `cleanup` method of `volume/csi` isolator.
    
    Review: https://reviews.apache.org/r/72734
---
 .../mesos/isolators/volume/csi/isolator.cpp        | 78 ++++++++++++++++++++++
 .../mesos/isolators/volume/csi/isolator.hpp        |  4 ++
 2 files changed, 82 insertions(+)

diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
index 90a526f..d61fe30 100644
--- a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
@@ -318,6 +318,84 @@ Future<Option<ContainerLaunchInfo>> VolumeCSIIsolatorProcess::_prepare(
 Future<Nothing> VolumeCSIIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
+  if (!infos.contains(containerId)) {
+    VLOG(1) << "Ignoring cleanup request for unknown container " << containerId;
+    return Nothing();
+  }
+
+  hashmap<CSIVolume, int> references;
+  foreachvalue (const Owned<Info>& info, infos) {
+    foreach (const CSIVolume& volume, info->volumes) {
+      if (!references.contains(volume)) {
+        references[volume] = 1;
+      } else {
+        references[volume]++;
+      }
+    }
+  }
+
+  vector<Future<Nothing>> futures;
+
+  foreach (const CSIVolume& volume, infos[containerId]->volumes) {
+    if (references.contains(volume) && references[volume] > 1) {
+      VLOG(1) << "Cannot unpublish the volume with plugin '"
+              << volume.plugin_name() << "' and ID '" << volume.id()
+              << "' for container " << containerId
+              << " since its reference count is " << references[volume];
+      continue;
+    }
+
+    LOG(INFO) << "Unpublishing the volume with plugin '"
+              << volume.plugin_name() << "' and ID '" << volume.id()
+              << "' for container " << containerId;
+
+    // Invoke CSI server to unpublish the volumes.
+    futures.push_back(
+        csiServer->unpublishVolume(volume.plugin_name(), volume.id()));
+  }
+
+  // Erase the `Info` struct of this container before unpublishing the volumes.
+  // This is to ensure the reference count of the volume will not be wrongly
+  // increased if unpublishing volumes fail, otherwise next time when another
+  // container using the same volume is destroyed, we would NOT unpublish the
+  // volume since its reference count would be larger than 1.
+  infos.erase(containerId);
+
+  return await(futures)
+    .then(defer(
+        PID<VolumeCSIIsolatorProcess>(this),
+        &VolumeCSIIsolatorProcess::_cleanup,
+        containerId,
+        lambda::_1));
+}
+
+
+Future<Nothing> VolumeCSIIsolatorProcess::_cleanup(
+    const ContainerID& containerId,
+    const vector<Future<Nothing>>& futures)
+{
+  vector<string> messages;
+  foreach (const Future<Nothing>& future, futures) {
+    if (!future.isReady()) {
+      messages.push_back(future.isFailed() ? future.failure() : "discarded");
+    }
+  }
+
+  if (!messages.empty()) {
+    return Failure(strings::join("\n", messages));
+  }
+
+  const string containerDir = csi::paths::getContainerDir(rootDir, containerId);
+  Try<Nothing> rmdir = os::rmdir(containerDir);
+  if (rmdir.isError()) {
+    return Failure(
+        "Failed to remove the container directory at '" +
+        containerDir + "': " + rmdir.error());
+  }
+
+  LOG(INFO) << "Removed the container directory at '" << containerDir
+            << "' for container " << containerId;
+
   return Nothing();
 }
 
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
index a70da4f..e05a7b8 100644
--- a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
@@ -94,6 +94,10 @@ private:
       const std::vector<Mount>& mounts,
       const std::vector<process::Future<std::string>>& futures);
 
+  process::Future<Nothing> _cleanup(
+      const ContainerID& containerId,
+      const std::vector<process::Future<Nothing>>& futures);
+
   const Flags flags;
   CSIServer* csiServer;
 


[mesos] 05/05: Enabled the `volume/csi` isolator in `MesosContainerizer`.

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

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

commit 1fb79fefa4c28e0cf0be2686ee3d808dc30d2bfc
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Mon Aug 10 09:39:54 2020 +0800

    Enabled the `volume/csi` isolator in `MesosContainerizer`.
    
    Review: https://reviews.apache.org/r/72754
---
 src/slave/containerizer/containerizer.cpp       |  7 +++++--
 src/slave/containerizer/containerizer.hpp       |  4 +++-
 src/slave/containerizer/mesos/containerizer.cpp | 10 +++++++++-
 src/slave/containerizer/mesos/containerizer.hpp |  4 +++-
 4 files changed, 20 insertions(+), 5 deletions(-)

diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index 9e44e5e..ba3ab43 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -33,6 +33,7 @@
 
 #include "hook/manager.hpp"
 
+#include "slave/csi_server.hpp"
 #include "slave/flags.hpp"
 #include "slave/gc.hpp"
 #include "slave/slave.hpp"
@@ -220,7 +221,8 @@ Try<Containerizer*> Containerizer::create(
     GarbageCollector* gc,
     SecretResolver* secretResolver,
     VolumeGidManager* volumeGidManager,
-    PendingFutureTracker* futureTracker)
+    PendingFutureTracker* futureTracker,
+    CSIServer* csiServer)
 {
   // Get the set of containerizer types.
   const vector<string> _types = strings::split(flags.containerizers, ",");
@@ -297,7 +299,8 @@ Try<Containerizer*> Containerizer::create(
           secretResolver,
           nvidia,
           volumeGidManager,
-          futureTracker);
+          futureTracker,
+          csiServer);
 
       if (containerizer.isError()) {
         return Error("Could not create MesosContainerizer: " +
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 2b3c4c0..691fdfe 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -38,6 +38,7 @@
 
 #include "common/future_tracker.hpp"
 
+#include "slave/csi_server.hpp"
 #include "slave/gc.hpp"
 
 #include "slave/volume_gid_manager/volume_gid_manager.hpp"
@@ -78,7 +79,8 @@ public:
       GarbageCollector* gc,
       SecretResolver* secretResolver = nullptr,
       VolumeGidManager* volumeGidManager = nullptr,
-      PendingFutureTracker* futureTracker = nullptr);
+      PendingFutureTracker* futureTracker = nullptr,
+      CSIServer* csiServer = nullptr);
 
   // Determine slave resources from flags, probing the system or
   // querying a delegate.
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 3c1840c..31d45a0 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -61,6 +61,7 @@
 
 #include "module/manager.hpp"
 
+#include "slave/csi_server.hpp"
 #include "slave/gc.hpp"
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"
@@ -114,6 +115,7 @@
 #include "slave/containerizer/mesos/isolators/volume/host_path.hpp"
 #include "slave/containerizer/mesos/isolators/volume/image.hpp"
 #include "slave/containerizer/mesos/isolators/volume/secret.hpp"
+#include "slave/containerizer/mesos/isolators/volume/csi/isolator.hpp"
 #endif // __linux__
 
 #if ENABLE_SECCOMP_ISOLATOR
@@ -180,7 +182,8 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     SecretResolver* secretResolver,
     const Option<NvidiaComponents>& nvidia,
     VolumeGidManager* volumeGidManager,
-    PendingFutureTracker* futureTracker)
+    PendingFutureTracker* futureTracker,
+    CSIServer* csiServer)
 {
   Try<hashset<string>> isolations = [&flags]() -> Try<hashset<string>> {
     const vector<string> tokens(strings::tokenize(flags.isolation, ","));
@@ -467,6 +470,11 @@ Try<MesosContainerizer*> MesosContainerizer::create(
       [secretResolver] (const Flags& flags) -> Try<Isolator*> {
         return VolumeSecretIsolatorProcess::create(flags, secretResolver);
       }},
+
+    {"volume/csi",
+      [csiServer] (const Flags& flags) -> Try<Isolator*> {
+        return VolumeCSIIsolatorProcess::create(flags, csiServer);
+      }},
 #endif // __linux__
 
     // Disk isolators.
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 56e4c49..62174df 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -36,6 +36,7 @@
 #include <stout/multihashmap.hpp>
 #include <stout/os/int_fd.hpp>
 
+#include "slave/csi_server.hpp"
 #include "slave/gc.hpp"
 #include "slave/state.hpp"
 
@@ -75,7 +76,8 @@ public:
       SecretResolver* secretResolver = nullptr,
       const Option<NvidiaComponents>& nvidia = None(),
       VolumeGidManager* volumeGidManager = nullptr,
-      PendingFutureTracker* futureTracker = nullptr);
+      PendingFutureTracker* futureTracker = nullptr,
+      CSIServer* csiServer = nullptr);
 
   static Try<MesosContainerizer*> create(
       const Flags& flags,


[mesos] 04/05: Implemented the `recover` method of `volume/csi` isolator.

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

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

commit 64cd6b82786de0bde3ddaaf221b5ab9a106c87c7
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Aug 8 23:53:31 2020 +0800

    Implemented the `recover` method of `volume/csi` isolator.
    
    Review: https://reviews.apache.org/r/72753
---
 .../mesos/isolators/volume/csi/isolator.cpp        | 142 +++++++++++++++++++++
 .../mesos/isolators/volume/csi/isolator.hpp        |   2 +
 2 files changed, 144 insertions(+)

diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
index d61fe30..535974b 100644
--- a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
@@ -14,6 +14,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <list>
 #include <string>
 #include <vector>
 
@@ -35,6 +36,7 @@
 #include "slave/containerizer/mesos/isolators/volume/csi/isolator.hpp"
 #include "slave/containerizer/mesos/isolators/volume/csi/paths.hpp"
 
+using std::list;
 using std::string;
 using std::vector;
 
@@ -104,6 +106,146 @@ Future<Nothing> VolumeCSIIsolatorProcess::recover(
     const vector<ContainerState>& states,
     const hashset<ContainerID>& orphans)
 {
+  foreach (const ContainerState& state, states) {
+    const ContainerID& containerId = state.container_id();
+
+    Try<Nothing> recover = recoverContainer(containerId);
+    if (recover.isError()) {
+      return Failure(
+          "Failed to recover CSI volumes for container " +
+          stringify(containerId) + ": " + recover.error());
+    }
+  }
+
+  // Recover any orphan containers that we might have check pointed.
+  // These orphan containers will be destroyed by the containerizer
+  // through the regular cleanup path. See MESOS-2367 for details.
+  foreach (const ContainerID& containerId, orphans) {
+    Try<Nothing> recover = recoverContainer(containerId);
+    if (recover.isError()) {
+      return Failure(
+          "Failed to recover CSI volumes for orphan container " +
+          stringify(containerId) + ": " + recover.error());
+    }
+  }
+
+  // Walk through all the checkpointed containers to determine if
+  // there are any unknown orphan containers.
+  Try<list<string>> entries = os::ls(rootDir);
+  if (entries.isError()) {
+    return Failure(
+        "Unable to list CSI volume checkpoint directory '" +
+        rootDir + "': " + entries.error());
+  }
+
+  foreach (const string& entry, entries.get()) {
+    ContainerID containerId =
+      protobuf::parseContainerId(Path(entry).basename());
+
+    // Check if this container has already been recovered.
+    if (infos.contains(containerId)) {
+      continue;
+    }
+
+    // An unknown orphan container. Recover it and then clean it up.
+    Try<Nothing> recover = recoverContainer(containerId);
+    if (recover.isError()) {
+      return Failure(
+          "Failed to recover CSI volumes for orphan container " +
+          stringify(containerId) + ": " + recover.error());
+    }
+
+    LOG(INFO) << "Cleaning up CSI volumes for unknown orphaned "
+              << "container " << containerId;
+
+    cleanup(containerId);
+  }
+
+  return Nothing();
+}
+
+
+Try<Nothing> VolumeCSIIsolatorProcess::recoverContainer(
+    const ContainerID& containerId)
+{
+  const string containerDir = csi::paths::getContainerDir(rootDir, containerId);
+  if (!os::exists(containerDir)) {
+    // This may occur in the following cases:
+    //   1. The container has exited and the isolator has removed the
+    //      container directory in '_cleanup()' but agent dies before
+    //      noticing this.
+    //   2. Agent dies before the isolator checkpoints CSI volumes for
+    //      the container in 'prepare()'.
+    // For the above cases, we do not need to do anything since there
+    // is nothing to clean up for this container after agent restarts.
+    return Nothing();
+  }
+
+  const string volumesPath = csi::paths::getVolumesPath(rootDir, containerId);
+  if (!os::exists(volumesPath)) {
+    // This may occur if agent dies after creating the container directory
+    // but before it checkpoints anything in it.
+    LOG(WARNING) << "The CSI volumes checkpoint file expected at '"
+                 << volumesPath << "' for container " << containerId
+                 << " does not exist";
+
+    // Construct an info object with empty CSI volumes since no CSI volumes
+    // are mounted yet for this container, and this container will be cleaned
+    // up by containerizer (as known orphan container) or by `recover` (as
+    // unknown orphan container).
+    infos.put(containerId, Owned<Info>(new Info(hashset<CSIVolume>())));
+
+    return Nothing();
+  }
+
+  Result<string> read = state::read<string>(volumesPath);
+  if (read.isError()) {
+    return Error(
+        "Failed to read the CSI volumes checkpoint file '" +
+        volumesPath + "': " + read.error());
+  }
+
+  if (read->empty()) {
+    // This could happen if agent is hard rebooted after the checkpoint file is
+    // created but before the data is synced on disk.
+    LOG(WARNING) << "The CSI volumes checkpointed at '" << volumesPath
+                 << "' for container " << containerId << " is empty";
+
+    // Construct an info object with empty CSI volumes since no CSI volumes
+    // are mounted yet for this container, and this container will be cleaned
+    // up by containerizer (as known orphan container) or by `recover` (as
+    // unknown orphan container).
+    infos.put(containerId, Owned<Info>(new Info(hashset<CSIVolume>())));
+
+    return Nothing();
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(read.get());
+  if (json.isError()) {
+    return Error("JSON parse failed: " + json.error());
+  }
+
+  Try<CSIVolumes> parse = ::protobuf::parse<CSIVolumes>(json.get());
+  if (parse.isError()) {
+    return Error("Protobuf parse failed: " + parse.error());
+  }
+
+  hashset<CSIVolume> volumes;
+  foreach (const CSIVolume& volume, parse->volumes()) {
+    VLOG(1) << "Recovering CSI volume with plugin '" << volume.plugin_name()
+            << "' and ID '" << volume.id() << "' for container " << containerId;
+
+    if (volumes.contains(volume)) {
+      return Error(
+          "Duplicate CSI volume with plugin '" + volume.plugin_name() +
+          "' and ID '" + volume.id() + "'");
+    }
+
+    volumes.insert(volume);
+  }
+
+  infos.put(containerId, Owned<Info>(new Info(volumes)));
+
   return Nothing();
 }
 
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
index e05a7b8..373b629 100644
--- a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
@@ -98,6 +98,8 @@ private:
       const ContainerID& containerId,
       const std::vector<process::Future<Nothing>>& futures);
 
+  Try<Nothing> recoverContainer(const ContainerID& containerId);
+
   const Flags flags;
   CSIServer* csiServer;
 


[mesos] 02/05: Implemented the `prepare` method of `volume/csi` isolator.

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

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

commit 83ae449fa396d92e708e2ba8bb6f1312eb0fd5dd
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Tue Aug 4 15:44:24 2020 +0800

    Implemented the `prepare` method of `volume/csi` isolator.
    
    Review: https://reviews.apache.org/r/72733
---
 include/mesos/mesos.proto                          |   8 +-
 include/mesos/v1/mesos.proto                       |   8 +-
 src/CMakeLists.txt                                 |   1 +
 src/Makefile.am                                    |   4 +
 src/common/validation.cpp                          |  12 ++
 .../mesos/isolators/volume/csi/isolator.cpp        | 214 ++++++++++++++++++++-
 .../mesos/isolators/volume/csi/isolator.hpp        |  24 +++
 .../mesos/isolators/volume/csi/state.hpp           |  61 ++++++
 .../mesos/isolators/volume/csi/state.proto         |  29 +++
 9 files changed, 352 insertions(+), 9 deletions(-)

diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 0f91d88..661f746 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -3055,10 +3055,10 @@ message Volume {
   // TODO(gyliu513): Make this as `optional` after deprecation cycle of 1.0.
   required Mode mode = 3;
 
-  // Path pointing to a directory or file in the container. If the
-  // path is a relative path, it is relative to the container work
-  // directory. If the path is an absolute path, that path must
-  // already exist.
+  // Path pointing to a directory or file in the container. If the path
+  // is a relative path, it is relative to the container work directory.
+  // If the path is an absolute path and the container does not have its
+  // own rootfs, that path must already exist in the agent host rootfs.
   required string container_path = 1;
 
   // The following specifies the source of this volume. At most one of
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index f25db8a..ffe45c3 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -3044,10 +3044,10 @@ message Volume {
   // TODO(gyliu513): Make this as `optional` after deprecation cycle of 1.0.
   required Mode mode = 3;
 
-  // Path pointing to a directory or file in the container. If the
-  // path is a relative path, it is relative to the container work
-  // directory. If the path is an absolute path, that path must
-  // already exist.
+  // Path pointing to a directory or file in the container. If the path
+  // is a relative path, it is relative to the container work directory.
+  // If the path is an absolute path and the container does not have its
+  // own rootfs, that path must already exist in the agent host rootfs.
   required string container_path = 1;
 
   // The following specifies the source of this volume. At most one of
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index f3abdbf..a976dc1 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -96,6 +96,7 @@ if (NOT WIN32)
   PROTOC_GENERATE(INTERNAL TARGET csi/state)
   PROTOC_GENERATE(INTERNAL TARGET resource_provider/storage/disk_profile)
   PROTOC_GENERATE(INTERNAL TARGET slave/containerizer/mesos/isolators/docker/volume/state)
+  PROTOC_GENERATE(INTERNAL TARGET slave/containerizer/mesos/isolators/volume/csi/state)
   PROTOC_GENERATE(INTERNAL TARGET slave/containerizer/mesos/provisioner/docker/message)
   PROTOC_GENERATE(INTERNAL TARGET slave/volume_gid_manager/state)
 endif ()
diff --git a/src/Makefile.am b/src/Makefile.am
index 70e844d..6d68ed0 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -414,6 +414,8 @@ CXX_PROTOS +=								\
   slave/containerizer/mesos/isolators/docker/volume/state.pb.h		\
   slave/containerizer/mesos/isolators/network/cni/spec.pb.cc		\
   slave/containerizer/mesos/isolators/network/cni/spec.pb.h		\
+  slave/containerizer/mesos/isolators/volume/csi/state.pb.cc		\
+  slave/containerizer/mesos/isolators/volume/csi/state.pb.h		\
   slave/volume_gid_manager/state.pb.cc					\
   slave/volume_gid_manager/state.pb.h
 
@@ -1027,6 +1029,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
   slave/containerizer/mesos/provisioner/docker/message.proto		\
   slave/containerizer/mesos/isolators/docker/volume/state.proto		\
   slave/containerizer/mesos/isolators/network/cni/spec.proto		\
+  slave/containerizer/mesos/isolators/volume/csi/state.proto		\
   slave/volume_gid_manager/state.proto
 
 # TODO(tillt): Remove authentication/cram_md5/* which will enable us to
@@ -1253,6 +1256,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/isolators/volume/sandbox_path.hpp		\
   slave/containerizer/mesos/isolators/volume/csi/paths.cpp		\
   slave/containerizer/mesos/isolators/volume/csi/paths.hpp		\
+  slave/containerizer/mesos/isolators/volume/csi/state.hpp		\
   slave/containerizer/mesos/isolators/windows/cpu.hpp			\
   slave/containerizer/mesos/isolators/windows/mem.hpp			\
   slave/containerizer/mesos/launch.cpp					\
diff --git a/src/common/validation.cpp b/src/common/validation.cpp
index 14a8c7b..e22d564 100644
--- a/src/common/validation.cpp
+++ b/src/common/validation.cpp
@@ -255,6 +255,18 @@ Option<Error> validateVolume(const Volume& volume)
               "'source.secret' is not set for SECRET volume");
         }
         break;
+      case Volume::Source::CSI_VOLUME:
+        if (!volume.source().has_csi_volume()) {
+          return Error(
+              "'source.csi_volume' is not set for CSI volume");
+        }
+
+        if (!volume.source().csi_volume().has_static_provisioning()) {
+          return Error(
+              "'source.csi_volume.static_provisioning' "
+              "is not set for CSI volume");
+        }
+        break;
       default:
         return Error("'source.type' is unknown");
     }
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
index 7ec3a4e..90a526f 100644
--- a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
@@ -17,6 +17,7 @@
 #include <string>
 #include <vector>
 
+#include <process/collect.hpp>
 #include <process/future.hpp>
 #include <process/owned.hpp>
 
@@ -24,14 +25,23 @@
 
 #include <stout/os/realpath.hpp>
 
+#include "common/protobuf_utils.hpp"
+
+#include "linux/fs.hpp"
+#include "linux/ns.hpp"
+
+#include "slave/state.hpp"
+
 #include "slave/containerizer/mesos/isolators/volume/csi/isolator.hpp"
 #include "slave/containerizer/mesos/isolators/volume/csi/paths.hpp"
 
 using std::string;
 using std::vector;
 
+using process::Failure;
 using process::Future;
 using process::Owned;
+using process::PID;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
@@ -42,6 +52,9 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
+using AccessMode = Volume::Source::CSIVolume::VolumeCapability::AccessMode;
+
+
 Try<Isolator*> VolumeCSIIsolatorProcess::create(
     const Flags& flags,
     CSIServer* csiServer)
@@ -99,7 +112,206 @@ Future<Option<ContainerLaunchInfo>> VolumeCSIIsolatorProcess::prepare(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
-  return None();
+  if (!containerConfig.has_container_info()) {
+    return None();
+  }
+
+  if (containerConfig.container_info().type() != ContainerInfo::MESOS) {
+    return Failure("Can only prepare CSI volumes for a MESOS container");
+  }
+
+  // The hashset is used to check if there are duplicated CSI volumes for the
+  // same container.
+  hashset<CSIVolume> volumeSet;
+
+  // Represents the CSI volume mounts that we want to do for the container.
+  vector<Mount> mounts;
+
+  foreach (const Volume& _volume, containerConfig.container_info().volumes()) {
+    if (!_volume.has_source() ||
+        !_volume.source().has_type() ||
+        _volume.source().type() != Volume::Source::CSI_VOLUME) {
+      continue;
+    }
+
+    CHECK(_volume.source().has_csi_volume());
+    CHECK(_volume.source().csi_volume().has_static_provisioning());
+
+    const Volume::Source::CSIVolume& csiVolume = _volume.source().csi_volume();
+    const string& pluginName = csiVolume.plugin_name();
+    const string& volumeId = csiVolume.static_provisioning().volume_id();
+    const AccessMode& accessMode =
+      csiVolume.static_provisioning().volume_capability().access_mode();
+
+    if (csiVolume.static_provisioning().readonly() &&
+        _volume.mode() == Volume::RW) {
+      return Failure(
+          "Cannot publish the volume '" + volumeId +
+          "' in read-only mode but use it in read-write mode");
+    }
+
+    if ((accessMode.mode() == AccessMode::SINGLE_NODE_READER_ONLY ||
+         accessMode.mode() == AccessMode::MULTI_NODE_READER_ONLY) &&
+        _volume.mode() == Volume::RW) {
+      return Failure(
+          "Cannot use the read-only volume '" +
+          volumeId + "' in read-write mode");
+    }
+
+    CSIVolume volume;
+    volume.set_plugin_name(pluginName);
+    volume.set_id(volumeId);
+
+    if (volumeSet.contains(volume)) {
+      return Failure(
+          "Found duplicate CSI volume with plugin '" +
+          pluginName + "' and volume ID '" + volumeId + "'");
+    }
+
+    // Determine the target of the mount.
+    string target;
+
+    // The logic to determine a volume mount target is identical to Linux
+    // filesystem isolator, because this isolator has a dependency on that
+    // isolator, and it assumes that if the container specifies a rootfs
+    // the sandbox is already bind mounted into the container.
+    if (path::is_absolute(_volume.container_path())) {
+      // To specify a CSI volume for a container, frameworks should be allowed
+      // to define the `container_path` either as an absolute path or a relative
+      // path. Please see Linux filesystem isolator for details.
+      if (containerConfig.has_rootfs()) {
+        target = path::join(
+            containerConfig.rootfs(),
+            _volume.container_path());
+
+        Try<Nothing> mkdir = os::mkdir(target);
+        if (mkdir.isError()) {
+          return Failure(
+              "Failed to create the target of the mount at '" +
+              target + "': " + mkdir.error());
+        }
+      } else {
+        target = _volume.container_path();
+
+        if (!os::exists(target)) {
+          return Failure("Absolute container path '" + target + "' "
+                         "does not exist");
+        }
+      }
+    } else {
+      if (containerConfig.has_rootfs()) {
+        target = path::join(containerConfig.rootfs(),
+                            flags.sandbox_directory,
+                            _volume.container_path());
+      } else {
+        target = path::join(containerConfig.directory(),
+                            _volume.container_path());
+      }
+
+      // NOTE: We cannot create the mount point at `target` if
+      // container has rootfs defined. The bind mount of the sandbox
+      // will hide what's inside `target`. So we should always create
+      // the mount point in `directory`.
+      string mountPoint = path::join(
+          containerConfig.directory(),
+          _volume.container_path());
+
+      Try<Nothing> mkdir = os::mkdir(mountPoint);
+      if (mkdir.isError()) {
+        return Failure(
+            "Failed to create the target of the mount at '" +
+            mountPoint + "': " + mkdir.error());
+      }
+    }
+
+    Mount mount;
+    mount.csiVolume = csiVolume;
+    mount.target = target;
+    mount.volumeMode = _volume.mode();
+
+    mounts.push_back(mount);
+    volumeSet.insert(volume);
+  }
+
+  if (volumeSet.empty()) {
+    return None();
+  }
+
+  // Create the `CSIVolumes` protobuf message to checkpoint.
+  CSIVolumes state;
+  foreach (const CSIVolume& volume, volumeSet) {
+    state.add_volumes()->CopyFrom(volume);
+  }
+
+  const string volumesPath = csi::paths::getVolumesPath(rootDir, containerId);
+  Try<Nothing> checkpoint = state::checkpoint(volumesPath, state);
+  if (checkpoint.isError()) {
+    return Failure(
+        "Failed to checkpoint CSI volumes at '" +
+        volumesPath + "': " + checkpoint.error());
+  }
+
+  VLOG(1) << "Successfully created checkpoint at '" << volumesPath << "'";
+
+  infos.put(containerId, Owned<Info>(new Info(volumeSet)));
+
+  // Invoke CSI server to publish the volumes.
+  vector<Future<string>> futures;
+  futures.reserve(mounts.size());
+  foreach (const Mount& mount, mounts) {
+    futures.push_back(csiServer->publishVolume(mount.csiVolume));
+  }
+
+  return await(futures)
+    .then(defer(
+        PID<VolumeCSIIsolatorProcess>(this),
+        &VolumeCSIIsolatorProcess::_prepare,
+        containerId,
+        mounts,
+        lambda::_1));
+}
+
+
+Future<Option<ContainerLaunchInfo>> VolumeCSIIsolatorProcess::_prepare(
+    const ContainerID& containerId,
+    const vector<Mount>& mounts,
+    const vector<Future<string>>& futures)
+{
+
+  ContainerLaunchInfo launchInfo;
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
+
+  vector<string> messages;
+  vector<string> sources;
+  foreach (const Future<string>& future, futures) {
+    if (!future.isReady()) {
+      messages.push_back(future.isFailed() ? future.failure() : "discarded");
+      continue;
+    }
+
+    sources.push_back(strings::trim(future.get()));
+  }
+
+  if (!messages.empty()) {
+    return Failure(strings::join("\n", messages));
+  }
+
+  CHECK_EQ(sources.size(), mounts.size());
+
+  for (size_t i = 0; i < sources.size(); i++) {
+    const string& source = sources[i];
+    const Mount& mount = mounts[i];
+
+    LOG(INFO) << "Mounting CSI volume mount point '" << source
+              << "' to '" << mount.target << "' for container " << containerId;
+
+    *launchInfo.add_mounts() = protobuf::slave::createContainerMount(
+        source,
+        mount.target,
+        MS_BIND | MS_REC | (mount.volumeMode == Volume::RO ? MS_RDONLY : 0));
+  }
+
+  return launchInfo;
 }
 
 
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
index f943766..a70da4f 100644
--- a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
@@ -36,6 +36,8 @@
 
 #include "slave/containerizer/mesos/isolator.hpp"
 
+#include "slave/containerizer/mesos/isolators/volume/csi/state.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -63,6 +65,21 @@ public:
       const ContainerID& containerId) override;
 
 private:
+  struct Mount
+  {
+    Volume::Source::CSIVolume csiVolume;
+    std::string target;
+    Volume::Mode volumeMode;
+  };
+
+  struct Info
+  {
+    Info (const hashset<CSIVolume>& _volumes)
+      : volumes(_volumes) {}
+
+    hashset<CSIVolume> volumes;
+  };
+
   VolumeCSIIsolatorProcess(
       const Flags& _flags,
       CSIServer* _csiServer,
@@ -72,11 +89,18 @@ private:
     csiServer(_csiServer),
     rootDir(_rootDir) {}
 
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
+      const ContainerID& containerId,
+      const std::vector<Mount>& mounts,
+      const std::vector<process::Future<std::string>>& futures);
+
   const Flags flags;
   CSIServer* csiServer;
 
   // CSI volume information root directory.
   const std::string rootDir;
+
+  hashmap<ContainerID, process::Owned<Info>> infos;
 };
 
 } // namespace slave {
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/state.hpp b/src/slave/containerizer/mesos/isolators/volume/csi/state.hpp
new file mode 100644
index 0000000..6a547d0
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/state.hpp
@@ -0,0 +1,61 @@
+// 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_CSI_ISOLATOR_STATE_HPP__
+#define __VOLUME_CSI_ISOLATOR_STATE_HPP__
+
+#include <string>
+
+#include <boost/functional/hash.hpp>
+
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include "slave/containerizer/mesos/isolators/volume/csi/state.pb.h"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+inline bool operator==(const CSIVolume& left, const CSIVolume& right)
+{
+  return (left.plugin_name() == right.plugin_name()) && (left.id() == right.id());
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+
+namespace std {
+
+template <>
+struct hash<mesos::internal::slave::CSIVolume>
+{
+  typedef size_t result_type;
+
+  typedef mesos::internal::slave::CSIVolume argument_type;
+
+  result_type operator()(const argument_type& volume) const
+  {
+    size_t seed = 0;
+    boost::hash_combine(seed, std::hash<std::string>()(volume.plugin_name()));
+    boost::hash_combine(seed, std::hash<std::string>()(volume.id()));
+    return seed;
+  }
+};
+
+} // namespace std {
+
+#endif // __VOLUME_CSI_ISOLATOR_STATE_HPP__
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/state.proto b/src/slave/containerizer/mesos/isolators/volume/csi/state.proto
new file mode 100644
index 0000000..483fe59
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/state.proto
@@ -0,0 +1,29 @@
+// 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.
+
+syntax = "proto3";
+
+package mesos.internal.slave;
+
+message CSIVolume {
+  string plugin_name = 1;
+  string id = 2;
+}
+
+
+message CSIVolumes {
+  repeated CSIVolume volumes = 1;
+}


[mesos] 01/05: Implemented the framework and `create` method of `volume/csi` isolator.

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

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

commit a3e8fd89b1a34cf479c454e9991712cd2999affe
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Thu Jul 16 22:13:18 2020 +0800

    Implemented the framework and `create` method of `volume/csi` isolator.
    
    Review: https://reviews.apache.org/r/72690
---
 src/CMakeLists.txt                                 |   2 +
 src/Makefile.am                                    |   4 +
 .../mesos/isolators/volume/csi/isolator.cpp        | 114 +++++++++++++++++++++
 .../mesos/isolators/volume/csi/isolator.hpp        |  86 ++++++++++++++++
 .../mesos/isolators/volume/csi/paths.cpp           |  47 +++++++++
 .../mesos/isolators/volume/csi/paths.hpp           |  57 +++++++++++
 6 files changed, 310 insertions(+)

diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index c60d98a..f3abdbf 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -196,6 +196,7 @@ if (NOT WIN32)
     slave/containerizer/mesos/isolators/posix/disk.cpp
     slave/containerizer/mesos/isolators/posix/rlimits.cpp
     slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
+    slave/containerizer/mesos/isolators/volume/csi/paths.cpp
     slave/containerizer/mesos/provisioner/appc/cache.cpp
     slave/containerizer/mesos/provisioner/appc/fetcher.cpp
     slave/containerizer/mesos/provisioner/appc/paths.cpp
@@ -336,6 +337,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/volume/image.cpp
   slave/containerizer/mesos/isolators/volume/secret.cpp
   slave/containerizer/mesos/isolators/volume/utils.cpp
+  slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
   slave/containerizer/mesos/provisioner/backends/aufs.cpp
   slave/containerizer/mesos/provisioner/backends/bind.cpp
   slave/containerizer/mesos/provisioner/backends/overlay.cpp)
diff --git a/src/Makefile.am b/src/Makefile.am
index 49dab4b..70e844d 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1251,6 +1251,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/isolators/posix/rlimits.hpp			\
   slave/containerizer/mesos/isolators/volume/sandbox_path.cpp		\
   slave/containerizer/mesos/isolators/volume/sandbox_path.hpp		\
+  slave/containerizer/mesos/isolators/volume/csi/paths.cpp		\
+  slave/containerizer/mesos/isolators/volume/csi/paths.hpp		\
   slave/containerizer/mesos/isolators/windows/cpu.hpp			\
   slave/containerizer/mesos/isolators/windows/mem.hpp			\
   slave/containerizer/mesos/launch.cpp					\
@@ -1454,6 +1456,8 @@ MESOS_LINUX_FILES =									\
   slave/containerizer/mesos/isolators/volume/secret.hpp					\
   slave/containerizer/mesos/isolators/volume/utils.cpp					\
   slave/containerizer/mesos/isolators/volume/utils.hpp					\
+  slave/containerizer/mesos/isolators/volume/csi/isolator.cpp				\
+  slave/containerizer/mesos/isolators/volume/csi/isolator.hpp				\
   slave/containerizer/mesos/provisioner/backends/aufs.cpp				\
   slave/containerizer/mesos/provisioner/backends/aufs.hpp				\
   slave/containerizer/mesos/provisioner/backends/bind.cpp				\
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
new file mode 100644
index 0000000..7ec3a4e
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.cpp
@@ -0,0 +1,114 @@
+// 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 <string>
+#include <vector>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+
+#include <stout/os.hpp>
+
+#include <stout/os/realpath.hpp>
+
+#include "slave/containerizer/mesos/isolators/volume/csi/isolator.hpp"
+#include "slave/containerizer/mesos/isolators/volume/csi/paths.hpp"
+
+using std::string;
+using std::vector;
+
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Isolator*> VolumeCSIIsolatorProcess::create(
+    const Flags& flags,
+    CSIServer* csiServer)
+{
+  if (!strings::contains(flags.isolation, "filesystem/linux")) {
+    return Error("'filesystem/linux' isolator must be used");
+  }
+
+  if (csiServer == nullptr) {
+    return Error("No CSI server is provided");
+  }
+
+  const string csiRootDir = path::join(flags.runtime_dir, csi::paths::CSI_DIR);
+
+  // Create the CSI volume information root directory if it does not exist.
+  Try<Nothing> mkdir = os::mkdir(csiRootDir);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create CSI volume information root directory at '" +
+        csiRootDir + "': " + mkdir.error());
+  }
+
+  Result<string> rootDir = os::realpath(csiRootDir);
+  if (!rootDir.isSome()) {
+    return Error(
+        "Failed to determine canonical path of CSI volume information root"
+        " directory '" + csiRootDir + "': " +
+        (rootDir.isError() ? rootDir.error() : "No such file or directory"));
+  }
+
+  Owned<MesosIsolatorProcess> process(new VolumeCSIIsolatorProcess(
+      flags,
+      csiServer,
+      rootDir.get()));
+
+  return new MesosIsolator(process);
+}
+
+
+bool VolumeCSIIsolatorProcess::supportsNesting()
+{
+  return true;
+}
+
+
+Future<Nothing> VolumeCSIIsolatorProcess::recover(
+    const vector<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  return Nothing();
+}
+
+
+Future<Option<ContainerLaunchInfo>> VolumeCSIIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  return None();
+}
+
+
+Future<Nothing> VolumeCSIIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  return Nothing();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
new file mode 100644
index 0000000..f943766
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/isolator.hpp
@@ -0,0 +1,86 @@
+// 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_CSI_ISOLATOR_HPP__
+#define __VOLUME_CSI_ISOLATOR_HPP__
+
+#include <string>
+#include <vector>
+
+#include <mesos/mesos.hpp>
+
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/process.hpp>
+
+#include <stout/hashset.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+#include "slave/csi_server.hpp"
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class VolumeCSIIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(
+      const Flags& flags,
+      CSIServer* csiServer);
+
+  ~VolumeCSIIsolatorProcess() override {};
+
+  bool supportsNesting() override;
+
+  process::Future<Nothing> recover(
+      const std::vector<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans) override;
+
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig) override;
+
+  process::Future<Nothing> cleanup(
+      const ContainerID& containerId) override;
+
+private:
+  VolumeCSIIsolatorProcess(
+      const Flags& _flags,
+      CSIServer* _csiServer,
+      const std::string& _rootDir)
+  : ProcessBase(process::ID::generate("volume-csi-isolator")),
+    flags(_flags),
+    csiServer(_csiServer),
+    rootDir(_rootDir) {}
+
+  const Flags flags;
+  CSIServer* csiServer;
+
+  // CSI volume information root directory.
+  const std::string rootDir;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __VOLUME_CSI_ISOLATOR_HPP__
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/paths.cpp b/src/slave/containerizer/mesos/isolators/volume/csi/paths.cpp
new file mode 100644
index 0000000..4fdbc29
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/paths.cpp
@@ -0,0 +1,47 @@
+// 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 <mesos/type_utils.hpp>
+
+#include <stout/path.hpp>
+#include <stout/stringify.hpp>
+
+#include "slave/containerizer/mesos/isolators/volume/csi/paths.hpp"
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace csi {
+namespace paths {
+
+string getContainerDir(const string& rootDir, const ContainerID& containerId)
+{
+  return path::join(rootDir, stringify(containerId));
+}
+
+
+string getVolumesPath(const string& rootDir, const ContainerID& containerId)
+{
+  return path::join(getContainerDir(rootDir, containerId), "volumes");
+}
+
+} // namespace paths {
+} // namespace csi {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
diff --git a/src/slave/containerizer/mesos/isolators/volume/csi/paths.hpp b/src/slave/containerizer/mesos/isolators/volume/csi/paths.hpp
new file mode 100644
index 0000000..5b4a4ee
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/csi/paths.hpp
@@ -0,0 +1,57 @@
+// 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_CSI_ISOLATOR_PATHS_HPP__
+#define __VOLUME_CSI_ISOLATOR_PATHS_HPP__
+
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace csi {
+namespace paths {
+
+// The root directory where we keep the information of CSI volumes that each
+// container uses. The layout is as follows:
+//   /<runtime_dir>/isolators/volume/csi/
+//      |-- <ID of Container1>/
+//      |      |-- volumes
+//      |-- <ID of Container2>/
+//      |      |-- volumes
+//      |-- <ID of Container3>/
+//      |-- ...
+constexpr char CSI_DIR[] = "isolators/volume/csi";
+
+
+std::string getContainerDir(
+    const std::string& rootDir,
+    const ContainerID& containerId);
+
+
+std::string getVolumesPath(
+    const std::string& rootDir,
+    const ContainerID& containerId);
+
+} // namespace paths {
+} // namespace csi {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __VOLUME_CSI_ISOLATOR_PATHS_HPP__