You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2016/07/02 23:44:17 UTC

[1/2] mesos git commit: Added a new 'NvidiaVolume' component.

Repository: mesos
Updated Branches:
  refs/heads/master 5ed7389f4 -> 14458c81e


Added a new 'NvidiaVolume' component.

This component is responsible for building up a consolidated volume of
binaries / libraries from the user-space portion of the Nvidia GPU
drivers so that it can be injected into a container as a single
volume. Its core functionality mimics that of the
'nvidia-docker-plugin': https://github.com/NVIDIA/nvidia-docker/

We currently only implement the 'create()' function which is
responsible for building up the volume itself. In a subsequent commit
we will add support for reading a Docker image manifest and deciding
whether we should inject the volume into the docker container.

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


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

Branch: refs/heads/master
Commit: a7185a88c67eb462a0fe8da805193ae027fadca3
Parents: 5ed7389
Author: Kevin Klues <kl...@gmail.com>
Authored: Sat Jul 2 15:41:05 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Sat Jul 2 16:39:24 2016 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 .../mesos/isolators/gpu/nvidia.hpp              |   1 +
 .../mesos/isolators/gpu/volume.cpp              | 390 +++++++++++++++++++
 .../mesos/isolators/gpu/volume.hpp              |  72 ++++
 .../containerizer/nvidia_gpu_isolator_tests.cpp |  21 +
 6 files changed, 487 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a7185a88/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 09dd0e0..bf2db03 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -170,6 +170,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/gpu/allocator.cpp
   slave/containerizer/mesos/isolators/gpu/isolator.cpp
   slave/containerizer/mesos/isolators/gpu/nvml.cpp
+  slave/containerizer/mesos/isolators/gpu/volume.cpp
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp
   slave/containerizer/mesos/provisioner/backends/aufs.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7185a88/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 53f21cc..3522b77 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1055,6 +1055,7 @@ MESOS_LINUX_FILES =							\
   slave/containerizer/mesos/isolators/gpu/allocator.cpp			\
   slave/containerizer/mesos/isolators/gpu/isolator.cpp			\
   slave/containerizer/mesos/isolators/gpu/nvml.cpp			\
+  slave/containerizer/mesos/isolators/gpu/volume.cpp			\
   slave/containerizer/mesos/isolators/namespaces/pid.cpp		\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp		\
   slave/containerizer/mesos/provisioner/backends/aufs.cpp		\
@@ -1085,6 +1086,7 @@ MESOS_LINUX_FILES +=							\
   slave/containerizer/mesos/isolators/gpu/isolator.hpp			\
   slave/containerizer/mesos/isolators/gpu/nvidia.hpp			\
   slave/containerizer/mesos/isolators/gpu/nvml.hpp			\
+  slave/containerizer/mesos/isolators/gpu/volume.hpp			\
   slave/containerizer/mesos/isolators/namespaces/pid.hpp		\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp		\
   slave/containerizer/mesos/provisioner/backends/aufs.hpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7185a88/src/slave/containerizer/mesos/isolators/gpu/nvidia.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/gpu/nvidia.hpp b/src/slave/containerizer/mesos/isolators/gpu/nvidia.hpp
index 01cdab2..85cfa68 100644
--- a/src/slave/containerizer/mesos/isolators/gpu/nvidia.hpp
+++ b/src/slave/containerizer/mesos/isolators/gpu/nvidia.hpp
@@ -21,6 +21,7 @@
 #include "slave/containerizer/mesos/isolators/gpu/allocator.hpp"
 #include "slave/containerizer/mesos/isolators/gpu/isolator.hpp"
 #include "slave/containerizer/mesos/isolators/gpu/nvml.hpp"
+#include "slave/containerizer/mesos/isolators/gpu/volume.hpp"
 #endif
 
 #include "slave/containerizer/mesos/isolators/gpu/components.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7185a88/src/slave/containerizer/mesos/isolators/gpu/volume.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/gpu/volume.cpp b/src/slave/containerizer/mesos/isolators/gpu/volume.cpp
