You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by tn...@apache.org on 2015/09/25 18:32:56 UTC

[04/17] mesos git commit: Add Docker image provisioner.

Add Docker image provisioner.

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


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

Branch: refs/heads/master
Commit: 1ac371de5688a23dc7084cfad926aa3d5a81b5c5
Parents: 38319bf
Author: Lily Chen <li...@mesosphere.io>
Authored: Mon Jul 6 16:50:12 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   2 +
 src/slave/containerizer/provisioner.cpp         |  81 ++++++
 src/slave/containerizer/provisioners/docker.cpp | 269 +++++++++++++++++++
 src/slave/containerizer/provisioners/docker.hpp | 176 ++++++++++++
 src/slave/flags.cpp                             |  16 ++
 src/slave/flags.hpp                             |   3 +
 src/tests/containerizer/provisioner.hpp         |   8 +-
 7 files changed, 552 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index da72818..916be39 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -692,6 +692,7 @@ if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/filesystem/shared.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/linux_launcher.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/backends/bind.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/store.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
@@ -816,6 +817,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backend.hpp			\
 	slave/containerizer/provisioner/backends/bind.hpp		\
 	slave/containerizer/provisioner/backends/copy.hpp		\
+	slave/containerizer/provisioner/docker.hpp			\
 	slave/containerizer/provisioner/docker/registry_client.hpp	\
 	slave/containerizer/provisioner/docker/store.hpp		\
 	slave/containerizer/provisioner/docker/token_manager.hpp	\

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/slave/containerizer/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner.cpp b/src/slave/containerizer/provisioner.cpp
new file mode 100644
index 0000000..1ff3f10
--- /dev/null
+++ b/src/slave/containerizer/provisioner.cpp
@@ -0,0 +1,81 @@
+/**
+ * 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 <stout/hashset.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+
+#include "slave/containerizer/provisioner.hpp"
+
+#include "slave/containerizer/provisioners/appc/provisioner.hpp"
+#include "slave/containerizer/provisioners/docker/provisioner.hpp"
+
+using namespace process;
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<hashmap<Image::Type, Owned<Provisioner>>> Provisioner::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  if (flags.provisioners.isNone()) {
+    return hashmap<Image::Type, Owned<Provisioner>>();
+  }
+
+  hashmap<Image::Type,
+          Try<Owned<Provisioner>>(*)(const Flags&, Fetcher*)> creators;
+
+  // Register all supported creators.
+  creators.put(Image::APPC, &appc::AppcProvisioner::create);
+  creators.put(Image::DOCKER, &docker::DockerProvisioner::create);
+
+  hashmap<Image::Type, Owned<Provisioner>> provisioners;
+
+  // NOTE: Change in '--provisioners' flag may result in leaked rootfs
+  // files on the disk but it's at least safe because files managed by
+  // different provisioners are totally separated.
+  foreach (const string& type,
+           strings::tokenize(flags.provisioners.get(), ",")) {
+     Image::Type imageType;
+     if (!Image::Type_Parse(strings::upper(type), &imageType)) {
+       return Error("Unknown provisioner '" + type + "'");
+     }
+
+     if (!creators.contains(imageType)) {
+       return Error("Unsupported provisioner '" + type + "'");
+     }
+
+     Try<Owned<Provisioner>> provisioner = creators[imageType](flags, fetcher);
+     if (provisioner.isError()) {
+       return Error("Failed to create '" + stringify(imageType) +
+                    "' provisioner: " + provisioner.error());
+     }
+
+     provisioners[imageType] = provisioner.get();
+  }
+
+  return provisioners;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
new file mode 100644
index 0000000..27bd88f
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker.cpp
@@ -0,0 +1,269 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "slave/containerizer/provisioners/docker.hpp"
+
+#include <glog/logging.h>
+
+#include <stout/json.hpp>
+#include <stout/nothing.hpp>
+#include <stout/os.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/owned.hpp>
+#include <process/sequence.hpp>
+
+#include "linux/fs.hpp"
+
+#include "slave/containerizer/provisioners/docker/store.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+using mesos::slave::ContainerState;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+Try<Owned<Provisioner>> DockerProvisioner::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  Try<Owned<DockerProvisionerProcess>> create =
+    DockerProvisionerProcess::create(flags, fetcher);
+  if (create.isError()) {
+    return Error(create.error());
+  }
+
+  return Owned<Provisioner>(new DockerProvisioner(create.get()));
+}
+
+
+DockerProvisioner::DockerProvisioner(Owned<DockerProvisionerProcess> _process)
+  : process(_process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+DockerProvisioner::~DockerProvisioner()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<Nothing> DockerProvisioner::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  return dispatch(
+      process.get(),
+      &DockerProvisionerProcess::recover,
+      states,
+      orphans);
+}
+
+
+Future<string> DockerProvisioner::provision(
+    const ContainerID& containerId,
+    const Image& image,
+    const std::string& sandbox)
+{
+  return dispatch(
+      process.get(),
+      &DockerProvisionerProcess::provision,
+      containerId,
+      image,
+      sandbox);
+}
+
+
+Future<bool> DockerProvisioner::destroy(const ContainerID& containerId)
+{
+  return dispatch(
+      process.get(),
+      &DockerProvisionerProcess::destroy,
+      containerId);
+}
+
+
+Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  Try<Nothing> mkdir = os::mkdir(flags.docker_rootfs_dir);
+  if (mkdir.isError()) {
+    return Error("Failed to create provisioner rootfs directory '" +
+                 flags.docker_rootfs_dir + "': " + mkdir.error());
+  }
+
+  Try<Owned<Store>> store = Store::create(flags, fetcher);
+  if (store.isError()) {
+    return Error("Failed to create image store: " + store.error());
+  }
+
+  hashmap<string, Owned<mesos::internal::slave::Backend>> backendOptions =
+    mesos::internal::slave::Backend::create(flags);
+
+  return Owned<DockerProvisionerProcess>(
+      new DockerProvisionerProcess(
+          flags,
+          store.get(),
+          backendOptions[flags.docker_backend]));
+}
+
+
+DockerProvisionerProcess::DockerProvisionerProcess(
+    const Flags& _flags,
+    const Owned<Store>& _store,
+    const Owned<mesos::internal::slave::Backend>& _backend)
+  : flags(_flags),
+    store(_store),
+    backend(_backend) {}
+
+
+Future<Nothing> DockerProvisionerProcess::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  return Nothing();
+}
+
+
+Future<string> DockerProvisionerProcess::provision(
+    const ContainerID& containerId,
+    const Image& image,
+    const string& sandbox)
+{
+  if (image.type() != Image::DOCKER) {
+    return Failure("Unsupported container image type");
+  }
+
+  if (!image.has_docker()) {
+    return Failure("Missing Docker image info");
+  }
+
+  return fetch(image.docker().name(), sandbox)
+    .then(defer(self(),
+                &Self::_provision,
+                containerId,
+                lambda::_1));
+}
+
+
+Future<string> DockerProvisionerProcess::_provision(
+    const ContainerID& containerId,
+    const DockerImage& image)
+{
+  // Create root directory.
+  string base = path::join(flags.docker_rootfs_dir,
+                           stringify(containerId));
+
+  string rootfs = path::join(base, "rootfs");
+
+  Try<Nothing> mkdir = os::mkdir(base);
+  if (mkdir.isError()) {
+    return Failure("Failed to create directory for container filesystem: " +
+                    mkdir.error());
+  }
+
+  LOG(INFO) << "Provisioning rootfs for container '" << containerId << "'"
+            << " to '" << base << "'";
+
+  vector<string> layerPaths;
+  foreach (const string& layerId, image.layers) {
+    layerPaths.push_back(path::join(flags.docker_store_dir, layerId, "rootfs"));
+  }
+
+  return backend->provision(layerPaths, base)
+    .then([=]() -> Future<string> {
+      // Bind mount the rootfs to itself so we can pivot_root. We do
+      // it now so any subsequent mounts by the containerizer or
+      // isolators are correctly handled by pivot_root.
+      Try<Nothing> mount =
+        fs::mount(rootfs, rootfs, None(), MS_BIND | MS_SHARED, NULL);
+      if (mount.isError()) {
+        return Failure("Failure to bind mount rootfs: " + mount.error());
+      }
+
+      return rootfs;
+    });
+}
+
+
+// Fetch an image and all dependencies.
+Future<DockerImage> DockerProvisionerProcess::fetch(
+    const string& name,
+    const string& sandbox)
+{
+  return store->get(name)
+    .then([=](const Option<DockerImage>& image) -> Future<DockerImage> {
+      if (image.isSome()) {
+        return image.get();
+      }
+
+      Try<string> uri = path::join(
+          "file:///",
+          flags.docker_discovery_local_dir,
+          name);
+      if (uri.isError()) {
+        return Failure("Unable to join discovery local path: " + uri.error());
+      }
+      return store->put(uri.get(), name, sandbox);
+    });
+}
+
+
+Future<bool> DockerProvisionerProcess::destroy(
+    const ContainerID& containerId)
+{
+  string base = path::join(flags.docker_rootfs_dir, stringify(containerId));
+
+  if (!os::exists(base)) {
+    return false;
+  }
+
+  LOG(INFO) << "Destroying container rootfs for container '"
+            << containerId << "'";
+
+  Try<fs::MountInfoTable> mountTable = fs::MountInfoTable::read();
+  if (mountTable.isError()) {
+    return Failure("Failed to read mount table: " + mountTable.error());
+  }
+
+  foreach (const fs::MountInfoTable::Entry& entry, mountTable.get().entries) {
+    if (strings::startsWith(entry.target, base)) {
+      fs::unmount(entry.target, MNT_DETACH);
+    }
+  }
+
+  return backend->destroy(base);
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
new file mode 100644
index 0000000..736abe6
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -0,0 +1,176 @@
+/**
+ * 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 __MESOS_DOCKER__
+#define __MESOS_DOCKER__
+
+#include <list>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <stout/hashmap.hpp>
+#include <stout/json.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+#include <process/shared.hpp>
+
+#include <mesos/resources.hpp>
+
+#include "slave/containerizer/provisioner.hpp"
+#include "slave/containerizer/provisioners/backend.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Forward declaration.
+class Store;
+
+struct DockerLayer {
+  DockerLayer(
+      const std::string& hash,
+      const JSON::Object& manifest,
+      const std::string& path,
+      const std::string& version,
+      const Option<process::Shared<DockerLayer>> parent)
+    : hash(hash),
+      manifest(manifest),
+      path(path),
+      version(version),
+      parent(parent) {}
+
+  DockerLayer() {}
+
+  std::string hash;
+  JSON::Object manifest;
+  std::string path;
+  std::string version;
+  Option<process::Shared<DockerLayer>> parent;
+};
+
+
+struct DockerImage
+{
+  DockerImage(
+      const std::string& name,
+      const Option<process::Shared<DockerLayer>>& layer)
+    : name(name), layer(layer) {}
+
+  static Try<std::pair<std::string, std::string>> parseTag(
+      const std::string& name)
+  {
+    std::size_t found = name.find_last_of(':');
+    if (found == std::string::npos) {
+      return make_pair(name, "latest");
+    }
+    return make_pair(name.substr(0, found), name.substr(found + 1));
+  }
+
+  DockerImage() {}
+
+  std::string name;
+  Option<process::Shared<DockerLayer>> layer;
+};
+
+// Forward declaration.
+class DockerProvisionerProcess;
+
+class DockerProvisioner : public Provisioner
+{
+public:
+  static Try<process::Owned<Provisioner>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  virtual ~DockerProvisioner();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  virtual process::Future<std::string> provision(
+      const ContainerID& containerId,
+      const Image& image,
+      const std::string& sandbox);
+
+  virtual process::Future<bool> destroy(const ContainerID& containerId);
+
+private:
+  explicit DockerProvisioner(process::Owned<DockerProvisionerProcess> process);
+  DockerProvisioner(const DockerProvisioner&); // Not copyable.
+  DockerProvisioner& operator=(const DockerProvisioner&); // Not assignable.
+
+  process::Owned<DockerProvisionerProcess> process;
+};
+
+
+class DockerProvisionerProcess :
+  public process::Process<DockerProvisionerProcess>
+{
+public:
+  static Try<process::Owned<DockerProvisionerProcess>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  process::Future<std::string> provision(
+      const ContainerID& containerId,
+      const Image& image,
+      const std::string& sandbox);
+
+  process::Future<bool> destroy(const ContainerID& containerId);
+
+private:
+  DockerProvisionerProcess(
+      const Flags& flags,
+      const process::Owned<Store>& store,
+      const process::Owned<mesos::internal::slave::Backend>& backend);
+
+  process::Future<std::string> _provision(
+      const ContainerID& containerId,
+      const DockerImage& image);
+
+  process::Future<DockerImage> fetch(
+      const std::string& name,
+      const std::string& sandbox);
+
+  const Flags flags;
+
+  process::Owned<Store> store;
+  process::Owned<mesos::internal::slave::Backend> backend;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER__

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 5406ef8..3ce787f 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -98,6 +98,22 @@ mesos::internal::slave::Flags::Flags()
       "Directory the docker provisioner will store images in",
       "/tmp/mesos/store/docker");
 
+  add(&Flags::docker_backend,
+      "docker_backend",
+      "Strategy for docker provisioning container rootfs from images",
+      "copy");
+
+  add(&Flags::docker_rootfs_dir,
+      "docker_rootfs_dir",
+      "Directory the docker provisioner will store container root\n"
+      "filesystems in",
+      "/tmp/mesos/containers/docker");
+
+  add(&Flags::docker_discovery_local_dir,
+      "docker_discovery_local_dir",
+      "Directory for docker provisioner to look in for local images",
+      "/tmp/mesos/images/docker");
+
   add(&Flags::default_role,
       "default_role",
       "Any resources in the --resources flag that\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 1bfb447..de0cac4 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -54,6 +54,9 @@ public:
   std::string appc_store_dir;
 
   std::string docker_store_dir;
+  std::string docker_backend;
+  std::string docker_rootfs_dir;
+  std::string docker_discovery_local_dir;
   std::string default_role;
   Option<std::string> attributes;
   Bytes fetcher_cache_size;

http://git-wip-us.apache.org/repos/asf/mesos/blob/1ac371de/src/tests/containerizer/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner.hpp b/src/tests/containerizer/provisioner.hpp
index 54aab5f..c01a441 100644
--- a/src/tests/containerizer/provisioner.hpp
+++ b/src/tests/containerizer/provisioner.hpp
@@ -67,11 +67,12 @@ public:
           const std::list<mesos::slave::ContainerState>& states,
           const hashset<ContainerID>& orphans));
 
-  MOCK_METHOD2(
+  MOCK_METHOD3(
       provision,
       process::Future<std::string>(
           const ContainerID& containerId,
-          const Image& image));
+          const Image& image,
+          const std::string& sandbox));
 
   MOCK_METHOD1(
       destroy,
@@ -87,7 +88,8 @@ public:
 
   process::Future<std::string> unmocked_provision(
       const ContainerID& containerId,
-      const Image& image)
+      const Image& image,
+      const std::string& sandbox)
   {
     if (image.type() != Image::APPC) {
       return process::Failure(