new file mode 100644
index 0000000..9aac3cc
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/gpu/volume.cpp
@@ -0,0 +1,390 @@
+// 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 <linux/limits.h>
+
+#include <string>
+#include <vector>
+
+#include <mesos/docker/spec.hpp>
+
+#include <process/owned.hpp>
+
+#include <stout/elf.hpp>
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/fs.hpp>
+#include <stout/nothing.hpp>
+#include <stout/path.hpp>
+#include <stout/result.hpp>
+#include <stout/strings.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/mkdir.hpp>
+#include <stout/os/realpath.hpp>
+#include <stout/os/rmdir.hpp>
+#include <stout/os/shell.hpp>
+
+#include "linux/ldcache.hpp"
+
+#include "slave/containerizer/mesos/isolators/gpu/nvml.hpp"
+#include "slave/containerizer/mesos/isolators/gpu/volume.hpp"
+
+using docker::spec::v1::ImageManifest;
+
+using process::Owned;
+
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Much of the logic in this file (including the contents of the
+// `BINARIES` and `LIBRARIES` arrays below) is borrowed from the
+// nvidia-docker-plugin (https://github.com/NVIDIA/nvidia-docker).
+// Copyright (c) 2015-2016, NVIDIA CORPORATION. All rights reserved.
+
+static constexpr char HOST_VOLUME_PATH_PREFIX[] =
+  "/var/run/mesos/isolators/gpu/nvidia_";
+static constexpr char CONTAINER_VOLUME_PATH[] =
+  "/usr/local/nvidia";
+
+
+static constexpr const char* BINARIES[] = {
+  // "nvidia-modprobe",      // Kernel module loader.
+  // "nvidia-settings",      // X server settings.
+  // "nvidia-xconfig",       // X xorg.conf editor.
+  "nvidia-cuda-mps-control", // Multi process service CLI.
+  "nvidia-cuda-mps-server",  // Multi process service server.
+  "nvidia-debugdump",        // GPU coredump utility.
+  "nvidia-persistenced",     // Persistence mode utility.
+  "nvidia-smi",              // System management interface.
+};
+
+
+static constexpr const char* LIBRARIES[] = {
+  // ------- X11 -------
+
+  // "libnvidia-cfg.so",  // GPU configuration.
+  // "libnvidia-gtk2.so", // GTK2.
+  // "libnvidia-gtk3.so", // GTK3.
+  // "libnvidia-wfb.so",  // Wrapped software rendering module for X server.
+  // "libglx.so",         // GLX extension module for X server.
+
+  // ----- Compute -----
+
+  "libnvidia-ml.so",              // Management library.
+  "libcuda.so",                   // CUDA driver library.
+  "libnvidia-ptxjitcompiler.so",  // PTX-SASS JIT compiler.
+  "libnvidia-fatbinaryloader.so", // fatbin loader.
+  "libnvidia-opencl.so",          // NVIDIA OpenCL ICD.
+  "libnvidia-compiler.so",        // NVVM-PTX compiler for OpenCL.
+  // "libOpenCL.so",              // OpenCL ICD loader.
+
+  // ------ Video ------
+
+  "libvdpau_nvidia.so",  // NVIDIA VDPAU ICD.
+  "libnvidia-encode.so", // Video encoder.
+  "libnvcuvid.so",       // Video decoder.
+  "libnvidia-fbc.so",    // Framebuffer capture.
+  "libnvidia-ifr.so",    // OpenGL framebuffer capture.
+
+  // ----- Graphic -----
+
+  // In an ideal world we would only mount nvidia_* vendor specific
+  // libraries and install ICD loaders inside a container. However,
+  // for backwards compatibility we need to mount everything. This
+  // will hopefully change once GLVND is well established.
+
+  "libGL.so",         // OpenGL/GLX legacy _or_ compatibility wrapper (GLVND).
+  "libGLX.so",        // GLX ICD loader (GLVND).
+  "libOpenGL.so",     // OpenGL ICD loader (GLVND).
+  "libGLESv1_CM.so",  // OpenGL ES v1 legacy _or_ ICD loader (GLVND).
+  "libGLESv2.so",     // OpenGL ES v2 legacy _or_ ICD loader (GLVND).
+  "libEGL.so",        // EGL ICD loader.
+  "libGLdispatch.so", // OpenGL dispatch (GLVND).
+
+  "libGLX_nvidia.so",         // OpenGL/GLX ICD (GLVND).
+  "libEGL_nvidia.so",         // EGL ICD (GLVND).
+  "libGLESv2_nvidia.so",      // OpenGL ES v2 ICD (GLVND).
+  "libGLESv1_CM_nvidia.so",   // OpenGL ES v1 ICD (GLVND).
+  "libnvidia-eglcore.so",     // EGL core.
+  "libnvidia-egl-wayland.so", // EGL wayland extensions.
+  "libnvidia-glcore.so",      // OpenGL core.
+  "libnvidia-tls.so",         // Thread local storage.
+  "libnvidia-glsi.so",        // OpenGL system interaction.
+};
+
+
+static Try<bool> isBlacklisted(
+    const string& library,
+    const Owned<elf::File>& elf)
+{
+  // Blacklist EGL/OpenGL libraries issued by other vendors.
+  if (library == "libEGL.so" ||
+      library == "libGLESv1_CM.so" ||
+      library == "libGLESv2.so" ||
+      library == "libGL.so") {
+    Try<vector<string>> dependencies =
+      elf->get_dynamic_strings(elf::DynamicTag::NEEDED);
+
+    if (dependencies.isError()) {
+      return Error("Failed reading external dependencies in ELF file"
+                   " '" + library + "': " + dependencies.error());
+    }
+
+    foreach (const string& dependency, dependencies.get()) {
+      if (dependency == "libGLdispatch.so" ||
+          strings::startsWith(dependency, "libnvidia-gl") ||
+          strings::startsWith(dependency, "libnvidia-egl")) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  // Blacklist TLS libraries using the old ABI (i.e. those != 2.3.99).
+  if (library == "libnvidia-tls.so") {
+    Result<Version> abi = elf->get_abi_version();
+    if (!abi.isSome()) {
+      return Error(
+          "Failed to read ELF ABI version:"
+          " " + (abi.isError() ? abi.error() : "No ABI version found"));
+    }
+
+    if (abi.get() != Version(2, 3, 99)) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+
+const string& NvidiaVolume::HOST_PATH() const
+{
+  return hostPath;
+}
+
+
+const string& NvidiaVolume::CONTAINER_PATH() const
+{
+  return containerPath;
+}
+
+
+Try<NvidiaVolume> NvidiaVolume::create()
+{
+  // Append the Nvidia driver version to the name of the volume.
+  Try<Nothing> initialized = nvml::initialize();
+  if (initialized.isError()) {
+    return Error("Failed to nvml::initialize: " + initialized.error());
+  }
+
+  Try<string> version = nvml::systemGetDriverVersion();
+  if (version.isError()) {
+    return Error("Failed to nvml::systemGetDriverVersion: " + version.error());
+  }
+
+  // Create the volume on the host.
+  string hostPath = HOST_VOLUME_PATH_PREFIX + version.get();
+  if (!os::exists(hostPath)) {
+    Try<Nothing> mkdir = os::mkdir(hostPath);
+    if (mkdir.isError()) {
+      return Error("Failed to os::mkdir '" + hostPath + "': " + mkdir.error());
+    }
+  }
+
+  // Create some directories in the volume if they don't yet exist.
+  string directories[] = {"bin", "lib", "lib64" };
+  foreach (const string& directory, directories) {
+    string path = path::join(hostPath, directory);
+
+    if (!os::exists(path)) {
+      Try<Nothing> mkdir = os::mkdir(path);
+      if (mkdir.isError()) {
+        return Error("Failed to os::mkdir '" + path + "': " + mkdir.error());
+      }
+    }
+  }
+
+  // Fill in the `/bin` directory with BINARIES.
+  foreach (const string& binary, BINARIES) {
+    string path = path::join(hostPath, "bin", binary);
+
+    if (!os::exists(path)) {
+      string command = "which " + binary;
+      Try<string> which = os::shell(command);
+      if (which.isError()) {
+        return Error("Failed to os::shell '" + command + "': " + which.error());
+      }
+
+      which = strings::trim(which.get());
+
+      Result<string> realpath = os::realpath(which.get());
+      if (!realpath.isSome()) {
+        return Error("Failed to os::realpath '" + which.get() + "':"
+                     " " + (realpath.isError()
+                            ? realpath.error()
+                            : "No such file or directory"));
+      }
+
+      command = "cp " + realpath.get() + " " + path;
+      Try<string> cp = os::shell(command);
+      if (cp.isError()) {
+        return Error("Failed to os::shell '" + command + "': " + cp.error());
+      }
+    }
+  }
+
+  // Fill in the `/lib*` directories with LIBRARIES. Process all
+  // versions of a library that match `lib*.so*` in the ldcache.
+  Try<vector<ldcache::Entry>> cache = ldcache::parse();
+  if (cache.isError()) {
+    return Error("Failed to ldcache::parse: " + cache.error());
+  }
+
+  foreach (const string& library, LIBRARIES) {
+    foreach (const ldcache::Entry& entry, cache.get()) {
+      if (strings::startsWith(entry.name, library)) {
+        // Copy the fully resolved `entry.path` (i.e. the path of the
+        // library after following all symlinks) into either the
+        // `/lib` folder if it is 32-bit or `/lib64` if it is 64 bit.
+        Result<string> realpath = os::realpath(entry.path);
+        if (!realpath.isSome()) {
+          return Error("Failed to os::realpath '" + entry.path + "':"
+                       " " + (realpath.isError()
+                              ? realpath.error()
+                              : "No such file or directory"));
+        }
+
+        Try<elf::File*> load = elf::File::load(realpath.get());
+        if (load.isError()) {
+          return Error("Failed to elf::File::load '" + realpath.get() + "':"
+                       " " + load.error());
+        }
+
+        Owned<elf::File> file(load.get());
+
+        // If the library is blacklisted, skip it.
+        Try<bool> blacklisted = isBlacklisted(library, file);
+        if (blacklisted.isError()) {
+          return Error("Failed to check blacklist: " + blacklisted.error());
+        }
+
+        if (blacklisted.get()) {
+          continue;
+        }
+
+        Option<string> libraryDirectory = None();
+
+        Try<elf::Class> c = file->get_class();
+        if (c.isError()) {
+          return Error("Failed to get ELF class for '" + entry.name + "':"
+                       " " + c.error());
+        }
+
+        if (c.get() == elf::CLASS32) {
+          libraryDirectory = "lib";
+        } else if (c.get() == elf::CLASS64) {
+          libraryDirectory = "lib64";
+        } else {
+          return Error("Unknown ELF class: " + stringify(c.get()));
+        }
+
+        CHECK_SOME(libraryDirectory);
+
+        string libraryPath = path::join(
+            hostPath,
+            libraryDirectory.get(),
+            Path(realpath.get()).basename());
+
+        if (!os::exists(libraryPath)) {
+          string command = "cp " + realpath.get() + " " + libraryPath;
+          Try<string> cp = os::shell(command);
+          if (cp.isError()) {
+            return Error("Failed to os::shell '" + command + "':"
+                         " " + cp.error());
+          }
+        }
+
+        // Set up symlinks between `entry.name` and the fully resolved
+        // path we just copied. This preserves the list of libraries
+        // we have on our host system in the mounted volume. If
+        // `entry.path` and the fully resolved path are the same, we
+        // don't make a symlink.
+        string symlinkPath =
+          path::join(hostPath, libraryDirectory.get(), entry.name);
+
+        if (!os::exists(symlinkPath)) {
+          Try<Nothing> symlink =
+            fs::symlink(Path(realpath.get()).basename(), symlinkPath);
+          if (symlink.isError()) {
+            return Error("Failed to fs::symlink"
+                         " '" + symlinkPath + "'"
+                         " -> '" + Path(realpath.get()).basename() + "':"
+                         " " + symlink.error());
+          }
+        }
+
+        // GLVND requires an extra symlink for indirect GLX support.
+        // This is a temproary workaround and won't be needed once we
+        // have an indirect GLX vendor neutral library.
+        //
+        // TODO(klueska): Including this symlink was borrowed
+        // from the `nvidia-docker-plugin` code. Remove this
+        // symlink when `nvidia-docker-plugin` does the same.
+        if (library == "libGLX_nvidia.so") {
+          string libraryName = strings::replace(
+              entry.name, "GLX_nvidia", "GLX_indirect");
+
+          string symlinkPath =
+            path::join(hostPath, libraryDirectory.get(), libraryName);
+
+          if (!os::exists(symlinkPath)) {
+            Try<Nothing> symlink =
+              fs::symlink(Path(realpath.get()).basename(), symlinkPath);
+            if (symlink.isError()) {
+              return Error("Failed to fs::symlink"
+                           " '" + symlinkPath + "'"
+                           " -> '" + Path(realpath.get()).basename() + "':"
+                           " " + symlink.error());
+            }
+          }
+        }
+      }
+    }
+  }
+
+  // Return the actual volume object with the fully
+  // resolved host path and the container path set.
+  return NvidiaVolume(hostPath, CONTAINER_VOLUME_PATH);
+}
+
+
+bool NvidiaVolume::shouldInject(const ImageManifest& manifest) const
+{
+  // TODO(klueska): Parse the `ImageManifest` to decide if we
+  // should inject the volume into the docker container or not.
+  return false;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7185a88/src/slave/containerizer/mesos/isolators/gpu/volume.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/gpu/volume.hpp b/src/slave/containerizer/mesos/isolators/gpu/volume.hpp
new file mode 100644
index 0000000..e71fe95
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/gpu/volume.hpp
@@ -0,0 +1,72 @@
+// 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 __NVIDIA_VOLUME_HPP__
+#define __NVIDIA_VOLUME_HPP__
+
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+#include <mesos/docker/spec.hpp>
+
+#include <stout/try.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Provides a consolidated volume of binaries and libraries
+// from the user-space portion of the Nvidia GPU drivers.
+// We follow the approach taken by Nvidia's docker plugin:
+//   https://github.com/NVIDIA/nvidia-docker/
+class NvidiaVolume
+{
+public:
+  // Returns the Nvidia volume. Once we create the volume it is
+  // never deleted until reboot. This means that if an agent
+  // crashes and tries to recreate the volume it will only ever
+  // *add* new libraries to it (nothing will be deleted from it).
+  // This prevents already-running containers from having
+  // whatever libraries they depend on ripped out from under them.
+  static Try<NvidiaVolume> create();
+
+  const std::string& HOST_PATH() const;
+  const std::string& CONTAINER_PATH() const;
+
+  // Returns whether the container based on a docker image
+  // should have the volume injected. We follow nvidia-docker's
+  // label conventions:
+  //   https://github.com/NVIDIA/nvidia-docker/wiki/Image-inspection
+  bool shouldInject(const ::docker::spec::v1::ImageManifest& manifest) const;
+
+private:
+  NvidiaVolume() = delete;
+
+  NvidiaVolume(const std::string& _hostPath,
+               const std::string& _containerPath)
+    : hostPath(_hostPath),
+      containerPath(_containerPath) {}
+
+  std::string hostPath;
+  std::string containerPath;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __NVIDIA_GPU_VOLUME_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7185a88/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp b/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
index 6770f05..474040c 100644
--- a/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
+++ b/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
@@ -29,6 +29,7 @@
 #include <process/owned.hpp>
 
 #include <stout/jsonify.hpp>
+#include <stout/os/exists.hpp>
 
 #include "master/master.hpp"
 
@@ -49,6 +50,7 @@ using mesos::internal::slave::Gpu;
 using mesos::internal::slave::MesosContainerizer;
 using mesos::internal::slave::MesosContainerizerProcess;
 using mesos::internal::slave::NvidiaGpuAllocator;
+using mesos::internal::slave::NvidiaVolume;
 using mesos::internal::slave::Slave;
 
 using mesos::master::detector::MasterDetector;
@@ -461,6 +463,25 @@ TEST_F(NvidiaGpuTest, NVIDIA_GPU_Allocator)
   AWAIT_FAILED(allocator->allocate({ bogus }));
 }
 
+
+// Tests that we can create the volume that consolidates
+// the Nvidia libraries and binaries.
+TEST_F(NvidiaGpuTest, NVIDIA_GPU_VolumeCreation)
+{
+  Try<NvidiaVolume> volume = NvidiaVolume::create();
+  ASSERT_SOME(volume);
+
+  ASSERT_TRUE(os::exists(volume->HOST_PATH()));
+
+  vector<string> directories = { "bin", "lib", "lib64" };
+  foreach (const string& directory, directories) {
+    EXPECT_TRUE(os::exists(volume->HOST_PATH() + "/" + directory));
+  }
+
+  EXPECT_TRUE(os::exists(volume->HOST_PATH() + "/bin/nvidia-smi"));
+  EXPECT_TRUE(os::exists(volume->HOST_PATH() + "/lib64/libnvidia-ml.so.1"));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[2/2] mesos git commit: Integrated the 'NvidiaVolume' component into 'NvidiaComponents'.

Posted by bm...@apache.org.
Integrated the 'NvidiaVolume' component into 'NvidiaComponents'.

This makes it so that the 'NvidiaVolume' component can be shared
across containerizers in the same way that the 'NvidiaGpuAllocator' is
currently being shared.

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


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

Branch: refs/heads/master
Commit: 14458c81ecdc0f592a6f07db551a237f7173eca7
Parents: a7185a8
Author: Kevin Klues <kl...@gmail.com>
Authored: Sat Jul 2 16:39:35 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Sat Jul 2 16:39:35 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/containerizer.cpp                  | 8 +++++++-
 src/slave/containerizer/mesos/isolators/gpu/components.hpp | 9 +++++++--
 2 files changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/14458c81/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index a436a5d..d66356d 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -231,7 +231,13 @@ Try<Containerizer*> Containerizer::create(
                    allocator.error());
     }
 
-    nvidia = NvidiaComponents(allocator.get());
+    Try<NvidiaVolume> volume = NvidiaVolume::create();
+
+    if (volume.isError()) {
+      return Error("Failed to NvidiaVolume::create: " + volume.error());
+    }
+
+    nvidia = NvidiaComponents(allocator.get(), volume.get());
   }
 #endif
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/14458c81/src/slave/containerizer/mesos/isolators/gpu/components.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/gpu/components.hpp b/src/slave/containerizer/mesos/isolators/gpu/components.hpp
index 39c6126..82fca5d 100644
--- a/src/slave/containerizer/mesos/isolators/gpu/components.hpp
+++ b/src/slave/containerizer/mesos/isolators/gpu/components.hpp
@@ -19,6 +19,7 @@
 
 #ifdef __linux__
 #include "slave/containerizer/mesos/isolators/gpu/allocator.hpp"
+#include "slave/containerizer/mesos/isolators/gpu/volume.hpp"
 #endif
 
 namespace mesos {
@@ -33,10 +34,14 @@ namespace slave {
 struct NvidiaComponents
 {
 #ifdef __linux__
-  NvidiaComponents(const NvidiaGpuAllocator& _allocator)
-    : allocator(_allocator) {}
+  NvidiaComponents(
+    const NvidiaGpuAllocator& _allocator,
+    const NvidiaVolume& _volume)
+    : allocator(_allocator),
+      volume(_volume) {}
 
   NvidiaGpuAllocator allocator;
+  NvidiaVolume volume;
 #endif
 };