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:53 UTC

[01/17] mesos git commit: Added Docker provisioner paths which handles path manipulation.

Repository: mesos
Updated Branches:
  refs/heads/master 15c0d8029 -> 9fb62cecc


Added Docker provisioner paths which handles path manipulation.

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


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

Branch: refs/heads/master
Commit: 19d787b5c6a634d09067fe0ce83ed29e0346eaf5
Parents: 31b62d6
Author: Lily Chen <li...@mesosphere.io>
Authored: Fri Aug 14 17:12:52 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   4 +-
 .../provisioners/docker/local_store.cpp         |  68 ++++++------
 .../containerizer/provisioners/docker/paths.cpp | 104 +++++++++++++++++++
 .../containerizer/provisioners/docker/paths.hpp |  88 ++++++++++++++++
 .../provisioners/docker/reference_store.cpp     |  22 ++--
 5 files changed, 235 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/19d787b5/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 0b7018c..fd367d3 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -701,6 +701,7 @@ if OS_LINUX
   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/local_store.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/paths.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/reference_store.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
@@ -827,7 +828,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backends/bind.hpp		\
 	slave/containerizer/provisioner/backends/copy.hpp		\
 	slave/containerizer/provisioner/docker.hpp			\
-	slave/containerizer/provisioners/docker/local_store.hpp         \
+	slave/containerizer/provisioner/docker/local_store.hpp		\
+	slave/containerizer/provisioner/docker/paths.hpp		\
 	slave/containerizer/provisioner/docker/reference_store.hpp	\
 	slave/containerizer/provisioner/docker/registry_client.hpp	\
 	slave/containerizer/provisioner/docker/store.hpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/19d787b5/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
index 5f80b43..d46f4bc 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -34,9 +34,11 @@
 #include "common/status_utils.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
-#include "slave/flags.hpp"
 
 #include "slave/containerizer/provisioners/docker/store.hpp"
+#include "slave/containerizer/provisioners/docker/paths.hpp"
+
+#include "slave/flags.hpp"
 
 using namespace process;
 
@@ -175,22 +177,24 @@ Future<DockerImage> LocalStoreProcess::put(
     const string& name,
     const string& sandbox)
 {
-  string tarName = name + ".tar";
-  Try<string> tarPath = path::join(flags.docker_discovery_local_dir, tarName);
-  if (tarPath.isError()) {
-    return Failure(tarPath.error());
+  string tarPath =
+    paths::getLocalImageTarPath(flags.docker_discovery_local_dir, name);
+  if (!os::exists(tarPath)) {
+    return Failure("No Docker image tar archive found");
   }
-  if (!os::exists(tarPath.get())) {
-    return Failure("No Docker image tar archive found: " + name);
+
+  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
+    os::mkdir(paths::getStagingDir(flags.docker_store_dir));
   }
 
   // Create a temporary staging directory.
-  Try<string> staging = os::mkdtemp();
+  Try<string> staging =
+    os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
   if (staging.isError()) {
     return Failure("Failed to create a staging directory");
   }
 
-  return untarImage(tarPath.get(), staging.get())
+  return untarImage(tarPath, staging.get())
     .then(defer(self(), &Self::putImage, name, staging.get(), sandbox));
 }
 
@@ -199,7 +203,7 @@ Future<Nothing> LocalStoreProcess::untarImage(
     const string& tarPath,
     const string& staging)
 {
-  LOG(INFO) << "Untarring image at: " << tarPath;
+  VLOG(1) << "Untarring image at: " << tarPath;
 
   // Untar discovery_local_dir/name.tar into staging/.
   vector<string> argv = {
@@ -244,13 +248,8 @@ Future<DockerImage> LocalStoreProcess::putImage(
     const string& sandbox)
 {
   ImageName imageName(name);
-  // Read repository json.
-  Try<string> repoPath = path::join(staging, "repositories");
-  if (repoPath.isError()) {
-    return Failure("Failed to create path to repository: " + repoPath.error());
-  }
 
-  Try<string> value = os::read(repoPath.get());
+  Try<string> value = os::read(paths::getLocalImageRepositoriesPath(staging));
   if (value.isError()) {
     return Failure("Failed to read repository JSON: " + value.error());
   }
@@ -279,16 +278,10 @@ Future<DockerImage> LocalStoreProcess::putImage(
     return Failure("Tag JSON value expected to be JSON::String");
   }
 
-  Try<string> layerPath = path::join(
-      staging,
-      entry->second.as<JSON::String>().value);
-  if (layerPath.isError()) {
-    return Failure("Failed to create path to image layer: " +
-                    layerPath.error());
-  }
   string layerId = entry->second.as<JSON::String>().value;
 
-  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
+  Try<string> manifest =
+    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
   if (manifest.isError()) {
     return Failure("Failed to read manifest: " + manifest.error());
   }
@@ -320,7 +313,8 @@ Result<string> LocalStoreProcess::getParentId(
     const string& staging,
     const string& layerId)
 {
-  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
+  Try<string> manifest =
+    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
   if (manifest.isError()) {
     return Error("Failed to read manifest: " + manifest.error());
   }
@@ -363,28 +357,29 @@ Future<Nothing> LocalStoreProcess::untarLayer(
     const string& sandbox)
 {
   // Check if image layer is already in store.
-  if (os::exists(path::join(flags.docker_store_dir, id))) {
+  if (os::exists(paths::getImageLayerPath(flags.docker_store_dir, id))) {
     VLOG(1) << "Image layer: " << id << " already in store. Skipping untar"
             << " and putLayer.";
     return Nothing();
   }
 
   // Image layer has been untarred but is not present in the store directory.
-  if (os::exists(path::join(staging, id, "rootfs"))) {
+  string localRootfsPath = paths::getLocalImageLayerRootfsPath(staging, id);
+  if (os::exists(localRootfsPath)) {
     LOG(WARNING) << "Image layer rootfs present at but not in store directory: "
-                << path::join(staging, id) << "Skipping untarLayer.";
+                 << localRootfsPath << "Skipping untarLayer.";
     return moveLayer(staging, id, sandbox);
   }
 
-  os::mkdir(path::join(staging, id, "rootfs"));
+  os::mkdir(localRootfsPath);
   // Untar staging/id/layer.tar into staging/id/rootfs.
   vector<string> argv = {
     "tar",
     "-C",
-    path::join(staging, id, "rootfs"),
+    localRootfsPath,
     "-x",
     "-f",
-    path::join(staging, id, "layer.tar")
+    paths::getLocalImageLayerTarPath(staging, id)
   };
 
   Try<Subprocess> s = subprocess(
@@ -399,11 +394,10 @@ Future<Nothing> LocalStoreProcess::untarLayer(
 
   return s.get().status()
     .then([=](const Option<int>& status) -> Future<Nothing> {
-      Try<string> layerPath = path::join(staging, id, "rootfs");
       if (status.isNone()) {
         return Failure("Failed to reap subprocess to untar image");
       } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-        return Failure("Untar image failed with exit code: " +
+        return Failure("Untar failed with exit code: " +
                         WSTRINGIFY(status.get()));
       }
 
@@ -442,13 +436,13 @@ Future<Nothing> LocalStoreProcess::moveLayer(
     os::mkdir(flags.docker_store_dir);
   }
 
-  if (!os::exists(path::join(flags.docker_store_dir, id))) {
-    os::mkdir(path::join(flags.docker_store_dir, id));
+  if (!os::exists(paths::getImageLayerPath(flags.docker_store_dir, id))) {
+    os::mkdir(paths::getImageLayerPath(flags.docker_store_dir, id));
   }
 
   Try<Nothing> status = os::rename(
-      path::join(staging, id, "rootfs"),
-      path::join(flags.docker_store_dir, id, "rootfs"));
+      paths::getLocalImageLayerRootfsPath(staging, id),
+      paths::getImageLayerRootfsPath(flags.docker_store_dir, id));
 
   if (status.isError()) {
     return Failure("Failed to move layer to store directory:" + status.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/19d787b5/src/slave/containerizer/provisioners/docker/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.cpp b/src/slave/containerizer/provisioners/docker/paths.cpp
new file mode 100644
index 0000000..ca35e0a
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/paths.cpp
@@ -0,0 +1,104 @@
+/**
+ * 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/paths.hpp"
+
+#include <stout/path.hpp>
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+namespace paths {
+
+string getStagingDir(const string& storeDir)
+{
+  return path::join(storeDir, "staging");
+}
+
+string getTempStaging(const string& storeDir)
+{
+  return path::join(getStagingDir(storeDir), "XXXXXX");
+}
+
+string getLocalImageTarPath(
+    const string& discoveryDir,
+    const string& name)
+{
+  return path::join(discoveryDir, name + ".tar");
+}
+
+string getLocalImageRepositoriesPath(const string& staging)
+{
+  return path::join(staging, "repositories");
+}
+
+std::string getLocalImageLayerPath(
+    const string& staging,
+    const string& layerId)
+{
+  return path::join(staging, layerId);
+}
+
+string getLocalImageLayerManifestPath(
+    const string& staging,
+    const string& layerId)
+{
+  return path::join(getLocalImageLayerPath(staging, layerId), "json");
+}
+
+string getLocalImageLayerTarPath(
+  const string& staging,
+  const string& layerId)
+{
+  return path::join(getLocalImageLayerPath(staging, layerId), "layer.tar");
+}
+
+string getLocalImageLayerRootfsPath(
+    const string& staging,
+    const string& layerId)
+{
+  return path::join(getLocalImageLayerPath(staging, layerId), "rootfs");
+}
+
+string getImageLayerPath(
+    const string& storeDir,
+    const string& layerId)
+{
+  return path::join(storeDir, layerId);
+}
+
+string getImageLayerRootfsPath(
+    const string& storeDir,
+    const string& layerId)
+{
+  return path::join(getImageLayerPath(storeDir, layerId), "rootfs");
+}
+
+string getStoredImagesPath(const string& storeDir)
+{
+  return path::join(storeDir, "storedImages");
+}
+
+} // namespace paths {
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/19d787b5/src/slave/containerizer/provisioners/docker/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.hpp b/src/slave/containerizer/provisioners/docker/paths.hpp
new file mode 100644
index 0000000..4aae0d0
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/paths.hpp
@@ -0,0 +1,88 @@
+/**
+ * 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_PATHS__
+#define __MESOS_DOCKER_PATHS__
+
+#include <list>
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+namespace paths {
+
+/**
+ * The Docker store file system layout is as follows:
+ * <root>
+ * |-- Local image discovery dir ('--docker_discovery_local_dir' slave flag)
+ *    |--<name>.tar
+ * |
+ * |-- Image store dir ('--docker_store_dir' slave flag)
+ *    |--staging
+ *    |--<image_id>
+ *        |--rootfs
+ *    |--storedImages
+ */
+
+std::string getStagingDir(const std::string& storeDir);
+
+std::string getTempStaging(const std::string& storeDir);
+
+std::string getLocalImageTarPath(
+    const std::string& discoveryDir,
+    const std::string& name);
+
+std::string getLocalImageRepositoriesPath(const std::string& staging);
+
+std::string getLocalImageLayerPath(
+    const std::string& staging,
+    const std::string& layerId);
+
+std::string getLocalImageLayerManifestPath(
+    const std::string& staging,
+    const std::string& layerId);
+
+std::string getLocalImageLayerTarPath(
+  const std::string& staging,
+  const std::string& layerId);
+
+std::string getLocalImageLayerRootfsPath(
+  const std::string& staging,
+  const std::string& layerId);
+
+std::string getImageLayerPath(
+    const std::string& storeDir,
+    const std::string& layerId);
+
+std::string getImageLayerRootfsPath(
+    const std::string& storeDir,
+    const std::string& layerId);
+
+std::string getStoredImagesPath(const std::string& storeDir);
+
+} // namespace paths {
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_PATHS__

http://git-wip-us.apache.org/repos/asf/mesos/blob/19d787b5/src/slave/containerizer/provisioners/docker/reference_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.cpp b/src/slave/containerizer/provisioners/docker/reference_store.cpp
index b435ed4..e890b3c 100644
--- a/src/slave/containerizer/provisioners/docker/reference_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/reference_store.cpp
@@ -33,6 +33,7 @@
 
 #include "messages/docker_provisioner.hpp"
 
+#include "slave/containerizer/provisioners/docker/paths.hpp"
 #include "slave/containerizer/provisioners/docker/reference_store.hpp"
 #include "slave/state.hpp"
 
@@ -147,14 +148,8 @@ Try<Nothing> ReferenceStoreProcess::persist()
     }
   }
 
-  Try<string> path = path::join(flags.docker_store_dir, "storedImages");
-  if (path.isError()) {
-    return Error("Failure to construct path to repositories lookup: " +
-                    path.error());
-  }
-
-  Try<Nothing> status =
-    mesos::internal::slave::state::checkpoint(path.get(), images);
+  Try<Nothing> status = mesos::internal::slave::state::checkpoint(
+      paths::getStoredImagesPath(flags.docker_store_dir), images);
   if (status.isError()) {
     return Error("Failed to perform checkpoint: " + status.error());
   }
@@ -165,17 +160,17 @@ Try<Nothing> ReferenceStoreProcess::persist()
 
 void ReferenceStoreProcess::initialize()
 {
-  Try<string> path = path::join(flags.docker_store_dir, "storedImages");
+  string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
 
   storedImages.clear();
-  if (!os::exists(path.get())) {
+  if (!os::exists(storedImagesPath)) {
     LOG(INFO) << "No images to load from disk. Docker provisioner image "
-              << "storage path: " << path.get() << " does not exist.";
+              << "storage path: " << storedImagesPath << " does not exist.";
     return;
   }
 
   Result<DockerProvisionerImages> images =
-    ::protobuf::read<DockerProvisionerImages>(path.get());
+    ::protobuf::read<DockerProvisionerImages>(storedImagesPath);
   if (images.isError()) {
     LOG(ERROR) << "Failed to read protobuf for Docker provisioner image: "
                << images.error();
@@ -192,7 +187,8 @@ void ReferenceStoreProcess::initialize()
 
       layers.push_back(layerId);
 
-      if (!os::exists(path::join(flags.docker_store_dir, layerId))) {
+      if (!os::exists(
+              paths::getImageLayerPath(flags.docker_store_dir, layerId))) {
         missingLayers.push_back(layerId);
       }
     }


[10/17] mesos git commit: Restructure Docker provisioner and protos.

Posted by tn...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/metadata_manager.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/metadata_manager.cpp b/src/slave/containerizer/provisioners/docker/metadata_manager.cpp
deleted file mode 100644
index 55eb382..0000000
--- a/src/slave/containerizer/provisioners/docker/metadata_manager.cpp
+++ /dev/null
@@ -1,254 +0,0 @@
-/**
- * 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 <vector>
-
-#include <glog/logging.h>
-
-#include <stout/foreach.hpp>
-#include <stout/hashset.hpp>
-#include <stout/os.hpp>
-#include <stout/protobuf.hpp>
-
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-#include <process/owned.hpp>
-
-#include "common/status_utils.hpp"
-
-#include "messages/docker_provisioner.hpp"
-
-#include "slave/containerizer/provisioners/docker/paths.hpp"
-#include "slave/containerizer/provisioners/docker/metadata_manager.hpp"
-#include "slave/state.hpp"
-
-using namespace process;
-
-using std::list;
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-
-class MetadataManagerProcess : public process::Process<MetadataManagerProcess>
-{
-public:
-  ~MetadataManagerProcess() {}
-
-  static Try<process::Owned<MetadataManagerProcess>> create(
-      const Flags& flags);
-
-  Future<DockerImage> put(
-      const ImageName& name,
-      const std::list<std::string>& layerIds);
-
-  Future<Option<DockerImage>> get(const ImageName& name);
-
-  Future<Nothing> recover();
-
-  // TODO(chenlily): Implement removal of unreferenced images.
-
-private:
-  MetadataManagerProcess(const Flags& flags);
-
-  // Write out metadata manager state to persistent store.
-  Try<Nothing> persist();
-
-  const Flags flags;
-
-  // This is a lookup table for images that are stored in memory. It is keyed
-  // by the name of the DockerImage.
-  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
-  hashmap<std::string, DockerImage> storedImages;
-};
-
-
-Try<Owned<MetadataManager>> MetadataManager::create(const Flags& flags)
-{
-  Try<Owned<MetadataManagerProcess>> process =
-    MetadataManagerProcess::create(flags);
-  if (process.isError()) {
-    return Error("Failed to create Metadata Manager: " + process.error());
-  }
-  return Owned<MetadataManager>(new MetadataManager(process.get()));
-}
-
-
-MetadataManager::MetadataManager(Owned<MetadataManagerProcess> process)
-  : process(process)
-{
-  process::spawn(CHECK_NOTNULL(process.get()));
-}
-
-
-MetadataManager::~MetadataManager()
-{
-  process::terminate(process.get());
-  process::wait(process.get());
-}
-
-
-Future<Nothing> MetadataManager::recover()
-{
-  return process::dispatch(process.get(), &MetadataManagerProcess::recover);
-}
-
-
-Future<DockerImage> MetadataManager::put(
-    const ImageName& name,
-    const list<string>& layerIds)
-{
-  return dispatch(
-      process.get(), &MetadataManagerProcess::put, name, layerIds);
-}
-
-
-Future<Option<DockerImage>> MetadataManager::get(const ImageName& name)
-{
-  return dispatch(process.get(), &MetadataManagerProcess::get, name);
-}
-
-
-MetadataManagerProcess::MetadataManagerProcess(const Flags& flags)
-  : flags(flags) {}
-
-
-Try<Owned<MetadataManagerProcess>> MetadataManagerProcess::create(
-    const Flags& flags)
-{
-  Owned<MetadataManagerProcess> metadataManager =
-    Owned<MetadataManagerProcess>(new MetadataManagerProcess(flags));
-
-  return metadataManager;
-}
-
-
-Future<DockerImage> MetadataManagerProcess::put(
-    const ImageName& name,
-    const list<string>& layerIds)
-{
-  storedImages[name.name()] = DockerImage(name, layerIds);
-
-  Try<Nothing> status = persist();
-  if (status.isError()) {
-    return Failure("Failed to save state of Docker images" + status.error());
-  }
-
-  return storedImages[name.name()];
-}
-
-
-Future<Option<DockerImage>> MetadataManagerProcess::get(const ImageName& name)
-{
-  if (!storedImages.contains(name.name())) {
-    return None();
-  }
-
-  return storedImages[name.name()];
-}
-
-
-Try<Nothing> MetadataManagerProcess::persist()
-{
-  DockerProvisionerImages images;
-
-  foreachpair(
-      const string& name, const DockerImage& dockerImage, storedImages) {
-    DockerProvisionerImages::Image* image = images.add_images();
-
-    image->set_name(name);
-
-    foreach (const string& layerId, dockerImage.layerIds) {
-      image->add_layer_ids(layerId);
-    }
-  }
-
-  Try<Nothing> status = mesos::internal::slave::state::checkpoint(
-      paths::getStoredImagesPath(flags.docker_store_dir), images);
-  if (status.isError()) {
-    return Error("Failed to perform checkpoint: " + status.error());
-  }
-
-  return Nothing();
-}
-
-
-Future<Nothing> MetadataManagerProcess::recover()
-{
-  string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
-
-  storedImages.clear();
-  if (!os::exists(storedImagesPath)) {
-    LOG(INFO) << "No images to load from disk. Docker provisioner image "
-              << "storage path: " << storedImagesPath << " does not exist.";
-    return Nothing();
-  }
-
-  Result<DockerProvisionerImages> images =
-    ::protobuf::read<DockerProvisionerImages>(storedImagesPath);
-  if (images.isError()) {
-    return Failure("Failed to read protobuf for Docker provisioner image: " +
-                   images.error());
-  }
-
-  for (int i = 0; i < images.get().images_size(); i++) {
-    string name = images.get().images(i).name();
-
-    list<string> layerIds;
-    vector<string> missingLayerIds;
-    for (int j = 0; j < images.get().images(i).layer_ids_size(); j++) {
-      string layerId = images.get().images(i).layer_ids(j);
-
-      layerIds.push_back(layerId);
-
-      if (!os::exists(
-              paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId))) {
-        missingLayerIds.push_back(layerId);
-      }
-    }
-
-    if (!missingLayerIds.empty()) {
-      foreach (const string& layerId, missingLayerIds) {
-        LOG(WARNING) << "Image layer: " << layerId << " required for Docker "
-                     << "image: " << name << " is not on disk.";
-      }
-      LOG(WARNING) << "Skipped loading image: " << name
-                   << " due to missing layers.";
-      continue;
-    }
-
-    Try<ImageName> imageName = ImageName::create(name);
-    if (imageName.isError()) {
-      return Failure("Unable to parse Docker image name: " + imageName.error());
-    }
-    storedImages[imageName.get().name()] = DockerImage(imageName.get(), layerIds);
-  }
-
-  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
-
-  return Nothing();
-}
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/metadata_manager.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/metadata_manager.hpp b/src/slave/containerizer/provisioners/docker/metadata_manager.hpp
deleted file mode 100644
index 9db3f47..0000000
--- a/src/slave/containerizer/provisioners/docker/metadata_manager.hpp
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * 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_METADATA_MANAGER_HPP__
-#define __MESOS_DOCKER_METADATA_MANAGER_HPP__
-
-#include <list>
-#include <string>
-
-#include <stout/hashmap.hpp>
-#include <stout/json.hpp>
-#include <stout/option.hpp>
-#include <stout/protobuf.hpp>
-#include <stout/try.hpp>
-
-#include <process/future.hpp>
-#include <process/owned.hpp>
-#include <process/process.hpp>
-
-#include "slave/containerizer/provisioners/docker.hpp"
-
-#include "slave/flags.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-// Forward Declaration.
-class MetadataManagerProcess;
-
-/**
- * The MetadataManager tracks the Docker images cached by the
- * provisioner that are stored on disk. It keeps track of the layers
- * that Docker images are composed of and recovers DockerImage objects
- * upon initialization by checking for dependent layers stored on disk.
- * Currently, image layers are stored indefinitely, with no garbage
- * collection of unreferenced image layers.
- */
-class MetadataManager
-{
-public:
-  static Try<process::Owned<MetadataManager>> create(const Flags& flags);
-
-   ~MetadataManager();
-
-  /**
-   * Create a DockerImage, put it in metadata manager and persist the reference
-   * store state to disk.
-   *
-   * @param name     the name of the Docker image to place in the reference
-   *                 store.
-   * @param layerIds the list of layer ids that comprise the Docker image in
-   *                 order where the root layer's id (no parent layer) is first
-   *                 and the leaf layer's id is last.
-   */
-  process::Future<DockerImage> put(
-      const ImageName& name,
-      const std::list<std::string>& layerIds);
-
-  /**
-   * Retrieve DockerImage based on image name if it is among the DockerImages
-   * stored in memory.
-   *
-   * @param name  the name of the Docker image to retrieve
-   */
-  process::Future<Option<DockerImage>> get(const ImageName& name);
-
-  /**
-   * Recover all stored DockerImage and its layer references.
-   */
-  process::Future<Nothing> recover();
-
-private:
-  explicit MetadataManager(process::Owned<MetadataManagerProcess> process);
-
-  MetadataManager(const MetadataManager&); // Not copyable.
-  MetadataManager& operator=(const MetadataManager&); // Not assignable.
-
-  process::Owned<MetadataManagerProcess> process;
-};
-
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_METADATA_MANAGER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.cpp b/src/slave/containerizer/provisioners/docker/paths.cpp
deleted file mode 100644
index 5a54ba1..0000000
--- a/src/slave/containerizer/provisioners/docker/paths.cpp
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * 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/paths.hpp"
-
-#include <stout/path.hpp>
-
-using std::string;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-namespace paths {
-
-string getStagingDir(const string& storeDir)
-{
-  return path::join(storeDir, "staging");
-}
-
-string getTempStaging(const string& storeDir)
-{
-  return path::join(getStagingDir(storeDir), "XXXXXX");
-}
-
-string getLocalImageTarPath(
-    const string& discoveryDir,
-    const string& name)
-{
-  return path::join(discoveryDir, name + ".tar");
-}
-
-string getLocalImageRepositoriesPath(const string& staging)
-{
-  return path::join(staging, "repositories");
-}
-
-std::string getLocalImageLayerPath(
-    const string& staging,
-    const string& layerId)
-{
-  return path::join(staging, layerId);
-}
-
-string getLocalImageLayerManifestPath(
-    const string& staging,
-    const string& layerId)
-{
-  return path::join(getLocalImageLayerPath(staging, layerId), "json");
-}
-
-string getLocalImageLayerTarPath(
-  const string& staging,
-  const string& layerId)
-{
-  return path::join(getLocalImageLayerPath(staging, layerId), "layer.tar");
-}
-
-string getLocalImageLayerRootfsPath(
-    const string& staging,
-    const string& layerId)
-{
-  return path::join(getLocalImageLayerPath(staging, layerId), "rootfs");
-}
-
-string getImageLayerPath(
-    const string& storeDir,
-    const string& layerId)
-{
-  return path::join(storeDir, "layers", layerId);
-}
-
-string getImageLayerRootfsPath(
-    const string& storeDir,
-    const string& layerId)
-{
-  return path::join(getImageLayerPath(storeDir, layerId), "rootfs");
-}
-
-string getStoredImagesPath(const string& storeDir)
-{
-  return path::join(storeDir, "storedImages");
-}
-
-} // namespace paths {
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.hpp b/src/slave/containerizer/provisioners/docker/paths.hpp
deleted file mode 100644
index 02f129f..0000000
--- a/src/slave/containerizer/provisioners/docker/paths.hpp
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * 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_PATHS_HPP__
-#define __MESOS_DOCKER_PATHS_HPP__
-
-#include <list>
-#include <string>
-
-#include <mesos/mesos.hpp>
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-namespace paths {
-
-/**
- * The Docker store file system layout is as follows:
- * Image store dir ('--docker_store_dir' slave flag)
- *    |--staging (contains temp directories for downloads and extract)
- *    |--layers
- *       |--<layer_id>
- *           |--rootfs
- *    |--rootfses
- *    |--storedImages (file holding on cached images)
- */
-
-std::string getStagingDir(const std::string& storeDir);
-
-std::string getTempStaging(const std::string& storeDir);
-
-std::string getLocalImageTarPath(
-    const std::string& discoveryDir,
-    const std::string& name);
-
-std::string getLocalImageRepositoriesPath(const std::string& staging);
-
-std::string getLocalImageLayerPath(
-    const std::string& staging,
-    const std::string& layerId);
-
-std::string getLocalImageLayerManifestPath(
-    const std::string& staging,
-    const std::string& layerId);
-
-std::string getLocalImageLayerTarPath(
-  const std::string& staging,
-  const std::string& layerId);
-
-std::string getLocalImageLayerRootfsPath(
-  const std::string& staging,
-  const std::string& layerId);
-
-std::string getImageLayerPath(
-    const std::string& storeDir,
-    const std::string& layerId);
-
-std::string getImageLayerRootfsPath(
-    const std::string& storeDir,
-    const std::string& layerId);
-
-std::string getStoredImagesPath(const std::string& storeDir);
-
-} // namespace paths {
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_PATHS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
deleted file mode 100644
index 03958bf..0000000
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * 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_STORE_HPP__
-#define __MESOS_DOCKER_STORE_HPP__
-
-#include <string>
-
-#include <stout/hashmap.hpp>
-#include <stout/nothing.hpp>
-#include <stout/option.hpp>
-#include <stout/try.hpp>
-
-#include <process/future.hpp>
-
-#include "slave/containerizer/fetcher.hpp"
-#include "slave/containerizer/provisioners/docker.hpp"
-
-#include "slave/flags.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-// Store fetches the Docker images and stores them on disk.
-class Store
-{
-public:
-  static Try<process::Owned<Store>> create(
-      const Flags& flags,
-      Fetcher* fetcher);
-
-  virtual ~Store() {}
-
-  /**
-   * Get image by name.
-   *
-   * @param name The name of the Docker image to retrieve from store.
-   *
-   * @return The DockerImage that holds the Docker layers.
-   */
-  virtual process::Future<DockerImage> get(const ImageName& name) = 0;
-
-  /**
-   * Recover all stored images
-   */
-  virtual process::Future<Nothing> recover() = 0;
-
-  // TODO(chenlily): Implement removing an image.
-
-protected:
-  Store() {}
-};
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index b0af3d8..bf26b98 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -98,13 +98,13 @@ 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_store_discovery,
+      "docker_store_discovery",
+      "Strategy for docker store to fetch images",
+      "local");
 
-  add(&Flags::docker_discovery_local_dir,
-      "docker_discovery_local_dir",
+  add(&Flags::docker_store_discovery_local_dir,
+      "docker_store_discovery_local_dir",
       "Directory for docker provisioner to look in for local images",
       "/tmp/mesos/images/docker");
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 758d495..1ce123c 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -53,10 +53,11 @@ public:
   std::string image_provisioner_backend;
   std::string appc_store_dir;
 
-  std::string docker_store;
+  std::string docker_provisioner_backend;
   std::string docker_store_dir;
-  std::string docker_backend;
-  std::string docker_discovery_local_dir;
+  std::string docker_store_discovery;
+  std::string docker_store_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/4dedbf4a/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index dd4f813..3a817c0 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -702,7 +702,7 @@ class DockerProvisionerLocalStoreTest : public TemporaryDirectoryTest
 public:
   void verifyLocalDockerImage(
       const slave::Flags& flags,
-      const DockerImage& dockerImage)
+      const vector<string>& layers)
   {
     string layersPath = path::join(flags.docker_store_dir, "layers");
 
@@ -717,12 +717,10 @@ public:
         os::read(path::join(layersPath, "456", "rootfs", "temp")));
 
     // Verify the Docker Image provided.
-    EXPECT_EQ(dockerImage.imageName.repository, "abc");
-    EXPECT_EQ(dockerImage.imageName.tag, "latest");
-    list<string> expectedLayers;
+    vector<string> expectedLayers;
     expectedLayers.push_back("123");
     expectedLayers.push_back("456");
-    EXPECT_EQ(dockerImage.layerIds, expectedLayers);
+    EXPECT_EQ(expectedLayers, layers);
   }
 
 protected:
@@ -796,25 +794,25 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
   ASSERT_SOME(os::mkdir(image));
 
   slave::Flags flags;
-  flags.docker_store = "local";
+  flags.docker_store_discovery = "local";
   flags.docker_store_dir = path::join(os::getcwd(), "store");
-  flags.docker_discovery_local_dir = imageDir;
+  flags.docker_store_discovery_local_dir = imageDir;
 
-  // Fetcher is not relevant to local store. It is passed through from the
-  // provisioner interface.
-  Fetcher fetcher;
-  Try<Owned<Store>> store = Store::create(flags, &fetcher);
+  Try<Owned<slave::Store>> store =
+    mesos::internal::slave::docker::Store::create(flags);
   ASSERT_SOME(store);
 
   string sandbox = path::join(os::getcwd(), "sandbox");
   ASSERT_SOME(os::mkdir(sandbox));
-  Try<ImageName> imageName = ImageName::create("abc");
-  ASSERT_SOME(imageName);
 
-  Future<DockerImage> dockerImage = store.get()->get(imageName.get());
-  AWAIT_READY(dockerImage);
+  Image mesosImage;
+  mesosImage.set_type(Image::DOCKER);
+  mesosImage.mutable_docker()->set_name("abc");
 
-  verifyLocalDockerImage(flags, dockerImage.get());
+  Future<vector<string>> layers = store.get()->get(mesosImage);
+  AWAIT_READY(layers);
+
+  verifyLocalDockerImage(flags, layers.get());
 }
 
 // This tests the ability of the reference store to recover the images it has
@@ -822,32 +820,33 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
 TEST_F(DockerProvisionerLocalStoreTest, MetadataManagerInitialization)
 {
   slave::Flags flags;
-  flags.docker_store = "local";
+  flags.docker_store_discovery = "local";
   flags.docker_store_dir = path::join(os::getcwd(), "store");
-  flags.docker_discovery_local_dir = path::join(os::getcwd(), "images");
+  flags.docker_store_discovery_local_dir = path::join(os::getcwd(), "images");
 
-  // Fetcher is not relevant to local store. It is passed through from the
-  // provisioner interface.
-  Fetcher fetcher;
-  Try<Owned<Store>> store = Store::create(flags, &fetcher);
+  Try<Owned<slave::Store>> store =
+    mesos::internal::slave::docker::Store::create(flags);
   ASSERT_SOME(store);
 
   string sandbox = path::join(os::getcwd(), "sandbox");
   ASSERT_SOME(os::mkdir(sandbox));
-  Try<ImageName> imageName = ImageName::create("abc");
-  ASSERT_SOME(imageName);
-  Future<DockerImage> dockerImage = store.get()->get(imageName.get());
-  AWAIT_READY(dockerImage);
+
+  Image image;
+  image.set_type(Image::DOCKER);
+  image.mutable_docker()->set_name("abc");
+
+  Future<vector<string>> layers = store.get()->get(image);
+  AWAIT_READY(layers);
 
   // Store is deleted and recreated. Reference Store is initialized upon
   // creation of the store.
   store.get().reset();
-  store = Store::create(flags, &fetcher);
+  store = mesos::internal::slave::docker::Store::create(flags);
   ASSERT_SOME(store);
 
-  dockerImage = store.get()->get(imageName.get());
-  AWAIT_READY(dockerImage);
-  verifyLocalDockerImage(flags, dockerImage.get());
+  layers = store.get()->get(image);
+  AWAIT_READY(layers);
+  verifyLocalDockerImage(flags, layers.get());
 }
 
 } // namespace tests {


[13/17] mesos git commit: Rename Docker reference store to metadata manager.

Posted by tn...@apache.org.
Rename Docker reference store to metadata manager.


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

Branch: refs/heads/master
Commit: 2cc0dfb20643188570bcc61af6e66c83297ade8d
Parents: f392f75
Author: Timothy Chen <tn...@gmail.com>
Authored: Tue Sep 8 22:19:05 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   4 +-
 src/slave/containerizer/provisioners/docker.cpp |  27 +-
 src/slave/containerizer/provisioners/docker.hpp |  76 ++++--
 .../provisioners/docker/local_store.cpp         |  65 ++---
 .../provisioners/docker/local_store.hpp         |  12 +-
 .../provisioners/docker/metadata_manager.cpp    | 254 +++++++++++++++++++
 .../provisioners/docker/metadata_manager.hpp    | 104 ++++++++
 .../provisioners/docker/reference_store.cpp     | 249 ------------------
 .../provisioners/docker/reference_store.hpp     | 102 --------
 .../containerizer/provisioners/docker/store.hpp |   4 +-
 .../containerizer/provisioner_docker_tests.cpp  |  27 +-
 11 files changed, 492 insertions(+), 432 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index cd8b2ca..a2a64dc 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -531,8 +531,8 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/provisioner/backends/copy.cpp		\
 	slave/containerizer/provisioner/docker.cpp			\
 	slave/containerizer/provisioner/docker/local_store.cpp		\
+	slave/containerizer/provisioner/docker/metadata_manager.cpp	\
 	slave/containerizer/provisioner/docker/paths.cpp		\
-	slave/containerizer/provisioner/docker/reference_store.cpp	\
 	slave/containerizer/provisioner/docker/registry_client.cpp	\
 	slave/containerizer/provisioner/docker/token_manager.cpp	\
 	slave/resource_estimators/noop.cpp				\
@@ -831,8 +831,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backends/copy.hpp		\
 	slave/containerizer/provisioner/docker.hpp			\
 	slave/containerizer/provisioner/docker/local_store.hpp		\
+	slave/containerizer/provisioners/docker/metadata_manager.hpp    \
 	slave/containerizer/provisioner/docker/paths.hpp		\
-	slave/containerizer/provisioner/docker/reference_store.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/2cc0dfb2/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
index b1f737f..b5c5a7b 100644
--- a/src/slave/containerizer/provisioners/docker.cpp
+++ b/src/slave/containerizer/provisioners/docker.cpp
@@ -97,9 +97,10 @@ private:
 };
 
 
-ImageName::ImageName(const std::string& name)
+Try<ImageName> ImageName::create(const std::string& name)
 {
-  registry = None();
+  ImageName imageName;
+  Option<string> registry = None();
   std::vector<std::string> components = strings::split(name, "/");
   if (components.size() > 2) {
     registry = name.substr(0, name.find_last_of("/"));
@@ -107,12 +108,14 @@ ImageName::ImageName(const std::string& name)
 
   std::size_t found = components.back().find_last_of(':');
   if (found == std::string::npos) {
-    repo = components.back();
-    tag = "latest";
+    imageName.repository = components.back();
+    imageName.tag = "latest";
   } else {
-    repo = components.back().substr(0, found);
-    tag = components.back().substr(found + 1);
+    imageName.repository = components.back().substr(0, found);
+    imageName.tag = components.back().substr(found + 1);
   }
+
+  return imageName;
 }
 
 Try<Owned<Provisioner>> DockerProvisioner::create(
@@ -191,7 +194,7 @@ Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
     }
   }
 
-  hashmap<string, Owned<Backend>> backends = Backend::create(flags);
+  const hashmap<string, Owned<Backend>> backends = Backend::create(flags);
   if (backends.empty()) {
     return Error("No usable Docker provisioner backend created");
   }
@@ -345,7 +348,13 @@ Future<string> DockerProvisionerProcess::provision(
 
   infos[containerId]->rootfses[flags.docker_backend].put(rootfsId, rootfs);
 
-  return store->get(image.docker().name())
+  Try<ImageName> imageName = ImageName::create(image.docker().name());
+  if (imageName.isError()) {
+    return Failure("Unable to able to parse Docker image name '" +
+                   image.docker().name() + "': " + imageName.error());
+  }
+
+  return store->get(imageName.get())
     .then(defer(self(), &Self::_provision, lambda::_1, containerId, rootfs));
 }
 
@@ -361,7 +370,7 @@ Future<string> DockerProvisionerProcess::_provision(
             << " to '" << rootfs << "'";
 
   vector<string> layerPaths;
-  foreach (const string& layerId, image.layers) {
+  foreach (const string& layerId, image.layerIds) {
     layerPaths.push_back(
         paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId));
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
index d3ada9d..cd7911c 100644
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -52,50 +52,83 @@ namespace docker {
 // Forward declaration.
 class Store;
 
+/**
+ * Represents Docker Image Name, which composes of a repository and a
+ * tag.
+ */
 struct ImageName
 {
-  std::string repo;
-  std::string tag;
-  Option<std::string> registry;
-
-  ImageName(const std::string& name);
+  static Try<ImageName> create(const std::string& name);
 
   ImageName(
-      const std::string& repo,
-      const std::string& tag,
-      const Option<std::string>& registry = None())
-    : repo(repo), tag(tag), registry(registry) {}
+      const std::string& _repository,
+      const std::string& _tag,
+      const Option<std::string>& _registry = None())
+    : repository(_repository),
+      tag(_tag),
+      registry(_registry) {}
 
   ImageName() {}
+
+  /**
+   * The string representation of this image.
+   */
+  std::string name() const
+  {
+    if (registry.isSome()) {
+      return registry.get() +  "/" + repository + ":" + tag;
+    }
+
+    return repository + ":" + tag;
+  }
+
+  /**
+   * Repository of this image (e.g, ubuntu).
+   */
+  std::string repository;
+
+  /**
+   * Tag of this image (e.g: 14.04).
+   */
+  std::string tag;
+
+  /**
+   * Custom registry that the image points to.
+   */
+  Option<std::string> registry;
 };
 
 
 inline std::ostream& operator<<(std::ostream& stream, const ImageName& image)
 {
-  if (image.registry.isSome()) {
-    return stream << image.registry.get()
-                  << "/" << image.repo << ":" << image.tag;
-  }
-  return stream << image.repo << ":" << image.tag;
+  return stream << image.name();
 }
 
 
+/**
+ * Represents a Docker Image that holds its name and all of its layers
+ * sorted by its dependency.
+ */
 struct DockerImage
 {
   DockerImage() {}
 
   DockerImage(
-      const std::string& imageName,
-      const std::list<std::string>& layers)
-  : imageName(imageName), layers(layers) {}
+      const ImageName& _imageName,
+      const std::list<std::string>& _layerIds)
+  : imageName(_imageName), layerIds(_layerIds) {}
 
-  std::string imageName;
-  std::list<std::string> layers;
+  ImageName imageName;
+  std::list<std::string> layerIds;
 };
 
 // Forward declaration.
 class DockerProvisionerProcess;
 
+/**
+ * Docker Provisioner is responsible to provision rootfs for
+ * containers with Docker images.
+ */
 class DockerProvisioner : public mesos::internal::slave::Provisioner
 {
 public:
@@ -117,8 +150,9 @@ public:
 
 private:
   explicit DockerProvisioner(process::Owned<DockerProvisionerProcess> _process);
-  DockerProvisioner(const DockerProvisioner&); // Not copyable.
-  DockerProvisioner& operator=(const DockerProvisioner&); // Not assignable.
+
+  DockerProvisioner& operator=(const DockerProvisioner&) = delete; // Not assignable.
+  DockerProvisioner(const DockerProvisioner&) = delete; // Not copyable.
 
   process::Owned<DockerProvisionerProcess> process;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
index 80b5b06..ec0420e 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -35,8 +35,9 @@
 
 #include "slave/containerizer/fetcher.hpp"
 
-#include "slave/containerizer/provisioners/docker/store.hpp"
+#include "slave/containerizer/provisioners/docker/metadata_manager.hpp"
 #include "slave/containerizer/provisioners/docker/paths.hpp"
+#include "slave/containerizer/provisioners/docker/store.hpp"
 
 #include "slave/flags.hpp"
 
@@ -60,22 +61,22 @@ public:
       const Flags& flags,
       Fetcher* fetcher);
 
-  process::Future<DockerImage> get(const std::string& name);
+  process::Future<DockerImage> get(const ImageName& name);
 
   process::Future<Nothing> recover();
 
 private:
   LocalStoreProcess(
       const Flags& _flags,
-      Owned<ReferenceStore> _refStore)
-    : flags(_flags), refStore(_refStore) {}
+      Owned<MetadataManager> _metadataManager)
+    : flags(_flags), metadataManager(_metadataManager) {}
 
   process::Future<Nothing> untarImage(
       const std::string& tarPath,
       const std::string& staging);
 
   process::Future<DockerImage> putImage(
-      const std::string& name,
+      const ImageName& name,
       const std::string& staging);
 
   Result<std::string> getParentId(
@@ -84,7 +85,7 @@ private:
 
   process::Future<Nothing> putLayers(
       const std::string& staging,
-      const std::list<std::string>& layers);
+      const std::list<std::string>& layerIds);
 
   process::Future<Nothing> putLayer(
       const std::string& staging,
@@ -95,7 +96,7 @@ private:
       const std::string& id);
 
   const Flags flags;
-  process::Owned<ReferenceStore> refStore;
+  process::Owned<MetadataManager> metadataManager;
 };
 
 
@@ -142,7 +143,7 @@ LocalStore::~LocalStore()
 }
 
 
-Future<DockerImage> LocalStore::get(const string& name)
+Future<DockerImage> LocalStore::get(const ImageName& name)
 {
   return dispatch(process.get(), &LocalStoreProcess::get, name);
 }
@@ -174,18 +175,19 @@ Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
     }
   }
 
-  Try<Owned<ReferenceStore>> refStore = ReferenceStore::create(flags);
-  if (refStore.isError()) {
-    return Error(refStore.error());
+  Try<Owned<MetadataManager>> metadataManager = MetadataManager::create(flags);
+  if (metadataManager.isError()) {
+    return Error(metadataManager.error());
   }
 
-  return Owned<LocalStoreProcess>(new LocalStoreProcess(flags, refStore.get()));
+  return Owned<LocalStoreProcess>(
+      new LocalStoreProcess(flags, metadataManager.get()));
 }
 
 
-Future<DockerImage> LocalStoreProcess::get(const string& name)
+Future<DockerImage> LocalStoreProcess::get(const ImageName& name)
 {
-  return refStore->get(name)
+  return metadataManager->get(name)
     .then(defer(self(),
                 [this, name](
                     const Option<DockerImage>& image) -> Future<DockerImage> {
@@ -193,9 +195,12 @@ Future<DockerImage> LocalStoreProcess::get(const string& name)
         return image.get();
       }
 
-      string tarPath =
-        paths::getLocalImageTarPath(flags.docker_discovery_local_dir, name);
+      string tarPath = paths::getLocalImageTarPath(
+          flags.docker_discovery_local_dir,
+          name.name());
+
       if (!os::exists(tarPath)) {
+        VLOG(1) << "Unable to find image in local store with path: " << tarPath;
         return Failure("No Docker image tar archive found");
       }
 
@@ -214,7 +219,7 @@ Future<DockerImage> LocalStoreProcess::get(const string& name)
 
 Future<Nothing> LocalStoreProcess::recover()
 {
-  return refStore->recover();
+  return metadataManager->recover();
 }
 
 Future<Nothing> LocalStoreProcess::untarImage(
@@ -261,11 +266,9 @@ Future<Nothing> LocalStoreProcess::untarImage(
 
 
 Future<DockerImage> LocalStoreProcess::putImage(
-    const std::string& name,
+    const ImageName& name,
     const string& staging)
 {
-  ImageName imageName(name);
-
   Try<string> value = os::read(paths::getLocalImageRepositoriesPath(staging));
   if (value.isError()) {
     return Failure("Failed to read repository JSON: " + value.error());
@@ -277,20 +280,20 @@ Future<DockerImage> LocalStoreProcess::putImage(
   }
 
   Result<JSON::Object> repositoryValue =
-    json.get().find<JSON::Object>(imageName.repo);
+    json.get().find<JSON::Object>(name.repository);
   if (repositoryValue.isError()) {
     return Failure("Failed to find repository: " + repositoryValue.error());
   } else if (repositoryValue.isNone()) {
-    return Failure("Repository '" + imageName.repo + "' is not found");
+    return Failure("Repository '" + name.repository + "' is not found");
   }
 
   JSON::Object repositoryJson = repositoryValue.get();
 
   // We don't use JSON find here because a tag might contain a '.'.
   std::map<string, JSON::Value>::const_iterator entry =
-    repositoryJson.values.find(imageName.tag);
+    repositoryJson.values.find(name.tag);
   if (entry == repositoryJson.values.end()) {
-    return Failure("Tag '" + imageName.tag + "' is not found");
+    return Failure("Tag '" + name.tag + "' is not found");
   } else if (!entry->second.is<JSON::String>()) {
     return Failure("Tag JSON value expected to be JSON::String");
   }
@@ -308,20 +311,20 @@ Future<DockerImage> LocalStoreProcess::putImage(
     return Failure("Failed to parse manifest: " + manifestJson.error());
   }
 
-  list<string> layers;
-  layers.push_back(layerId);
+  list<string> layerIds;
+  layerIds.push_back(layerId);
   Result<string> parentId = getParentId(staging, layerId);
   while(parentId.isSome()) {
-    layers.push_front(parentId.get());
+    layerIds.push_front(parentId.get());
     parentId = getParentId(staging, parentId.get());
   }
   if (parentId.isError()) {
     return Failure("Failed to obtain parent layer id: " + parentId.error());
   }
 
-  return putLayers(staging, layers)
+  return putLayers(staging, layerIds)
     .then([=]() -> Future<DockerImage> {
-      return refStore->put(name, layers);
+      return metadataManager->put(name, layerIds);
     });
 }
 
@@ -353,10 +356,10 @@ Result<string> LocalStoreProcess::getParentId(
 
 Future<Nothing> LocalStoreProcess::putLayers(
     const string& staging,
-    const list<string>& layers)
+    const list<string>& layerIds)
 {
   list<Future<Nothing>> futures{ Nothing() };
-  foreach (const string& layer, layers) {
+  foreach (const string& layer, layerIds) {
     futures.push_back(
         futures.back().then(
           defer(self(), &Self::putLayer, staging, layer)));

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.hpp b/src/slave/containerizer/provisioners/docker/local_store.hpp
index b650b5e..64a6fc0 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.hpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.hpp
@@ -28,7 +28,6 @@ namespace docker {
 
 // Forward declaration.
 class LocalStoreProcess;
-class ReferenceStore;
 
 
 /**
@@ -40,22 +39,21 @@ class ReferenceStore;
 class LocalStore : public Store
 {
 public:
-  virtual ~LocalStore();
-
   static Try<process::Owned<Store>> create(
       const Flags& flags,
       Fetcher* fetcher);
 
-  virtual process::Future<DockerImage> get(const std::string& name);
+  virtual ~LocalStore();
+
+  virtual process::Future<DockerImage> get(const ImageName& name);
 
   virtual process::Future<Nothing> recover();
 
 private:
   explicit LocalStore(process::Owned<LocalStoreProcess> _process);
 
-  LocalStore(const LocalStore&); // Not copyable.
-
-  LocalStore& operator=(const LocalStore&); // Not assignable.
+  LocalStore& operator=(const LocalStore&) = delete; // Not assignable.
+  LocalStore(const LocalStore&) = delete; // Not copyable.
 
   process::Owned<LocalStoreProcess> process;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/metadata_manager.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/metadata_manager.cpp b/src/slave/containerizer/provisioners/docker/metadata_manager.cpp
new file mode 100644
index 0000000..55eb382
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/metadata_manager.cpp
@@ -0,0 +1,254 @@
+/**
+ * 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 <vector>
+
+#include <glog/logging.h>
+
+#include <stout/foreach.hpp>
+#include <stout/hashset.hpp>
+#include <stout/os.hpp>
+#include <stout/protobuf.hpp>
+
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/owned.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "messages/docker_provisioner.hpp"
+
+#include "slave/containerizer/provisioners/docker/paths.hpp"
+#include "slave/containerizer/provisioners/docker/metadata_manager.hpp"
+#include "slave/state.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+
+class MetadataManagerProcess : public process::Process<MetadataManagerProcess>
+{
+public:
+  ~MetadataManagerProcess() {}
+
+  static Try<process::Owned<MetadataManagerProcess>> create(
+      const Flags& flags);
+
+  Future<DockerImage> put(
+      const ImageName& name,
+      const std::list<std::string>& layerIds);
+
+  Future<Option<DockerImage>> get(const ImageName& name);
+
+  Future<Nothing> recover();
+
+  // TODO(chenlily): Implement removal of unreferenced images.
+
+private:
+  MetadataManagerProcess(const Flags& flags);
+
+  // Write out metadata manager state to persistent store.
+  Try<Nothing> persist();
+
+  const Flags flags;
+
+  // This is a lookup table for images that are stored in memory. It is keyed
+  // by the name of the DockerImage.
+  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
+  hashmap<std::string, DockerImage> storedImages;
+};
+
+
+Try<Owned<MetadataManager>> MetadataManager::create(const Flags& flags)
+{
+  Try<Owned<MetadataManagerProcess>> process =
+    MetadataManagerProcess::create(flags);
+  if (process.isError()) {
+    return Error("Failed to create Metadata Manager: " + process.error());
+  }
+  return Owned<MetadataManager>(new MetadataManager(process.get()));
+}
+
+
+MetadataManager::MetadataManager(Owned<MetadataManagerProcess> process)
+  : process(process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+MetadataManager::~MetadataManager()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<Nothing> MetadataManager::recover()
+{
+  return process::dispatch(process.get(), &MetadataManagerProcess::recover);
+}
+
+
+Future<DockerImage> MetadataManager::put(
+    const ImageName& name,
+    const list<string>& layerIds)
+{
+  return dispatch(
+      process.get(), &MetadataManagerProcess::put, name, layerIds);
+}
+
+
+Future<Option<DockerImage>> MetadataManager::get(const ImageName& name)
+{
+  return dispatch(process.get(), &MetadataManagerProcess::get, name);
+}
+
+
+MetadataManagerProcess::MetadataManagerProcess(const Flags& flags)
+  : flags(flags) {}
+
+
+Try<Owned<MetadataManagerProcess>> MetadataManagerProcess::create(
+    const Flags& flags)
+{
+  Owned<MetadataManagerProcess> metadataManager =
+    Owned<MetadataManagerProcess>(new MetadataManagerProcess(flags));
+
+  return metadataManager;
+}
+
+
+Future<DockerImage> MetadataManagerProcess::put(
+    const ImageName& name,
+    const list<string>& layerIds)
+{
+  storedImages[name.name()] = DockerImage(name, layerIds);
+
+  Try<Nothing> status = persist();
+  if (status.isError()) {
+    return Failure("Failed to save state of Docker images" + status.error());
+  }
+
+  return storedImages[name.name()];
+}
+
+
+Future<Option<DockerImage>> MetadataManagerProcess::get(const ImageName& name)
+{
+  if (!storedImages.contains(name.name())) {
+    return None();
+  }
+
+  return storedImages[name.name()];
+}
+
+
+Try<Nothing> MetadataManagerProcess::persist()
+{
+  DockerProvisionerImages images;
+
+  foreachpair(
+      const string& name, const DockerImage& dockerImage, storedImages) {
+    DockerProvisionerImages::Image* image = images.add_images();
+
+    image->set_name(name);
+
+    foreach (const string& layerId, dockerImage.layerIds) {
+      image->add_layer_ids(layerId);
+    }
+  }
+
+  Try<Nothing> status = mesos::internal::slave::state::checkpoint(
+      paths::getStoredImagesPath(flags.docker_store_dir), images);
+  if (status.isError()) {
+    return Error("Failed to perform checkpoint: " + status.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> MetadataManagerProcess::recover()
+{
+  string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
+
+  storedImages.clear();
+  if (!os::exists(storedImagesPath)) {
+    LOG(INFO) << "No images to load from disk. Docker provisioner image "
+              << "storage path: " << storedImagesPath << " does not exist.";
+    return Nothing();
+  }
+
+  Result<DockerProvisionerImages> images =
+    ::protobuf::read<DockerProvisionerImages>(storedImagesPath);
+  if (images.isError()) {
+    return Failure("Failed to read protobuf for Docker provisioner image: " +
+                   images.error());
+  }
+
+  for (int i = 0; i < images.get().images_size(); i++) {
+    string name = images.get().images(i).name();
+
+    list<string> layerIds;
+    vector<string> missingLayerIds;
+    for (int j = 0; j < images.get().images(i).layer_ids_size(); j++) {
+      string layerId = images.get().images(i).layer_ids(j);
+
+      layerIds.push_back(layerId);
+
+      if (!os::exists(
+              paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId))) {
+        missingLayerIds.push_back(layerId);
+      }
+    }
+
+    if (!missingLayerIds.empty()) {
+      foreach (const string& layerId, missingLayerIds) {
+        LOG(WARNING) << "Image layer: " << layerId << " required for Docker "
+                     << "image: " << name << " is not on disk.";
+      }
+      LOG(WARNING) << "Skipped loading image: " << name
+                   << " due to missing layers.";
+      continue;
+    }
+
+    Try<ImageName> imageName = ImageName::create(name);
+    if (imageName.isError()) {
+      return Failure("Unable to parse Docker image name: " + imageName.error());
+    }
+    storedImages[imageName.get().name()] = DockerImage(imageName.get(), layerIds);
+  }
+
+  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
+
+  return Nothing();
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/metadata_manager.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/metadata_manager.hpp b/src/slave/containerizer/provisioners/docker/metadata_manager.hpp
new file mode 100644
index 0000000..9db3f47
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/metadata_manager.hpp
@@ -0,0 +1,104 @@
+/**
+ * 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_METADATA_MANAGER_HPP__
+#define __MESOS_DOCKER_METADATA_MANAGER_HPP__
+
+#include <list>
+#include <string>
+
+#include <stout/hashmap.hpp>
+#include <stout/json.hpp>
+#include <stout/option.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include "slave/containerizer/provisioners/docker.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Forward Declaration.
+class MetadataManagerProcess;
+
+/**
+ * The MetadataManager tracks the Docker images cached by the
+ * provisioner that are stored on disk. It keeps track of the layers
+ * that Docker images are composed of and recovers DockerImage objects
+ * upon initialization by checking for dependent layers stored on disk.
+ * Currently, image layers are stored indefinitely, with no garbage
+ * collection of unreferenced image layers.
+ */
+class MetadataManager
+{
+public:
+  static Try<process::Owned<MetadataManager>> create(const Flags& flags);
+
+   ~MetadataManager();
+
+  /**
+   * Create a DockerImage, put it in metadata manager and persist the reference
+   * store state to disk.
+   *
+   * @param name     the name of the Docker image to place in the reference
+   *                 store.
+   * @param layerIds the list of layer ids that comprise the Docker image in
+   *                 order where the root layer's id (no parent layer) is first
+   *                 and the leaf layer's id is last.
+   */
+  process::Future<DockerImage> put(
+      const ImageName& name,
+      const std::list<std::string>& layerIds);
+
+  /**
+   * Retrieve DockerImage based on image name if it is among the DockerImages
+   * stored in memory.
+   *
+   * @param name  the name of the Docker image to retrieve
+   */
+  process::Future<Option<DockerImage>> get(const ImageName& name);
+
+  /**
+   * Recover all stored DockerImage and its layer references.
+   */
+  process::Future<Nothing> recover();
+
+private:
+  explicit MetadataManager(process::Owned<MetadataManagerProcess> process);
+
+  MetadataManager(const MetadataManager&); // Not copyable.
+  MetadataManager& operator=(const MetadataManager&); // Not assignable.
+
+  process::Owned<MetadataManagerProcess> process;
+};
+
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_METADATA_MANAGER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/reference_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.cpp b/src/slave/containerizer/provisioners/docker/reference_store.cpp
deleted file mode 100644
index 4b72319..0000000
--- a/src/slave/containerizer/provisioners/docker/reference_store.cpp
+++ /dev/null
@@ -1,249 +0,0 @@
-/**
- * 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 <vector>
-
-#include <glog/logging.h>
-
-#include <stout/foreach.hpp>
-#include <stout/hashset.hpp>
-#include <stout/os.hpp>
-#include <stout/protobuf.hpp>
-
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-#include <process/owned.hpp>
-
-#include "common/status_utils.hpp"
-
-#include "messages/docker_provisioner.hpp"
-
-#include "slave/containerizer/provisioners/docker/paths.hpp"
-#include "slave/containerizer/provisioners/docker/reference_store.hpp"
-#include "slave/state.hpp"
-
-using namespace process;
-
-using std::list;
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-
-class ReferenceStoreProcess : public process::Process<ReferenceStoreProcess>
-{
-public:
-  ~ReferenceStoreProcess() {}
-
-  static Try<process::Owned<ReferenceStoreProcess>> create(const Flags& flags);
-
-  Future<DockerImage> put(
-      const std::string& name,
-      const std::list<std::string>& layers);
-
-  Future<Option<DockerImage>> get(const std::string& name);
-
-  Future<Nothing> recover();
-
-  // TODO(chenlily): Implement removal of unreferenced images.
-
-private:
-  ReferenceStoreProcess(const Flags& flags);
-
-  // Write out reference store state to persistent store.
-  Try<Nothing> persist();
-
-  const Flags flags;
-
-  // This is a lookup table for images that are stored in memory. It is keyed
-  // by the name of the DockerImage.
-  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
-  hashmap<std::string, DockerImage> storedImages;
-};
-
-
-Try<Owned<ReferenceStore>> ReferenceStore::create(const Flags& flags)
-{
-  Try<Owned<ReferenceStoreProcess>> process =
-    ReferenceStoreProcess::create(flags);
-  if (process.isError()) {
-    return Error("Failed to create reference store: " + process.error());
-  }
-  return Owned<ReferenceStore>(new ReferenceStore(process.get()));
-}
-
-
-ReferenceStore::ReferenceStore(Owned<ReferenceStoreProcess> process)
-  : process(process)
-{
-  process::spawn(CHECK_NOTNULL(process.get()));
-}
-
-
-ReferenceStore::~ReferenceStore()
-{
-  process::terminate(process.get());
-  process::wait(process.get());
-}
-
-
-Future<Nothing> ReferenceStore::recover()
-{
-  return process::dispatch(process.get(), &ReferenceStoreProcess::recover);
-}
-
-
-Future<DockerImage> ReferenceStore::put(
-    const string& name,
-    const list<string>& layers)
-{
-  return dispatch(
-      process.get(), &ReferenceStoreProcess::put, name, layers);
-}
-
-
-Future<Option<DockerImage>> ReferenceStore::get(const string& name)
-{
-  return dispatch(process.get(), &ReferenceStoreProcess::get, name);
-}
-
-
-ReferenceStoreProcess::ReferenceStoreProcess(const Flags& flags)
-  : flags(flags) {}
-
-
-Try<Owned<ReferenceStoreProcess>> ReferenceStoreProcess::create(
-    const Flags& flags)
-{
-  Owned<ReferenceStoreProcess> referenceStore =
-    Owned<ReferenceStoreProcess>(new ReferenceStoreProcess(flags));
-
-  return referenceStore;
-}
-
-
-Future<DockerImage> ReferenceStoreProcess::put(
-    const string& name,
-    const list<string>& layers)
-{
-  storedImages[name] = DockerImage(name, layers);
-
-  Try<Nothing> status = persist();
-  if (status.isError()) {
-    return Failure("Failed to save state of Docker images" + status.error());
-  }
-
-  return storedImages[name];
-}
-
-
-Future<Option<DockerImage>> ReferenceStoreProcess::get(const string& name)
-{
-  if (!storedImages.contains(name)) {
-    return None();
-  }
-
-  return storedImages[name];
-}
-
-
-Try<Nothing> ReferenceStoreProcess::persist()
-{
-  DockerProvisionerImages images;
-
-  foreachpair(
-      const string& name, const DockerImage& dockerImage, storedImages) {
-    DockerProvisionerImages::Image* image = images.add_images();
-
-    image->set_name(name);
-
-    foreach (const string& layer, dockerImage.layers) {
-      image->add_layer_ids(layer);
-    }
-  }
-
-  Try<Nothing> status = mesos::internal::slave::state::checkpoint(
-      paths::getStoredImagesPath(flags.docker_store_dir), images);
-  if (status.isError()) {
-    return Error("Failed to perform checkpoint: " + status.error());
-  }
-
-  return Nothing();
-}
-
-
-Future<Nothing> ReferenceStoreProcess::recover()
-{
-  string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
-
-  storedImages.clear();
-  if (!os::exists(storedImagesPath)) {
-    LOG(INFO) << "No images to load from disk. Docker provisioner image "
-              << "storage path: " << storedImagesPath << " does not exist.";
-    return Nothing();
-  }
-
-  Result<DockerProvisionerImages> images =
-    ::protobuf::read<DockerProvisionerImages>(storedImagesPath);
-  if (images.isError()) {
-    return Failure("Failed to read protobuf for Docker provisioner image: " +
-                   images.error());
-  }
-
-  for (int i = 0; i < images.get().images_size(); i++) {
-    string imageName = images.get().images(i).name();
-
-    list<string> layers;
-    vector<string> missingLayers;
-    for (int j = 0; j < images.get().images(i).layer_ids_size(); j++) {
-      string layerId = images.get().images(i).layer_ids(j);
-
-      layers.push_back(layerId);
-
-      if (!os::exists(
-              paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId))) {
-        missingLayers.push_back(layerId);
-      }
-    }
-
-    if (!missingLayers.empty()) {
-      foreach (const string& layer, missingLayers) {
-        LOG(WARNING) << "Image layer: " << layer << " required for Docker "
-                     << "image: " << imageName << " is not on disk.";
-      }
-      LOG(WARNING) << "Skipped loading image: " << imageName
-                   << " due to missing layers.";
-      continue;
-    }
-
-    storedImages[imageName] = DockerImage(imageName, layers);
-  }
-
-  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
-
-  return Nothing();
-}
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/reference_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.hpp b/src/slave/containerizer/provisioners/docker/reference_store.hpp
deleted file mode 100644
index be652ae..0000000
--- a/src/slave/containerizer/provisioners/docker/reference_store.hpp
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * 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_REFERENCE_STORE_HPP__
-#define __MESOS_DOCKER_REFERENCE_STORE_HPP__
-
-#include <list>
-#include <string>
-
-#include <stout/hashmap.hpp>
-#include <stout/json.hpp>
-#include <stout/option.hpp>
-#include <stout/protobuf.hpp>
-#include <stout/try.hpp>
-
-#include <process/future.hpp>
-#include <process/owned.hpp>
-#include <process/process.hpp>
-
-#include "slave/containerizer/provisioners/docker.hpp"
-#include "slave/flags.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-// Forward Declaration.
-class ReferenceStoreProcess;
-
-/**
- * The Reference Store is a way to track the Docker images used by the
- * provisioner that are stored in on disk. It keeps track of the layers
- * that Docker images are composed of and recovers DockerImage objects upon
- * initialization by checking for dependent layers stored on disk.
- * Currently, image layers are stored indefinitely, with no garbage collection
- * of unreferenced image layers.
- */
-class ReferenceStore
-{
-public:
-  ~ReferenceStore();
-
-  static Try<process::Owned<ReferenceStore>> create(const Flags& flags);
-
-  /**
-   * Create a DockerImage, put it in reference store and persist the reference
-   * store state to disk.
-   *
-   * @param name   the name of the Docker image to place in the reference store.
-   * @param layers the list of layer ids that comprise the Docker image in
-   *               order where the root layer's id (no parent layer) is first
-   *               and the leaf layer's id is last.
-   */
-  process::Future<DockerImage> put(
-      const std::string& name,
-      const std::list<std::string>& layers);
-
-  /**
-   * Retrieve DockerImage based on image name if it is among the DockerImages
-   * stored in memory.
-   *
-   * @param name  the name of the Docker image to retrieve
-   */
-  process::Future<Option<DockerImage>> get(const std::string& name);
-
-  /**
-   * Recover all stored DockerImage and its layer references.
-   */
-  process::Future<Nothing> recover();
-
-private:
-  explicit ReferenceStore(process::Owned<ReferenceStoreProcess> process);
-
-  ReferenceStore(const ReferenceStore&); // Not copyable.
-  ReferenceStore& operator=(const ReferenceStore&); // Not assignable.
-
-  process::Owned<ReferenceStoreProcess> process;
-};
-
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_REFERENCE_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index a9201d5..03958bf 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -30,7 +30,7 @@
 
 #include "slave/containerizer/fetcher.hpp"
 #include "slave/containerizer/provisioners/docker.hpp"
-#include "slave/containerizer/provisioners/docker/reference_store.hpp"
+
 #include "slave/flags.hpp"
 
 namespace mesos {
@@ -55,7 +55,7 @@ public:
    *
    * @return The DockerImage that holds the Docker layers.
    */
-  virtual process::Future<DockerImage> get(const std::string& name) = 0;
+  virtual process::Future<DockerImage> get(const ImageName& name) = 0;
 
   /**
    * Recover all stored images

http://git-wip-us.apache.org/repos/asf/mesos/blob/2cc0dfb2/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index e1a311b..dd4f813 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -37,12 +37,15 @@
 
 #include <process/ssl/gtest.hpp>
 
+#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
 #include "slave/containerizer/provisioner/docker/registry_client.hpp"
+#include "slave/containerizer/provisioner/docker/store.hpp"
 #include "slave/containerizer/provisioner/docker/token_manager.hpp"
 
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 
+using std::list;
 using std::map;
 using std::string;
 using std::vector;
@@ -714,11 +717,12 @@ public:
         os::read(path::join(layersPath, "456", "rootfs", "temp")));
 
     // Verify the Docker Image provided.
-    EXPECT_EQ(dockerImage.imageName, "abc");
+    EXPECT_EQ(dockerImage.imageName.repository, "abc");
+    EXPECT_EQ(dockerImage.imageName.tag, "latest");
     list<string> expectedLayers;
     expectedLayers.push_back("123");
     expectedLayers.push_back("456");
-    EXPECT_EQ(dockerImage.layers, expectedLayers);
+    EXPECT_EQ(dockerImage.layerIds, expectedLayers);
   }
 
 protected:
@@ -727,7 +731,7 @@ protected:
     TemporaryDirectoryTest::SetUp();
 
     string imageDir = path::join(os::getcwd(), "images");
-    string image = path::join(imageDir, "abc");
+    string image = path::join(imageDir, "abc:latest");
     ASSERT_SOME(os::mkdir(imageDir));
     ASSERT_SOME(os::mkdir(image));
 
@@ -775,7 +779,7 @@ protected:
     ASSERT_SOME(os::rmdir(path::join(image, "456", "layer")));
 
     ASSERT_SOME(os::chdir(image));
-    ASSERT_SOME(os::tar(".", "../abc.tar"));
+    ASSERT_SOME(os::tar(".", "../abc:latest.tar"));
     ASSERT_SOME(os::chdir(cwd));
     ASSERT_SOME(os::rmdir(image));
   }
@@ -787,7 +791,7 @@ protected:
 TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
 {
   string imageDir = path::join(os::getcwd(), "images");
-  string image = path::join(imageDir, "abc");
+  string image = path::join(imageDir, "abc:latest");
   ASSERT_SOME(os::mkdir(imageDir));
   ASSERT_SOME(os::mkdir(image));
 
@@ -804,7 +808,10 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
 
   string sandbox = path::join(os::getcwd(), "sandbox");
   ASSERT_SOME(os::mkdir(sandbox));
-  Future<DockerImage> dockerImage = store.get()->get("abc");
+  Try<ImageName> imageName = ImageName::create("abc");
+  ASSERT_SOME(imageName);
+
+  Future<DockerImage> dockerImage = store.get()->get(imageName.get());
   AWAIT_READY(dockerImage);
 
   verifyLocalDockerImage(flags, dockerImage.get());
@@ -812,7 +819,7 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
 
 // This tests the ability of the reference store to recover the images it has
 // already stored on disk when it is initialized.
-TEST_F(DockerProvisionerLocalStoreTest, ReferenceStoreInitialization)
+TEST_F(DockerProvisionerLocalStoreTest, MetadataManagerInitialization)
 {
   slave::Flags flags;
   flags.docker_store = "local";
@@ -827,7 +834,9 @@ TEST_F(DockerProvisionerLocalStoreTest, ReferenceStoreInitialization)
 
   string sandbox = path::join(os::getcwd(), "sandbox");
   ASSERT_SOME(os::mkdir(sandbox));
-  Future<DockerImage> dockerImage = store.get()->get("abc");
+  Try<ImageName> imageName = ImageName::create("abc");
+  ASSERT_SOME(imageName);
+  Future<DockerImage> dockerImage = store.get()->get(imageName.get());
   AWAIT_READY(dockerImage);
 
   // Store is deleted and recreated. Reference Store is initialized upon
@@ -836,7 +845,7 @@ TEST_F(DockerProvisionerLocalStoreTest, ReferenceStoreInitialization)
   store = Store::create(flags, &fetcher);
   ASSERT_SOME(store);
 
-  dockerImage = store.get()->get("abc");
+  dockerImage = store.get()->get(imageName.get());
   AWAIT_READY(dockerImage);
   verifyLocalDockerImage(flags, dockerImage.get());
 }


[12/17] mesos git commit: Fix Docker provisioners include guard and comments.

Posted by tn...@apache.org.
Fix Docker provisioners include guard and comments.


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

Branch: refs/heads/master
Commit: 11dc5842ca7e1223918b462df15e14e267331043
Parents: 59404cd
Author: Timothy Chen <tn...@gmail.com>
Authored: Mon Sep 7 14:29:28 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |  4 +-
 src/slave/containerizer/provisioners/docker.hpp |  6 +-
 .../provisioners/docker/local_store.cpp         | 98 ++++++++++++++------
 .../provisioners/docker/local_store.hpp         |  6 +-
 .../containerizer/provisioners/docker/paths.hpp |  6 +-
 .../provisioners/docker/reference_store.cpp     | 39 +++++++-
 .../provisioners/docker/reference_store.hpp     | 40 +-------
 .../containerizer/provisioners/docker/store.hpp |  6 +-
 .../containerizer/provisioner_docker_tests.cpp  |  2 +-
 9 files changed, 125 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index fd367d3..65def70 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -249,7 +249,9 @@ STATE_PROTOS = messages/state.pb.cc messages/state.pb.h
 BUILT_SOURCES += $(STATE_PROTOS)
 CLEANFILES += $(STATE_PROTOS)
 
-DOCKER_PROVISIONER_PROTOS = messages/docker_provisioner.pb.cc messages/docker_provisioner.pb.h
+DOCKER_PROVISIONER_PROTOS =                                             \
+  messages/docker_provisioner.pb.cc                                     \
+  messages/docker_provisioner.pb.h
 
 BUILT_SOURCES += $(DOCKER_PROVISIONER_PROTOS)
 CLEANFILES += $(DOCKER_PROVISIONER_PROTOS)

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
index cda83cb..850ce85 100644
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER__
-#define __MESOS_DOCKER__
+#ifndef __MESOS_DOCKER_HPP__
+#define __MESOS_DOCKER_HPP__
 
 #include <list>
 #include <ostream>
@@ -130,4 +130,4 @@ private:
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER__
+#endif // __MESOS_DOCKER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
index 58af655..aec7df9 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -63,7 +63,9 @@ public:
   process::Future<DockerImage> get(const std::string& name);
 
 private:
-  LocalStoreProcess(const Flags& flags);
+  LocalStoreProcess(
+      const Flags& flags,
+      Owned<ReferenceStore> _refStore);
 
   process::Future<Nothing> untarImage(
       const std::string& tarPath,
@@ -81,7 +83,7 @@ private:
       const std::string& staging,
       const std::list<std::string>& layers)
 
-  process::Future<Nothing> untarLayer(
+  process::Future<Nothing> putLayer(
       const std::string& staging,
       const std::string& id)
 
@@ -115,18 +117,42 @@ Try<Owned<Store>> LocalStore::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
+  if (!os::exists(flags.docker_store_dir)) {
+    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store directory: " + mkdir.error());
+    }
+  }
+
+  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
+    Try<Nothing> mkdir =
+      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store staging directory: " +
+                   mkdir.error());
+    }
+  }
+
   Try<Owned<LocalStoreProcess>> process =
     LocalStoreProcess::create(flags, fetcher);
   if (process.isError()) {
-    return Error("Failed to create store: " + process.error());
+    return Error(process.error());
   }
 
-  return Owned<Store>(new LocalStore(process.get()));
+  Try<Owned<ReferenceStore>> refStore = ReferenceStore::create(flags);
+  if (refStore.isError()) {
+    return Error(refStore);
+  }
+
+  return Owned<Store>(new LocalStore(process.get(), refStore.get()));
 }
 
 
-LocalStore::LocalStore(Owned<LocalStoreProcess> process)
-  : process(process)
+LocalStore::LocalStore(
+    Owned<LocalStoreProcess> process,
+    Owned<ReferenceStore> refStore)
+  : process(process),
+    _refStore(refStore)
 {
   process::spawn(CHECK_NOTNULL(process.get()));
 }
@@ -170,10 +196,6 @@ Future<DockerImage> LocalStoreProcess::get(const string& name)
     return Failure("No Docker image tar archive found");
   }
 
-  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
-    os::mkdir(paths::getStagingDir(flags.docker_store_dir));
-  }
-
   // Create a temporary staging directory.
   Try<string> staging =
     os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
@@ -328,7 +350,7 @@ Future<Nothing> LocalStoreProcess::putLayers(
   foreach (const string& layer, layers) {
     futures.push_back(
         futures.back().then(
-          defer(self(), &Self::untarLayer, staging, layer)));
+          defer(self(), &Self::putLayer, staging, layer)));
   }
 
   return collect(futures)
@@ -336,26 +358,50 @@ Future<Nothing> LocalStoreProcess::putLayers(
 }
 
 
-Future<Nothing> LocalStoreProcess::untarLayer(
+Future<Nothing> LocalStoreProcess::putLayer(
     const string& staging,
     const string& id)
 {
-  // Check if image layer is already in store.
-  if (os::exists(paths::getImageLayerPath(flags.docker_store_dir, id))) {
-    VLOG(1) << "Image layer: " << id << " already in store. Skipping untar"
-            << " and putLayer.";
+  // We untar the layer from source into a staging directory, then
+  // move the layer into store. We do this instead of untarring
+  // directly to store to make sure we don't end up with partially
+  // untarred layer rootfs.
+
+  // Check if image layer rootfs is already in store.
+  if (os::exists(paths::getImageLayerRootfsPath(flags.docker_store_dir, id))) {
+    VLOG(1) << "Image layer '" << id << "' rootfs already in store. "
+            << "Skipping put.";
     return Nothing();
   }
 
+  const string imageLayerPath =
+    paths::getImageLayerPath(flags.docker_store_dir, id);
+  if (!os::exists()) {
+    Try<Nothing> mkdir = os::mkdir(imageLayerPath);
+    if (mkdir.isError()) {
+      return Failure("Failed to create Image layer directory '" +
+                     imageLayerPath + "': " + mkdir.error());
+    }
+  }
+
   // Image layer has been untarred but is not present in the store directory.
   string localRootfsPath = paths::getLocalImageLayerRootfsPath(staging, id);
   if (os::exists(localRootfsPath)) {
-    LOG(WARNING) << "Image layer rootfs present at but not in store directory: "
-                 << localRootfsPath << "Skipping untarLayer.";
-    return moveLayer(staging, id);
+    LOG(WARNING) << "Image layer '" << id << "' rootfs present at but not in "
+                 << "store directory '" << localRootfsPath << "'. Removing "
+                 << "staged rootfs and untarring layer again.";
+    Try<Nothing> rmdir = os::rmdir(localRootfsPath);
+    if (rmdir.isError()) {
+      return Failure("Failed to remove incomplete staged rootfs for layer '" +
+                     id + "': " + rmdir.error());
+    }
   }
 
-  os::mkdir(localRootfsPath);
+  Try<Nothing> mkdir = os::mkdir(localRootfsPath);
+  if (mkdir.isError()) {
+    return Failure("Failed to create rootfs path '" + localRootfsPath + "': " +
+                   mkdir.error());
+  }
   // Untar staging/id/layer.tar into staging/id/rootfs.
   vector<string> argv = {
     "tar",
@@ -394,21 +440,13 @@ Future<Nothing> LocalStoreProcess::moveLayer(
     const string& staging,
     const string& id)
 {
-  if (!os::exists(flags.docker_store_dir)) {
-    VLOG(1) << "Creating docker store directory";
-    os::mkdir(flags.docker_store_dir);
-  }
-
-  if (!os::exists(paths::getImageLayerPath(flags.docker_store_dir, id))) {
-    os::mkdir(paths::getImageLayerPath(flags.docker_store_dir, id));
-  }
-
   Try<Nothing> status = os::rename(
       paths::getLocalImageLayerRootfsPath(staging, id),
       paths::getImageLayerRootfsPath(flags.docker_store_dir, id));
 
   if (status.isError()) {
-    return Failure("Failed to move layer to store directory:" + status.error());
+    return Failure("Failed to move layer to store directory: "
+                   + status.error());
   }
 
   return Nothing();

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.hpp b/src/slave/containerizer/provisioners/docker/local_store.hpp
index 41a3559..2f0c9f1 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.hpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_LOCAL_STORE__
-#define __MESOS_DOCKER_LOCAL_STORE__
+#ifndef __MESOS_DOCKER_LOCAL_STORE_HPP__
+#define __MESOS_DOCKER_LOCAL_STORE_HPP__
 
 #include "slave/containerizer/provisioners/docker/store.hpp"
 
@@ -62,4 +62,4 @@ private:
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_LOCAL_STORE__
+#endif // __MESOS_DOCKER_LOCAL_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.hpp b/src/slave/containerizer/provisioners/docker/paths.hpp
index 0ad3b74..02f129f 100644
--- a/src/slave/containerizer/provisioners/docker/paths.hpp
+++ b/src/slave/containerizer/provisioners/docker/paths.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_PATHS__
-#define __MESOS_DOCKER_PATHS__
+#ifndef __MESOS_DOCKER_PATHS_HPP__
+#define __MESOS_DOCKER_PATHS_HPP__
 
 #include <list>
 #include <string>
@@ -83,4 +83,4 @@ std::string getStoredImagesPath(const std::string& storeDir);
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_PATHS__
+#endif // __MESOS_DOCKER_PATHS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker/reference_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.cpp b/src/slave/containerizer/provisioners/docker/reference_store.cpp
index e890b3c..1567248 100644
--- a/src/slave/containerizer/provisioners/docker/reference_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/reference_store.cpp
@@ -48,6 +48,42 @@ namespace internal {
 namespace slave {
 namespace docker {
 
+
+class ReferenceStoreProcess : public process::Process<ReferenceStoreProcess>
+{
+public:
+  ~ReferenceStoreProcess() {}
+
+  // Explicitly use 'initialize' since we are overloading below.
+  using process::ProcessBase::initialize;
+
+  void initialize();
+
+  static Try<process::Owned<ReferenceStoreProcess>> create(const Flags& flags);
+
+  process::Future<DockerImage> put(
+      const std::string& name,
+      const std::list<std::string>& layers);
+
+  process::Future<Option<DockerImage>> get(const std::string& name);
+
+  // TODO(chenlily): Implement removal of unreferenced images.
+
+private:
+  ReferenceStoreProcess(const Flags& flags);
+
+  // Write out reference store state to persistent store.
+  Try<Nothing> persist();
+
+  const Flags flags;
+
+  // This is a lookup table for images that are stored in memory. It is keyed
+  // by the name of the DockerImage.
+  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
+  hashmap<std::string, DockerImage> storedImages;
+};
+
+
 Try<Owned<ReferenceStore>> ReferenceStore::create(const Flags& flags)
 {
   Try<Owned<ReferenceStoreProcess>> process =
@@ -203,9 +239,10 @@ void ReferenceStoreProcess::initialize()
       continue;
     }
 
-    VLOG(1) << "Loaded Docker image: " << imageName << " from disk.";
     storedImages[imageName] = DockerImage(imageName, layers);
   }
+
+  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
 }
 
 } // namespace docker {

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker/reference_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.hpp b/src/slave/containerizer/provisioners/docker/reference_store.hpp
index d9f7070..66b7573 100644
--- a/src/slave/containerizer/provisioners/docker/reference_store.hpp
+++ b/src/slave/containerizer/provisioners/docker/reference_store.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_REFERENCE_STORE__
-#define __MESOS_DOCKER_REFERENCE_STORE__
+#ifndef __MESOS_DOCKER_REFERENCE_STORE_HPP__
+#define __MESOS_DOCKER_REFERENCE_STORE_HPP__
 
 #include <list>
 #include <string>
@@ -95,43 +95,9 @@ private:
 };
 
 
-class ReferenceStoreProcess : public process::Process<ReferenceStoreProcess>
-{
-public:
-  ~ReferenceStoreProcess() {}
-
-  // Explicitly use 'initialize' since we are overloading below.
-  using process::ProcessBase::initialize;
-
-  void initialize();
-
-  static Try<process::Owned<ReferenceStoreProcess>> create(const Flags& flags);
-
-  process::Future<DockerImage> put(
-      const std::string& name,
-      const std::list<std::string>& layers);
-
-  process::Future<Option<DockerImage>> get(const std::string& name);
-
-  // TODO(chenlily): Implement removal of unreferenced images.
-
-private:
-  ReferenceStoreProcess(const Flags& flags);
-
-  // Write out reference store state to persistent store.
-  Try<Nothing> persist();
-
-  const Flags flags;
-
-  // This is a lookup table for images that are stored in memory. It is keyed
-  // by the name of the DockerImage.
-  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
-  hashmap<std::string, DockerImage> storedImages;
-};
-
 } // namespace docker {
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_REFERENCE_STORE__
+#endif // __MESOS_DOCKER_REFERENCE_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index 0520a2c..b9cb770 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_STORE__
-#define __MESOS_DOCKER_STORE__
+#ifndef __MESOS_DOCKER_STORE_HPP__
+#define __MESOS_DOCKER_STORE_HPP__
 
 #include <string>
 
@@ -68,4 +68,4 @@ protected:
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_STORE__
+#endif // __MESOS_DOCKER_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/11dc5842/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index 3927009..3a9a6ec 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -713,7 +713,7 @@ public:
         "bar 456",
         os::read(path::join(flags.docker_store_dir, "456", "rootfs", "temp")));
 
-    // Verify the docker Image provided.
+    // Verify the Docker Image provided.
     EXPECT_EQ(dockerImage.imageName, "abc");
     list<string> expectedLayers;
     expectedLayers.push_back("123");


[03/17] mesos git commit: Refactored DockerImage struct to store a list of layer ids instead of linked list of DockerLayers.

Posted by tn...@apache.org.
Refactored DockerImage struct to store a list of layer ids instead of linked list of DockerLayers.

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


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

Branch: refs/heads/master
Commit: 86d87aa76b96d35ba7fa3477ea97e736e314943f
Parents: 1ac371d
Author: Lily Chen <li...@mesosphere.io>
Authored: Thu Jul 23 16:04:02 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/provisioners/docker.cpp |  36 +-
 src/slave/containerizer/provisioners/docker.hpp |  69 ++--
 .../containerizer/provisioners/docker/store.cpp | 403 +++++++++----------
 .../containerizer/provisioners/docker/store.hpp |  92 +++--
 src/slave/flags.hpp                             |   1 +
 5 files changed, 316 insertions(+), 285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/86d87aa7/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
index 27bd88f..888f17a 100644
--- a/src/slave/containerizer/provisioners/docker.cpp
+++ b/src/slave/containerizer/provisioners/docker.cpp
@@ -47,6 +47,24 @@ namespace internal {
 namespace slave {
 namespace docker {
 
+ImageName::ImageName(const std::string& name)
+{
+  registry = None();
+  std::vector<std::string> components = strings::split(name, "/");
+  if (components.size() > 2) {
+    registry = name.substr(0, name.find_last_of("/"));
+  }
+
+  std::size_t found = components.back().find_last_of(':');
+  if (found == std::string::npos) {
+    repo = components.back();
+    tag = "latest";
+  } else {
+    repo = components.back().substr(0, found);
+    tag = components.back().substr(found + 1);
+  }
+}
+
 Try<Owned<Provisioner>> DockerProvisioner::create(
     const Flags& flags,
     Fetcher* fetcher)
@@ -199,7 +217,7 @@ Future<string> DockerProvisionerProcess::_provision(
   }
 
   return backend->provision(layerPaths, base)
-    .then([=]() -> Future<string> {
+    .then([rootfs]() -> 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.
@@ -224,15 +242,7 @@ Future<DockerImage> DockerProvisionerProcess::fetch(
       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);
+      return store->put(name, sandbox);
     });
 }
 
@@ -256,7 +266,11 @@ Future<bool> DockerProvisionerProcess::destroy(
 
   foreach (const fs::MountInfoTable::Entry& entry, mountTable.get().entries) {
     if (strings::startsWith(entry.target, base)) {
-      fs::unmount(entry.target, MNT_DETACH);
+      Try<Nothing> unmount = fs::unmount(entry.target, MNT_DETACH);
+      if (unmount.isError()) {
+        return Failure("Failed to unmount mount table target: " +
+                        unmount.error());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/86d87aa7/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
index 736abe6..9cca662 100644
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -20,6 +20,7 @@
 #define __MESOS_DOCKER__
 
 #include <list>
+#include <ostream>
 #include <sstream>
 #include <string>
 #include <utility>
@@ -29,6 +30,7 @@
 #include <stout/json.hpp>
 #include <stout/nothing.hpp>
 #include <stout/option.hpp>
+#include <stout/strings.hpp>
 #include <stout/try.hpp>
 
 #include <process/future.hpp>
@@ -51,50 +53,45 @@ 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 ImageName
+{
+  std::string repo;
+  std::string tag;
+  Option<std::string> registry;
+
+  ImageName(const std::string& name);
+
+  ImageName(
+      const std::string& repo,
+      const std::string& tag,
+      const Option<std::string>& registry = None())
+    : repo(repo), tag(tag), registry(registry) {}
+
+  ImageName() {}
 };
 
 
-struct DockerImage
+inline std::ostream& operator<<(std::ostream& stream, const ImageName& image)
 {
-  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));
+  if (image.registry.isSome()) {
+    return stream << image.registry.get()
+                  << "/" << image.repo << ":" << image.tag;
   }
+  return stream << image.repo << ":" << image.tag;
+}
+
 
+struct DockerImage
+{
   DockerImage() {}
 
-  std::string name;
-  Option<process::Shared<DockerLayer>> layer;
+  DockerImage(
+      const std::string& imageName,
+      const std::list<std::string>& layers)
+  : imageName(imageName), layers(layers) {}
+
+  std::string imageName;
+  std::list<std::string> layers;
 };
 
 // Forward declaration.

http://git-wip-us.apache.org/repos/asf/mesos/blob/86d87aa7/src/slave/containerizer/provisioners/docker/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.cpp b/src/slave/containerizer/provisioners/docker/store.cpp
index e33b570..9453d6f 100644
--- a/src/slave/containerizer/provisioners/docker/store.cpp
+++ b/src/slave/containerizer/provisioners/docker/store.cpp
@@ -19,13 +19,13 @@
 #include "slave/containerizer/provisioners/docker/store.hpp"
 
 #include <list>
-#include <utility>
 
 #include <glog/logging.h>
 
 #include <stout/os.hpp>
 #include <stout/json.hpp>
 
+#include <process/collect.hpp>
 #include <process/defer.hpp>
 #include <process/dispatch.hpp>
 
@@ -39,7 +39,6 @@ using namespace process;
 using std::list;
 using std::string;
 using std::vector;
-using std::pair;
 
 namespace mesos {
 namespace internal {
@@ -91,11 +90,10 @@ LocalStore::~LocalStore()
 
 
 Future<DockerImage> LocalStore::put(
-    const string& uri,
     const string& name,
-    const string& directory)
+    const string& sandbox)
 {
-  return dispatch(process.get(), &LocalStoreProcess::put, uri, name, directory);
+  return dispatch(process.get(), &LocalStoreProcess::put, name, sandbox);
 }
 
 
@@ -110,9 +108,9 @@ Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
     Fetcher* fetcher)
 {
   Owned<LocalStoreProcess> store =
-    Owned<LocalStoreProcess>(new LocalStoreProcess(flags, fetcher));
+    Owned<LocalStoreProcess>(new LocalStoreProcess(flags));
 
-  Try<Nothing> restore = store->restore();
+  Try<Nothing> restore = store->restore(flags);
   if (restore.isError()) {
     return Error("Failed to restore store: " + restore.error());
   }
@@ -121,33 +119,85 @@ Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
 }
 
 
-LocalStoreProcess::LocalStoreProcess(
-    const Flags& flags,
-    Fetcher* fetcher)
-  : flags(flags),
-    fetcher(fetcher) {}
+LocalStoreProcess::LocalStoreProcess(const Flags& flags)
+  : flags(flags) {}
+
 
-// Currently only local file:// uri supported.
-// TODO(chenlily): Add support for fetching image from external uri.
 Future<DockerImage> LocalStoreProcess::put(
-    const string& uri,
     const string& name,
-    const string& directory)
+    const string& sandbox)
 {
-  string imageUri = uri;
-  if (strings::startsWith(imageUri, "file://")) {
-    imageUri = imageUri.substr(7);
+  string tarName = name + ".tar";
+  Try<string> tarPath = path::join(flags.docker_discovery_local_dir, tarName);
+  if (tarPath.isError()) {
+    return Failure(tarPath.error());
+  }
+  if (!os::exists(tarPath.get())) {
+    return Failure("No Docker image tar archive found: " + name);
+  }
+
+  // Create a temporary staging directory.
+  Try<string> staging = os::mkdtemp();
+  if (staging.isError()) {
+    return Failure("Failed to create a staging directory");
   }
 
-  Try<bool> isDir = os::stat::isdir(imageUri);
-  if (isDir.isError()) {
-    return Failure("Failed to check directory for uri '" +imageUri + "':"
-                   + isDir.error());
-  } else if (!isDir.get()) {
-    return Failure("Docker image uri '" + imageUri + "' is not a directory");
+  return untarImage(tarPath.get(), staging.get())
+    .then(defer(self(), &Self::putImage, name, staging.get(), sandbox));
+}
+
+
+Future<Nothing> LocalStoreProcess::untarImage(
+    const string& tarPath,
+    const string& staging)
+{
+  LOG(INFO) << "Untarring image at: " << tarPath;
+
+  // Untar discovery_local_dir/name.tar into staging/.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    staging,
+    "-x",
+    "-f",
+    tarPath
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
   }
 
-  Try<string> repoPath = path::join(imageUri, "repositories");
+  return s.get().status()
+    .then([=](const Option<int>& status) -> Future<Nothing> {
+      if (status.isNone()) {
+        return Failure("Failed to reap status for tar subprocess in " +
+                        tarPath);
+      }
+      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+          return Failure("Untar image failed with exit code: " +
+                          WSTRINGIFY(status.get()));
+      }
+
+      return Nothing();
+    });
+}
+
+
+Future<DockerImage> LocalStoreProcess::putImage(
+    const std::string& name,
+    const string& staging,
+    const string& sandbox)
+{
+  ImageName imageName(name);
+  // Read repository json.
+  Try<string> repoPath = path::join(staging, "repositories");
   if (repoPath.isError()) {
     return Failure("Failed to create path to repository: " + repoPath.error());
   }
@@ -162,95 +212,135 @@ Future<DockerImage> LocalStoreProcess::put(
     return Failure("Failed to parse JSON: " + json.error());
   }
 
-  Try<pair<string, string>> repoTag = DockerImage::parseTag(name);
-  if (repoTag.isError()) {
-    return Failure("Failed to parse Docker image name: " + repoTag.error());
-  }
-
-  string repository = repoTag.get().first;
-  string tag = repoTag.get().second;
-
   Result<JSON::Object> repositoryValue =
-    json.get().find<JSON::Object>(repository);
+    json.get().find<JSON::Object>(imageName.repo);
   if (repositoryValue.isError()) {
     return Failure("Failed to find repository: " + repositoryValue.error());
   } else if (repositoryValue.isNone()) {
-    return Failure("Repository '" + repository + "' is not found");
+    return Failure("Repository '" + imageName.repo + "' is not found");
   }
 
   JSON::Object repositoryJson = repositoryValue.get();
 
   // We don't use JSON find here because a tag might contain a '.'.
   std::map<string, JSON::Value>::const_iterator entry =
-    repositoryJson.values.find(tag);
+    repositoryJson.values.find(imageName.tag);
   if (entry == repositoryJson.values.end()) {
-    return Failure("Tag '" + tag + "' is not found");
+    return Failure("Tag '" + imageName.tag + "' is not found");
   } else if (!entry->second.is<JSON::String>()) {
     return Failure("Tag JSON value expected to be JSON::String");
   }
 
-  Try<string> layerUri = path::join(
-      imageUri,
+  Try<string> layerPath = path::join(
+      staging,
       entry->second.as<JSON::String>().value);
-  if (layerUri.isError()) {
-    return Failure("Failed to create path to image layer: " + layerUri.error());
+  if (layerPath.isError()) {
+    return Failure("Failed to create path to image layer: " +
+                    layerPath.error());
+  }
+  string layerId = entry->second.as<JSON::String>().value;
+
+  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
+  if (manifest.isError()) {
+    return Failure("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
+  if (manifestJson.isError()) {
+    return Failure("Failed to parse manifest: " + manifestJson.error());
   }
 
-  return putLayer(layerUri.get(), directory)
-    .then([=](const Shared<DockerLayer>& layer) -> Future<DockerImage> {
-      DockerImage image(name, layer);
-      images[name] = image;
-      return image;
+  list<string> layers;
+  layers.push_back(layerId);
+  Result<string> parentId = getParentId(staging, layerId);
+  while(parentId.isSome()) {
+    layers.push_front(parentId.get());
+    parentId = getParentId(staging, parentId.get());
+  }
+  if (parentId.isError()) {
+    return Failure("Failed to obtain parent layer id: " + parentId.error());
+  }
+
+  return putLayers(staging, layers, sandbox)
+    .then([=]() -> Future<DockerImage> {
+      images[name] = DockerImage(name, layers);
+
+      // TODO(chenlily): update reference store or replace with reference store
+      return images[name];
     });
 }
 
 
-Future<Shared<DockerLayer>> LocalStoreProcess::putLayer(
-    const string& uri,
-    const string& directory)
+Result<string> LocalStoreProcess::getParentId(
+    const string& staging,
+    const string& layerId)
 {
-  Try<string> hash = os::basename(uri);
-  if (hash.isError()) {
-    return Failure("Failed to determine hash for stored layer: " +
-                    hash.error());
+  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
+  if (manifest.isError()) {
+    return Error("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
+  if (json.isError()) {
+    return Error("Failed to parse manifest: " + json.error());
   }
 
-  if (layers.contains(hash.get())) {
-    return layers[hash.get()];
+  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
+  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
+    return None();
+  } else if (parentId.isError()) {
+    return Error("Failed to read parent of layer: " + parentId.error());
   }
+  return parentId.get().value;
+}
 
-  return untarLayer(uri)
-    .then([=]() {
-      return entry(uri, directory);
-    })
-    .then([=](const Shared<DockerLayer>& layer) {
-      VLOG(1) << "Stored layer with hash: " << hash.get();
-      layers[hash.get()] = layer;
 
-      return layer;
-    });
+Future<Nothing> LocalStoreProcess::putLayers(
+    const string& staging,
+    const list<string>& layers,
+    const string& sandbox)
+{
+  list<Future<Nothing>> futures{ Nothing() };
+  foreach (const string& layer, layers) {
+    futures.push_back(
+        futures.back().then(
+          defer(self(), &Self::untarLayer, staging, layer, sandbox)));
+  }
+
+  return collect(futures)
+    .then([]() -> Future<Nothing> { return Nothing(); });
 }
 
 
 Future<Nothing> LocalStoreProcess::untarLayer(
-    const string& uri)
+    const string& staging,
+    const string& id,
+    const string& sandbox)
 {
-  string rootFs = path::join(uri, "rootfs");
-
-  if (os::exists(rootFs)) {
+  // Check if image layer is already in store.
+  if (os::exists(path::join(flags.docker_store_dir, id))) {
+    VLOG(1) << "Image layer: " << id << " already in store. Skipping untar"
+            << " and putLayer.";
     return Nothing();
-  } else {
-    os::mkdir(rootFs);
   }
 
-  // Untar imageUri/hash/layer.tar into imageUri/hash/rootfs.
+  // Image layer has been untarred but is not present in the store directory.
+  if (os::exists(path::join(staging, id, "rootfs"))) {
+    LOG(WARNING) << "Image layer rootfs present at but not in store directory: "
+                << path::join(staging, id) << "Skipping untarLayer.";
+    return moveLayer(staging, id, sandbox);
+  }
+
+  os::mkdir(path::join(staging, id, "rootfs"));
+  // Untar staging/id/layer.tar into staging/id/rootfs.
   vector<string> argv = {
     "tar",
     "-C",
-    rootFs,
+    path::join(staging, id, "rootfs"),
     "-x",
     "-f",
-    path::join(uri, "layer.tar")};
+    path::join(staging, id, "layer.tar")
+  };
 
   Try<Subprocess> s = subprocess(
       "tar",
@@ -264,157 +354,62 @@ Future<Nothing> LocalStoreProcess::untarLayer(
 
   return s.get().status()
     .then([=](const Option<int>& status) -> Future<Nothing> {
-        if (status.isNone()) {
-          return Failure("Failed to reap status for tar subprocess in " +
-                          uri);
-        }
-
-        if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-          return Failure("Untar failed with exit code: " +
-                          WSTRINGIFY(status.get()));
-        }
-
-        return Nothing();
-      });
-}
-
-
-Future<Shared<DockerLayer>> LocalStoreProcess::storeLayer(
-    const string& hash,
-    const string& uri,
-    const string& directory)
-{
-  string store = uri;
-
-  // Only copy if the store directory doesn't exist.
-  Future<Option<int>> status;
-  if (os::exists(store)) {
-    LOG(INFO) << "Layer store '" << store << "' exists, skipping rename";
-    status = 0;
-  } else {
-    Try<int> out = os::open(
-        path::join(directory, "stdout"),
-        O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
-        S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
-    if (out.isError()) {
-      return Failure("Failed to create 'stdout' file: " + out.error());
-    }
-
-    // Repeat for stderr.
-    Try<int> err = os::open(
-        path::join(directory, "stderr"),
-        O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
-        S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
-    if (err.isError()) {
-      os::close(out.get());
-      return Failure("Failed to create 'stderr' file: " + err.error());
-    }
-
-    vector<string> argv{
-      "cp",
-      "--archive",
-      path::join(uri, "rootfs"),
-      store
-    };
-
-    VLOG(1) << "Copying image with command: " << strings::join(" ", argv);
-
-    Try<Subprocess> s = subprocess(
-      "cp",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::FD(out.get()),
-      Subprocess::FD(err.get()));
-    if (s.isError()) {
-      return Failure("Failed to create 'cp' subprocess: " + s.error());
-    }
-
-    status = s.get().status();
-  }
-
-  return status
-    .then([=](const Option<int>& status) -> Future<Shared<DockerLayer>> {
+      Try<string> layerPath = path::join(staging, id, "rootfs");
       if (status.isNone()) {
-        return Failure("Failed to reap subprocess to copy image");
+        return Failure("Failed to reap subprocess to untar image");
       } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-        return Failure("Copy image failed with exit code: " +
+        return Failure("Untar image failed with exit code: " +
                         WSTRINGIFY(status.get()));
       }
 
-      return entry(uri, directory);
-    })
-    .then([=](const Shared<DockerLayer>& layer) {
-      LOG(INFO) << "Stored layer with hash: " << hash;
-      layers[hash] = layer;
-
-      return layer;
+      return moveLayer(staging, id, sandbox);
     });
 }
 
 
-Future<Shared<DockerLayer>> LocalStoreProcess::entry(
-    const string& uri,
-    const string& directory)
-{
-  Result<string> realpath = os::realpath(uri);
-  if (realpath.isError()) {
-    return Failure("Error in checking store path: " + realpath.error());
-  } else if (realpath.isNone()) {
-    return Failure("Store path not found");
-  }
+Future<Nothing> LocalStoreProcess::moveLayer(
+    const string& staging,
+    const string& id,
+    const string& sandbox){
 
-  Try<string> hash = os::basename(realpath.get());
-  if (hash.isError()) {
-    return Failure(
-      "Failed to determine hash for stored image: " + hash.error());
-  }
+  Try<int> out = os::open(
+      path::join(sandbox, "stdout"),
+      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
+      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
 
-  Try<string> version = os::read(path::join(uri, "VERSION"));
-  if (version.isError()) {
-    return Failure("Failed to determine version of JSON: " + version.error());
+  if (out.isError()) {
+    return Failure("Failed to create 'stdout' file: " + out.error());
   }
 
-  Try<string> manifest = os::read(path::join(uri, "json"));
-  if (manifest.isError()) {
-    return Failure("Failed to read manifest: " + manifest.error());
-  }
+  // Repeat for stderr.
+  Try<int> err = os::open(
+      path::join(sandbox, "stderr"),
+      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
+      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
 
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
-  if (json.isError()) {
-    return Failure("Failed to parse manifest: " + json.error());
+  if (err.isError()) {
+    os::close(out.get());
+    return Failure("Failed to create 'stderr' file: " + err.error());
   }
 
-  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
-  if (parentId.isNone()) {
-    return Shared<DockerLayer>(new DockerLayer(
-        hash.get(),
-        json.get(),
-        realpath.get(),
-        version.get(),
-        None()));
-  } else if (parentId.isError()) {
-    return Failure("Failed to read parent of layer: " + parentId.error());
+  if (!os::exists(flags.docker_store_dir)) {
+    VLOG(1) << "Creating docker store directory";
+    os::mkdir(flags.docker_store_dir);
   }
 
-  Try<string> uriDir = os::dirname(uri);
-  if (uriDir.isError()) {
-    return Failure("Failed to obtain layer directory: " + uriDir.error());
+  if (!os::exists(path::join(flags.docker_store_dir, id))) {
+    os::mkdir(path::join(flags.docker_store_dir, id));
   }
 
-  Try<string> parentUri = path::join(uriDir.get(), parentId.get().value);
-  if (parentUri.isError()) {
-    return Failure("Failed to create parent layer uri: " + parentUri.error());
+  Try<Nothing> status = os::rename(
+      path::join(staging, id, "rootfs"),
+      path::join(flags.docker_store_dir, id, "rootfs"));
+
+  if (status.isError()) {
+    return Failure("Failed to move layer to store directory:" + status.error());
   }
 
-  return putLayer(parentUri.get(), directory)
-    .then([=](const Shared<DockerLayer>& parent) -> Shared<DockerLayer> {
-        return Shared<DockerLayer> (new DockerLayer(
-            hash.get(),
-            json.get(),
-            uri,
-            version.get(),
-            parent));
-    });
+  return Nothing();
 }
 
 
@@ -430,7 +425,7 @@ Future<Option<DockerImage>> LocalStoreProcess::get(const string& name)
 
 // Recover stored image layers and update layers map.
 // TODO(chenlily): Implement restore.
-Try<Nothing> LocalStoreProcess::restore()
+Try<Nothing> LocalStoreProcess::restore(const Flags& flags)
 {
   return Nothing();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/86d87aa7/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index 6dd1332..043f2d7 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -52,23 +52,37 @@ public:
 
   virtual ~Store() {}
 
-  // Put an image into to the store. Returns the DockerImage containing
-  // the manifest, hash of the image, and the path to the extracted
-  // image.
+  /**
+   * Put an image into the store. Returns the DockerImage containing
+   * the manifest, hash of the image, and the path to the extracted
+   * image.
+   *
+   * @param name The name of the Docker image being stored.
+   * @param sandbox The path of the directory in which the stderr and
+   *     stdout logs will be placed.
+   *
+   * @return The DockerImage placed in the store.
+   */
   virtual process::Future<DockerImage> put(
-      const std::string& uri,
       const std::string& name,
-      const std::string& directory) = 0;
-
-  // Get image by name.
+      const std::string& sandbox) = 0;
+
+  /**
+  * Get image by name.
+  *
+  * @param name The name of the Docker image to retrieve from store.
+  *
+  * @return The DockerImage or none if image is not in the store.
+  */
   virtual process::Future<Option<DockerImage>> get(const std::string& name) = 0;
 
+  // TODO(chenlily): Implement removing an image.
+
 protected:
   Store() {}
 };
 
-
-// Forward declaration.
+// Forward Declaration.
 class LocalStoreProcess;
 
 class LocalStore : public Store
@@ -80,10 +94,13 @@ public:
       const Flags& flags,
       Fetcher* fetcher);
 
+  /**
+   * Put assumes the image tar archive is located in the directory specified in
+   * the slave flag docker_discovery_local_dir and is named with <name>.tar .
+   */
   virtual process::Future<DockerImage> put(
-      const std::string& uri,
       const std::string& name,
-      const std::string& directory);
+      const std::string& sandbox);
 
   virtual process::Future<Option<DockerImage>> get(const std::string& name);
 
@@ -91,6 +108,7 @@ private:
   explicit LocalStore(process::Owned<LocalStoreProcess> process);
 
   LocalStore(const LocalStore&); // Not copyable.
+
   LocalStore& operator=(const LocalStore&); // Not assignable.
 
   process::Owned<LocalStoreProcess> process;
@@ -107,43 +125,49 @@ public:
       Fetcher* fetcher);
 
   process::Future<DockerImage> put(
-      const std::string& uri,
       const std::string& name,
-      const std::string& directory);
+      const std::string& sandbox);
 
   process::Future<Option<DockerImage>> get(const std::string& name);
 
 private:
-  LocalStoreProcess(
-      const Flags& flags,
-      Fetcher* fetcher);
+  LocalStoreProcess(const Flags& flags);
 
-  Try<Nothing> restore();
+  Try<Nothing> restore(const Flags& flags);
 
-  process::Future<process::Shared<DockerLayer>> putLayer(
-      const std::string& uri,
-      const std::string& directory);
+  process::Future<Nothing> untarImage(
+      const std::string& tarPath,
+      const std::string& staging);
 
-  process::Future<Nothing> untarLayer(
-      const std::string& uri);
+  process::Future<DockerImage> putImage(
+      const std::string& name,
+      const std::string& staging,
+      const std::string& sandbox);
+
+  Result<std::string> getParentId(
+      const std::string& staging,
+      const std::string& layerId);
 
-  process::Future<process::Shared<DockerLayer>> storeLayer(
-      const std::string& hash,
-      const std::string& uri,
-      const std::string& directory);
+  process::Future<Nothing> putLayers(
+      const std::string& staging,
+      const std::list<std::string>& layers,
+      const std::string& sandbox);
 
-  process::Future<process::Shared<DockerLayer>> entry(
-      const std::string& uri,
-      const std::string& directory);
+  process::Future<Nothing> untarLayer(
+      const std::string& staging,
+      const std::string& id,
+      const std::string& sandbox);
+
+  process::Future<Nothing> moveLayer(
+      const std::string& staging,
+      const std::string& id,
+      const std::string& sandbox);
 
   const Flags flags;
 
-  // name -> DockerImage
+  // This hashmap maps a Docker image by name to its corresponding DockerImage
+  // object.
   hashmap<std::string, DockerImage> images;
-  // hash -> DockerLayer
-  hashmap<std::string, process::Shared<DockerLayer>> layers;
-
-  Fetcher* fetcher;
 };
 
 } // namespace docker {

http://git-wip-us.apache.org/repos/asf/mesos/blob/86d87aa7/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index de0cac4..7df8062 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -53,6 +53,7 @@ public:
   std::string image_provisioner_backend;
   std::string appc_store_dir;
 
+  std::string docker_store;
   std::string docker_store_dir;
   std::string docker_backend;
   std::string docker_rootfs_dir;


[07/17] mesos git commit: Docker image store.

Posted by tn...@apache.org.
Docker image store.

Stored images currently kept indefinitely.

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


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

Branch: refs/heads/master
Commit: 38319bfb332138a0fea31944220a024c5e2ff1d1
Parents: 15c0d80
Author: Lily Chen <li...@mesosphere.io>
Authored: Mon Jul 6 15:39:55 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   2 +
 .../containerizer/provisioners/docker/store.cpp | 441 +++++++++++++++++++
 .../containerizer/provisioners/docker/store.hpp | 154 +++++++
 src/slave/flags.cpp                             |   5 +
 src/slave/flags.hpp                             |   1 +
 5 files changed, 603 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/38319bfb/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index f082484..da72818 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/store.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
   EXTRA_DIST += linux/fs.cpp
@@ -816,6 +817,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backends/bind.hpp		\
 	slave/containerizer/provisioner/backends/copy.hpp		\
 	slave/containerizer/provisioner/docker/registry_client.hpp	\
+	slave/containerizer/provisioner/docker/store.hpp		\
 	slave/containerizer/provisioner/docker/token_manager.hpp	\
 	slave/containerizer/isolators/posix.hpp				\
 	slave/containerizer/isolators/posix/disk.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/38319bfb/src/slave/containerizer/provisioners/docker/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.cpp b/src/slave/containerizer/provisioners/docker/store.cpp
new file mode 100644
index 0000000..e33b570
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/store.cpp
@@ -0,0 +1,441 @@
+/**
+ * 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/store.hpp"
+
+#include <list>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include <stout/os.hpp>
+#include <stout/json.hpp>
+
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "slave/containerizer/fetcher.hpp"
+#include "slave/flags.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+using std::pair;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+Try<Owned<Store>> Store::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  hashmap<string, Try<Owned<Store>>(*)(const Flags&, Fetcher*)> creators{
+    {"local", &LocalStore::create}
+  };
+
+  if (!creators.contains(flags.docker_store)) {
+    return Error("Unknown Docker store: " + flags.docker_store);
+  }
+
+  return creators[flags.docker_store](flags, fetcher);
+}
+
+
+Try<Owned<Store>> LocalStore::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  Try<Owned<LocalStoreProcess>> process =
+    LocalStoreProcess::create(flags, fetcher);
+  if (process.isError()) {
+    return Error("Failed to create store: " + process.error());
+  }
+
+  return Owned<Store>(new LocalStore(process.get()));
+}
+
+
+LocalStore::LocalStore(Owned<LocalStoreProcess> process)
+  : process(process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+LocalStore::~LocalStore()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<DockerImage> LocalStore::put(
+    const string& uri,
+    const string& name,
+    const string& directory)
+{
+  return dispatch(process.get(), &LocalStoreProcess::put, uri, name, directory);
+}
+
+
+Future<Option<DockerImage>> LocalStore::get(const string& name)
+{
+  return dispatch(process.get(), &LocalStoreProcess::get, name);
+}
+
+
+Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  Owned<LocalStoreProcess> store =
+    Owned<LocalStoreProcess>(new LocalStoreProcess(flags, fetcher));
+
+  Try<Nothing> restore = store->restore();
+  if (restore.isError()) {
+    return Error("Failed to restore store: " + restore.error());
+  }
+
+  return store;
+}
+
+
+LocalStoreProcess::LocalStoreProcess(
+    const Flags& flags,
+    Fetcher* fetcher)
+  : flags(flags),
+    fetcher(fetcher) {}
+
+// Currently only local file:// uri supported.
+// TODO(chenlily): Add support for fetching image from external uri.
+Future<DockerImage> LocalStoreProcess::put(
+    const string& uri,
+    const string& name,
+    const string& directory)
+{
+  string imageUri = uri;
+  if (strings::startsWith(imageUri, "file://")) {
+    imageUri = imageUri.substr(7);
+  }
+
+  Try<bool> isDir = os::stat::isdir(imageUri);
+  if (isDir.isError()) {
+    return Failure("Failed to check directory for uri '" +imageUri + "':"
+                   + isDir.error());
+  } else if (!isDir.get()) {
+    return Failure("Docker image uri '" + imageUri + "' is not a directory");
+  }
+
+  Try<string> repoPath = path::join(imageUri, "repositories");
+  if (repoPath.isError()) {
+    return Failure("Failed to create path to repository: " + repoPath.error());
+  }
+
+  Try<string> value = os::read(repoPath.get());
+  if (value.isError()) {
+    return Failure("Failed to read repository JSON: " + value.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
+  if (json.isError()) {
+    return Failure("Failed to parse JSON: " + json.error());
+  }
+
+  Try<pair<string, string>> repoTag = DockerImage::parseTag(name);
+  if (repoTag.isError()) {
+    return Failure("Failed to parse Docker image name: " + repoTag.error());
+  }
+
+  string repository = repoTag.get().first;
+  string tag = repoTag.get().second;
+
+  Result<JSON::Object> repositoryValue =
+    json.get().find<JSON::Object>(repository);
+  if (repositoryValue.isError()) {
+    return Failure("Failed to find repository: " + repositoryValue.error());
+  } else if (repositoryValue.isNone()) {
+    return Failure("Repository '" + repository + "' is not found");
+  }
+
+  JSON::Object repositoryJson = repositoryValue.get();
+
+  // We don't use JSON find here because a tag might contain a '.'.
+  std::map<string, JSON::Value>::const_iterator entry =
+    repositoryJson.values.find(tag);
+  if (entry == repositoryJson.values.end()) {
+    return Failure("Tag '" + tag + "' is not found");
+  } else if (!entry->second.is<JSON::String>()) {
+    return Failure("Tag JSON value expected to be JSON::String");
+  }
+
+  Try<string> layerUri = path::join(
+      imageUri,
+      entry->second.as<JSON::String>().value);
+  if (layerUri.isError()) {
+    return Failure("Failed to create path to image layer: " + layerUri.error());
+  }
+
+  return putLayer(layerUri.get(), directory)
+    .then([=](const Shared<DockerLayer>& layer) -> Future<DockerImage> {
+      DockerImage image(name, layer);
+      images[name] = image;
+      return image;
+    });
+}
+
+
+Future<Shared<DockerLayer>> LocalStoreProcess::putLayer(
+    const string& uri,
+    const string& directory)
+{
+  Try<string> hash = os::basename(uri);
+  if (hash.isError()) {
+    return Failure("Failed to determine hash for stored layer: " +
+                    hash.error());
+  }
+
+  if (layers.contains(hash.get())) {
+    return layers[hash.get()];
+  }
+
+  return untarLayer(uri)
+    .then([=]() {
+      return entry(uri, directory);
+    })
+    .then([=](const Shared<DockerLayer>& layer) {
+      VLOG(1) << "Stored layer with hash: " << hash.get();
+      layers[hash.get()] = layer;
+
+      return layer;
+    });
+}
+
+
+Future<Nothing> LocalStoreProcess::untarLayer(
+    const string& uri)
+{
+  string rootFs = path::join(uri, "rootfs");
+
+  if (os::exists(rootFs)) {
+    return Nothing();
+  } else {
+    os::mkdir(rootFs);
+  }
+
+  // Untar imageUri/hash/layer.tar into imageUri/hash/rootfs.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    rootFs,
+    "-x",
+    "-f",
+    path::join(uri, "layer.tar")};
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([=](const Option<int>& status) -> Future<Nothing> {
+        if (status.isNone()) {
+          return Failure("Failed to reap status for tar subprocess in " +
+                          uri);
+        }
+
+        if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+          return Failure("Untar failed with exit code: " +
+                          WSTRINGIFY(status.get()));
+        }
+
+        return Nothing();
+      });
+}
+
+
+Future<Shared<DockerLayer>> LocalStoreProcess::storeLayer(
+    const string& hash,
+    const string& uri,
+    const string& directory)
+{
+  string store = uri;
+
+  // Only copy if the store directory doesn't exist.
+  Future<Option<int>> status;
+  if (os::exists(store)) {
+    LOG(INFO) << "Layer store '" << store << "' exists, skipping rename";
+    status = 0;
+  } else {
+    Try<int> out = os::open(
+        path::join(directory, "stdout"),
+        O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
+        S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
+    if (out.isError()) {
+      return Failure("Failed to create 'stdout' file: " + out.error());
+    }
+
+    // Repeat for stderr.
+    Try<int> err = os::open(
+        path::join(directory, "stderr"),
+        O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
+        S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
+    if (err.isError()) {
+      os::close(out.get());
+      return Failure("Failed to create 'stderr' file: " + err.error());
+    }
+
+    vector<string> argv{
+      "cp",
+      "--archive",
+      path::join(uri, "rootfs"),
+      store
+    };
+
+    VLOG(1) << "Copying image with command: " << strings::join(" ", argv);
+
+    Try<Subprocess> s = subprocess(
+      "cp",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::FD(out.get()),
+      Subprocess::FD(err.get()));
+    if (s.isError()) {
+      return Failure("Failed to create 'cp' subprocess: " + s.error());
+    }
+
+    status = s.get().status();
+  }
+
+  return status
+    .then([=](const Option<int>& status) -> Future<Shared<DockerLayer>> {
+      if (status.isNone()) {
+        return Failure("Failed to reap subprocess to copy image");
+      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+        return Failure("Copy image failed with exit code: " +
+                        WSTRINGIFY(status.get()));
+      }
+
+      return entry(uri, directory);
+    })
+    .then([=](const Shared<DockerLayer>& layer) {
+      LOG(INFO) << "Stored layer with hash: " << hash;
+      layers[hash] = layer;
+
+      return layer;
+    });
+}
+
+
+Future<Shared<DockerLayer>> LocalStoreProcess::entry(
+    const string& uri,
+    const string& directory)
+{
+  Result<string> realpath = os::realpath(uri);
+  if (realpath.isError()) {
+    return Failure("Error in checking store path: " + realpath.error());
+  } else if (realpath.isNone()) {
+    return Failure("Store path not found");
+  }
+
+  Try<string> hash = os::basename(realpath.get());
+  if (hash.isError()) {
+    return Failure(
+      "Failed to determine hash for stored image: " + hash.error());
+  }
+
+  Try<string> version = os::read(path::join(uri, "VERSION"));
+  if (version.isError()) {
+    return Failure("Failed to determine version of JSON: " + version.error());
+  }
+
+  Try<string> manifest = os::read(path::join(uri, "json"));
+  if (manifest.isError()) {
+    return Failure("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
+  if (json.isError()) {
+    return Failure("Failed to parse manifest: " + json.error());
+  }
+
+  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
+  if (parentId.isNone()) {
+    return Shared<DockerLayer>(new DockerLayer(
+        hash.get(),
+        json.get(),
+        realpath.get(),
+        version.get(),
+        None()));
+  } else if (parentId.isError()) {
+    return Failure("Failed to read parent of layer: " + parentId.error());
+  }
+
+  Try<string> uriDir = os::dirname(uri);
+  if (uriDir.isError()) {
+    return Failure("Failed to obtain layer directory: " + uriDir.error());
+  }
+
+  Try<string> parentUri = path::join(uriDir.get(), parentId.get().value);
+  if (parentUri.isError()) {
+    return Failure("Failed to create parent layer uri: " + parentUri.error());
+  }
+
+  return putLayer(parentUri.get(), directory)
+    .then([=](const Shared<DockerLayer>& parent) -> Shared<DockerLayer> {
+        return Shared<DockerLayer> (new DockerLayer(
+            hash.get(),
+            json.get(),
+            uri,
+            version.get(),
+            parent));
+    });
+}
+
+
+Future<Option<DockerImage>> LocalStoreProcess::get(const string& name)
+{
+  if (!images.contains(name)) {
+    return None();
+  }
+
+  return images[name];
+}
+
+
+// Recover stored image layers and update layers map.
+// TODO(chenlily): Implement restore.
+Try<Nothing> LocalStoreProcess::restore()
+{
+  return Nothing();
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/38319bfb/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
new file mode 100644
index 0000000..6dd1332
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -0,0 +1,154 @@
+/**
+ * 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_STORE__
+#define __MESOS_DOCKER_STORE__
+
+#include <string>
+#include <vector>
+
+#include <stout/hashmap.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+#include <stout/result.hpp>
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+#include <process/shared.hpp>
+
+#include "slave/containerizer/fetcher.hpp"
+#include "slave/containerizer/provisioners/docker.hpp"
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Store fetches the Docker images and stores them on disk.
+class Store
+{
+public:
+  static Try<process::Owned<Store>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  virtual ~Store() {}
+
+  // Put an image into to the store. Returns the DockerImage containing
+  // the manifest, hash of the image, and the path to the extracted
+  // image.
+  virtual process::Future<DockerImage> put(
+      const std::string& uri,
+      const std::string& name,
+      const std::string& directory) = 0;
+
+  // Get image by name.
+  virtual process::Future<Option<DockerImage>> get(const std::string& name) = 0;
+
+protected:
+  Store() {}
+};
+
+
+// Forward declaration.
+class LocalStoreProcess;
+
+class LocalStore : public Store
+{
+public:
+  virtual ~LocalStore();
+
+  static Try<process::Owned<Store>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  virtual process::Future<DockerImage> put(
+      const std::string& uri,
+      const std::string& name,
+      const std::string& directory);
+
+  virtual process::Future<Option<DockerImage>> get(const std::string& name);
+
+private:
+  explicit LocalStore(process::Owned<LocalStoreProcess> process);
+
+  LocalStore(const LocalStore&); // Not copyable.
+  LocalStore& operator=(const LocalStore&); // Not assignable.
+
+  process::Owned<LocalStoreProcess> process;
+};
+
+
+class LocalStoreProcess : public process::Process<LocalStoreProcess>
+{
+public:
+  ~LocalStoreProcess() {}
+
+  static Try<process::Owned<LocalStoreProcess>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  process::Future<DockerImage> put(
+      const std::string& uri,
+      const std::string& name,
+      const std::string& directory);
+
+  process::Future<Option<DockerImage>> get(const std::string& name);
+
+private:
+  LocalStoreProcess(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  Try<Nothing> restore();
+
+  process::Future<process::Shared<DockerLayer>> putLayer(
+      const std::string& uri,
+      const std::string& directory);
+
+  process::Future<Nothing> untarLayer(
+      const std::string& uri);
+
+  process::Future<process::Shared<DockerLayer>> storeLayer(
+      const std::string& hash,
+      const std::string& uri,
+      const std::string& directory);
+
+  process::Future<process::Shared<DockerLayer>> entry(
+      const std::string& uri,
+      const std::string& directory);
+
+  const Flags flags;
+
+  // name -> DockerImage
+  hashmap<std::string, DockerImage> images;
+  // hash -> DockerLayer
+  hashmap<std::string, process::Shared<DockerLayer>> layers;
+
+  Fetcher* fetcher;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_STORE__

http://git-wip-us.apache.org/repos/asf/mesos/blob/38319bfb/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 8792162..5406ef8 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -93,6 +93,11 @@ mesos::internal::slave::Flags::Flags()
       "Directory the appc provisioner will store images in.",
       "/tmp/mesos/store/appc");
 
+  add(&Flags::docker_store_dir,
+      "docker_store_dir",
+      "Directory the docker provisioner will store images in",
+      "/tmp/mesos/store/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/38319bfb/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 3f6601a..1bfb447 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -53,6 +53,7 @@ public:
   std::string image_provisioner_backend;
   std::string appc_store_dir;
 
+  std::string docker_store_dir;
   std::string default_role;
   Option<std::string> attributes;
   Bytes fetcher_cache_size;


[16/17] mesos git commit: Refactor store implementations to pullers.

Posted by tn...@apache.org.
Refactor store implementations to pullers.


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

Branch: refs/heads/master
Commit: 9fb62ceccfd5eda3027560ef0d816e5d2caa937d
Parents: 4dedbf4
Author: Timothy Chen <tn...@apache.org>
Authored: Sun Sep 20 23:18:53 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |  13 +-
 src/slave/containerizer/provisioner.cpp         |  82 ----
 .../provisioner/docker/local_puller.cpp         | 349 ++++++++++++++
 .../provisioner/docker/local_puller.hpp         |  66 +++
 .../provisioner/docker/local_store.cpp          | 460 -------------------
 .../provisioner/docker/local_store.hpp          |  64 ---
 .../provisioner/docker/message.hpp              |   6 +-
 .../provisioner/docker/message.proto            |   7 +-
 .../provisioner/docker/metadata_manager.cpp     | 101 ++--
 .../provisioner/docker/metadata_manager.hpp     |  32 +-
 .../containerizer/provisioner/docker/paths.cpp  |  43 +-
 .../containerizer/provisioner/docker/paths.hpp  |  44 +-
 .../containerizer/provisioner/docker/puller.cpp |  46 ++
 .../containerizer/provisioner/docker/puller.hpp |  68 +++
 .../containerizer/provisioner/docker/store.cpp  | 247 +++++++++-
 .../containerizer/provisioner/docker/store.hpp  |  36 +-
 src/slave/containerizer/provisioner/store.cpp   |   3 +
 src/slave/flags.cpp                             |  20 +-
 src/slave/flags.hpp                             |   5 +-
 .../containerizer/provisioner_docker_tests.cpp  |  74 +--
 20 files changed, 971 insertions(+), 795 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 818d62d..8aa4566 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -253,7 +253,6 @@ DOCKER_PROVISIONER_PROTOS =                                             \
   slave/containerizer/provisioner/docker/message.pb.cc			\
   slave/containerizer/provisioner/docker/message.pb.h
 
-
 BUILT_SOURCES += $(DOCKER_PROVISIONER_PROTOS)
 CLEANFILES += $(DOCKER_PROVISIONER_PROTOS)
 
@@ -488,7 +487,6 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	master/validation.cpp						\
 	master/allocator/allocator.cpp					\
 	master/allocator/sorter/drf/sorter.cpp				\
-	messages/docker_provisioner.proto			        \
 	messages/flags.proto						\
 	messages/messages.cpp						\
 	messages/messages.proto						\
@@ -530,9 +528,11 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/provisioner/appc/store.cpp			\
 	slave/containerizer/provisioner/backend.cpp			\
 	slave/containerizer/provisioner/backends/copy.cpp		\
-	slave/containerizer/provisioner/docker/local_store.cpp		\
+	slave/containerizer/provisioner/docker/local_puller.cpp		\
+	slave/containerizer/provisioner/docker/message.proto		\
 	slave/containerizer/provisioner/docker/metadata_manager.cpp	\
 	slave/containerizer/provisioner/docker/paths.cpp		\
+	slave/containerizer/provisioner/docker/puller.cpp		\
 	slave/containerizer/provisioner/docker/registry_client.cpp	\
 	slave/containerizer/provisioner/docker/store.cpp		\
 	slave/containerizer/provisioner/docker/token_manager.cpp	\
@@ -800,7 +800,6 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	master/allocator/mesos/hierarchical.hpp				\
 	master/allocator/sorter/drf/sorter.hpp				\
 	master/allocator/sorter/sorter.hpp				\
-	messages/docker_provisioner.hpp				        \
 	messages/flags.hpp						\
 	messages/messages.hpp						\
 	module/manager.hpp						\
@@ -830,10 +829,11 @@ 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/local_store.hpp		\
+	slave/containerizer/provisioner/docker/local_puller.hpp		\
+	slave/containerizer/provisioner/docker/message.hpp		\
 	slave/containerizer/provisioner/docker/metadata_manager.hpp	\
 	slave/containerizer/provisioner/docker/paths.hpp		\
+	slave/containerizer/provisioner/docker/puller.hpp		\
 	slave/containerizer/provisioner/docker/registry_client.hpp	\
 	slave/containerizer/provisioner/docker/store.hpp		\
 	slave/containerizer/provisioner/docker/token_manager.hpp	\
@@ -1776,7 +1776,6 @@ mesos_tests_DEPENDENCIES = # Initialized to allow += below.
 if OS_LINUX
   mesos_tests_SOURCES += tests/containerizer/cgroups_isolator_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/cgroups_tests.cpp
-  mesos_tests_SOURCES += tests/containerizer/docker_provisioner_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/filesystem_isolator_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/fs_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/launch_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner.cpp b/src/slave/containerizer/provisioner.cpp
deleted file mode 100644
index 6a0faac..0000000
--- a/src/slave/containerizer/provisioner.cpp
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * 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/9fb62cec/src/slave/containerizer/provisioner/docker/local_puller.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/local_puller.cpp b/src/slave/containerizer/provisioner/docker/local_puller.cpp
new file mode 100644
index 0000000..4a0b7d1
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/local_puller.cpp
@@ -0,0 +1,349 @@
+/**
+ * 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 <list>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include <stout/json.hpp>
+#include <stout/os.hpp>
+#include <stout/result.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/subprocess.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "slave/containerizer/provisioner/docker/local_puller.hpp"
+#include "slave/containerizer/provisioner/docker/paths.hpp"
+#include "slave/containerizer/provisioner/docker/store.hpp"
+
+using namespace process;
+
+using std::list;
+using std::pair;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+class LocalPullerProcess : public process::Process<LocalPullerProcess>
+{
+public:
+  LocalPullerProcess(const Flags& _flags) : flags(_flags) {}
+
+  ~LocalPullerProcess() {}
+
+  process::Future<list<pair<string, string>>> pull(
+      const Image::Name& name,
+      const string& directory);
+
+private:
+  process::Future<Nothing> untarImage(
+      const std::string& tarPath,
+      const std::string& directory);
+
+  process::Future<list<pair<string, string>>> putImage(
+      const Image::Name& name,
+      const std::string& directory);
+
+  process::Future<list<pair<string, string>>> putLayers(
+      const std::string& directory,
+      const std::vector<std::string>& layerIds);
+
+  process::Future<pair<string, string>> putLayer(
+      const std::string& directory,
+      const std::string& layerId);
+
+  const Flags flags;
+};
+
+
+LocalPuller::LocalPuller(const Flags& flags)
+{
+  process = Owned<LocalPullerProcess>(new LocalPullerProcess(flags));
+  process::spawn(process.get());
+}
+
+
+LocalPuller::~LocalPuller()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<list<pair<string, string>>> LocalPuller::pull(
+    const Image::Name& name,
+    const string& directory)
+{
+  return dispatch(process.get(), &LocalPullerProcess::pull, name, directory);
+}
+
+
+Future<list<pair<string, string>>> LocalPullerProcess::pull(
+    const Image::Name& name,
+    const string& directory)
+{
+  string tarPath = paths::getImageArchiveTarPath(
+      flags.docker_local_archives_dir,
+      stringify(name));
+
+  if (!os::exists(tarPath)) {
+    return Failure("Failed to find archive for image '" + stringify(name) +
+                   "' at '" + tarPath + "'");
+  }
+
+  return untarImage(tarPath, directory)
+    .then(defer(self(), &Self::putImage, name, directory));
+}
+
+
+Future<Nothing> LocalPullerProcess::untarImage(
+    const string& tarPath,
+    const string& directory)
+{
+  VLOG(1) << "Untarring image from '" << directory
+          << "' to '" << tarPath << "'";
+
+  // Untar store_discovery_local_dir/name.tar into directory/.
+  // TODO(tnachen): Terminate tar process when slave exits.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    directory,
+    "-x",
+    "-f",
+    tarPath
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([tarPath](const Option<int>& status) -> Future<Nothing> {
+      if (status.isNone()) {
+        return Failure("Failed to reap status for tar subprocess in " +
+                        tarPath);
+      }
+      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+          return Failure("Untar image failed with exit code: " +
+                          WSTRINGIFY(status.get()));
+      }
+
+      return Nothing();
+    });
+}
+
+
+static Result<string> getParentId(
+    const string& directory,
+    const string& layerId)
+{
+  Try<string> manifest =
+    os::read(paths::getImageArchiveLayerManifestPath(directory, layerId));
+  if (manifest.isError()) {
+    return Error("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
+  if (json.isError()) {
+    return Error("Failed to parse manifest: " + json.error());
+  }
+
+  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
+  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
+    return None();
+  } else if (parentId.isError()) {
+    return Error("Failed to read parent of layer: " + parentId.error());
+  }
+  return parentId.get().value;
+}
+
+
+Future<list<pair<string, string>>> LocalPullerProcess::putImage(
+    const Image::Name& name,
+    const string& directory)
+{
+  Try<string> value =
+    os::read(paths::getImageArchiveRepositoriesPath(directory));
+  if (value.isError()) {
+    return Failure("Failed to read repository JSON: " + value.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
+  if (json.isError()) {
+    return Failure("Failed to parse JSON: " + json.error());
+  }
+
+  Result<JSON::Object> repositoryValue =
+    json.get().find<JSON::Object>(name.repository());
+  if (repositoryValue.isError()) {
+    return Failure("Failed to find repository: " + repositoryValue.error());
+  } else if (repositoryValue.isNone()) {
+    return Failure("Repository '" + name.repository() + "' is not found");
+  }
+
+  JSON::Object repositoryJson = repositoryValue.get();
+
+  // We don't use JSON find here because a tag might contain a '.'.
+  std::map<string, JSON::Value>::const_iterator entry =
+    repositoryJson.values.find(name.tag());
+  if (entry == repositoryJson.values.end()) {
+    return Failure("Tag '" + name.tag() + "' is not found");
+  } else if (!entry->second.is<JSON::String>()) {
+    return Failure("Tag JSON value expected to be JSON::String");
+  }
+
+  string layerId = entry->second.as<JSON::String>().value;
+
+  Try<string> manifest =
+    os::read(paths::getImageArchiveLayerManifestPath(directory, layerId));
+  if (manifest.isError()) {
+    return Failure("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
+  if (manifestJson.isError()) {
+    return Failure("Failed to parse manifest: " + manifestJson.error());
+  }
+
+  vector<string> layerIds;
+  layerIds.push_back(layerId);
+  Result<string> parentId = getParentId(directory, layerId);
+  while (parentId.isSome()) {
+    layerIds.insert(layerIds.begin(), parentId.get());
+    parentId = getParentId(directory, parentId.get());
+  }
+
+  if (parentId.isError()) {
+    return Failure("Failed to find parent layer id of layer '" + layerId +
+                   "': " + parentId.error());
+  }
+
+  return putLayers(directory, layerIds);
+}
+
+
+Future<list<pair<string, string>>> LocalPullerProcess::putLayers(
+    const string& directory,
+    const vector<string>& layerIds)
+{
+  list<Future<pair<string, string>>> futures;
+  foreach (const string& layerId, layerIds) {
+    futures.push_back(putLayer(directory, layerId));
+  }
+
+  return collect(futures);
+}
+
+
+Future<pair<string, string>> LocalPullerProcess::putLayer(
+    const string& directory,
+    const string& layerId)
+{
+  // We untar the layer from source into a directory, then move the
+  // layer into store. We do this instead of untarring directly to
+  // store to make sure we don't end up with partially untarred layer
+  // rootfs.
+
+  string localRootfsPath =
+    paths::getImageArchiveLayerRootfsPath(directory, layerId);
+
+  // Image layer has been untarred but is not present in the store directory.
+  if (os::exists(localRootfsPath)) {
+    LOG(WARNING) << "Image layer '" << layerId << "' rootfs present at but not "
+                 << "in store directory '" << localRootfsPath << "'. Removing "
+                 << "staged rootfs and untarring layer again.";
+
+    Try<Nothing> rmdir = os::rmdir(localRootfsPath);
+    if (rmdir.isError()) {
+      return Failure("Failed to remove incomplete staged rootfs for layer '" +
+                     layerId + "': " + rmdir.error());
+    }
+  }
+
+  Try<Nothing> mkdir = os::mkdir(localRootfsPath);
+  if (mkdir.isError()) {
+    return Failure("Failed to create rootfs path '" + localRootfsPath +
+                   "': " + mkdir.error());
+  }
+
+  // Untar directory/id/layer.tar into directory/id/rootfs.
+  // The tar file will be removed when the staging directory is
+  // removed.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    localRootfsPath,
+    "-x",
+    "-f",
+    paths::getImageArchiveLayerTarPath(directory, layerId)
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([directory, layerId](
+        const Option<int>& status) -> Future<pair<string, string>> {
+      if (status.isNone()) {
+        return Failure("Failed to reap subprocess to untar image");
+      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+        return Failure("Untar failed with exit code: " +
+                        WSTRINGIFY(status.get()));
+      }
+
+      const string rootfsPath =
+        paths::getImageArchiveLayerRootfsPath(directory, layerId);
+
+      if (!os::exists(rootfsPath)) {
+        return Failure("Failed to find the rootfs path after extracting layer"
+                       " '" + layerId + "'");
+      }
+
+      return pair<string, string>(layerId, rootfsPath);
+    });
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/local_puller.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/local_puller.hpp b/src/slave/containerizer/provisioner/docker/local_puller.hpp
new file mode 100644
index 0000000..4574e8a
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/local_puller.hpp
@@ -0,0 +1,66 @@
+/**
+ * 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 __PROVISIONER_DOCKER_LOCAL_PULLER_HPP__
+#define __PROVISIONER_DOCKER_LOCAL_PULLER_HPP__
+
+#include "slave/containerizer/provisioner/store.hpp"
+
+#include "slave/containerizer/provisioner/docker/message.hpp"
+#include "slave/containerizer/provisioner/docker/puller.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Forward declaration.
+class LocalPullerProcess;
+
+
+/**
+ * LocalPuller assumes Docker images are stored in a local directory
+ * (configured with flags.docker_local_archives_dir), with all the
+ * images saved as tars with file names in the form of <repo>:<tag>.tar.
+ */
+class LocalPuller : public Puller
+{
+public:
+  explicit LocalPuller(const Flags& flags);
+
+  ~LocalPuller();
+
+  process::Future<std::list<std::pair<std::string, std::string>>> pull(
+      const Image::Name& name,
+      const std::string& directory);
+
+private:
+  LocalPuller& operator=(const LocalPuller&) = delete; // Not assignable.
+  LocalPuller(const LocalPuller&) = delete; // Not copyable.
+
+  process::Owned<LocalPullerProcess> process;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __PROVISIONER_DOCKER_LOCAL_PULLER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/local_store.cpp b/src/slave/containerizer/provisioner/docker/local_store.cpp
deleted file mode 100644
index 6a73dbb..0000000
--- a/src/slave/containerizer/provisioner/docker/local_store.cpp
+++ /dev/null
@@ -1,460 +0,0 @@
-/**
- * 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 <list>
-#include <vector>
-
-#include <glog/logging.h>
-
-#include <stout/json.hpp>
-#include <stout/os.hpp>
-#include <stout/result.hpp>
-
-#include <process/collect.hpp>
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-
-#include "common/status_utils.hpp"
-
-#include "slave/containerizer/provisioner/docker/local_store.hpp"
-#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
-#include "slave/containerizer/provisioner/docker/paths.hpp"
-#include "slave/containerizer/provisioner/docker/store.hpp"
-
-#include "slave/flags.hpp"
-
-using namespace process;
-
-using std::list;
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-class LocalStoreProcess : public process::Process<LocalStoreProcess>
-{
-public:
-  LocalStoreProcess(
-      const Flags& _flags,
-      Owned<MetadataManager> _metadataManager)
-    : flags(_flags), metadataManager(_metadataManager) {}
-
-  ~LocalStoreProcess() {}
-
-  static Try<process::Owned<LocalStoreProcess>> create(const Flags& flags);
-
-  process::Future<vector<string>> get(const Image& image);
-
-  process::Future<Nothing> recover();
-
-private:
-  process::Future<DockerImage> _get(
-      const DockerImage::Name& name,
-      const Option<DockerImage>& image);
-
-  process::Future<Nothing> untarImage(
-      const std::string& tarPath,
-      const std::string& staging);
-
-  process::Future<DockerImage> putImage(
-      const DockerImage::Name& name,
-      const std::string& staging);
-
-  Result<std::string> getParentId(
-      const std::string& staging,
-      const std::string& layerId);
-
-  process::Future<Nothing> putLayers(
-      const std::string& staging,
-      const std::list<std::string>& layerIds);
-
-  process::Future<Nothing> putLayer(
-      const std::string& staging,
-      const std::string& id);
-
-  process::Future<Nothing> moveLayer(
-      const std::string& staging,
-      const std::string& id);
-
-  const Flags flags;
-  process::Owned<MetadataManager> metadataManager;
-};
-
-
-Try<Owned<slave::Store>> LocalStore::create(const Flags& flags)
-{
-  if (!os::exists(flags.docker_store_dir)) {
-    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker store directory: " + mkdir.error());
-    }
-  }
-
-  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
-    Try<Nothing> mkdir =
-      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker store staging directory: " +
-                   mkdir.error());
-    }
-  }
-
-  Try<Owned<MetadataManager>> metadataManager = MetadataManager::create(flags);
-  if (metadataManager.isError()) {
-    return Error(metadataManager.error());
-  }
-
-  Owned<LocalStoreProcess> process(
-      new LocalStoreProcess(flags, metadataManager.get()));
-
-  return Owned<slave::Store>(new LocalStore(process));
-}
-
-
-LocalStore::LocalStore(Owned<LocalStoreProcess> _process) : process(_process)
-{
-  process::spawn(CHECK_NOTNULL(process.get()));
-}
-
-
-LocalStore::~LocalStore()
-{
-  process::terminate(process.get());
-  process::wait(process.get());
-}
-
-
-Future<vector<string>> LocalStore::get(const Image& image)
-{
-  return dispatch(process.get(), &LocalStoreProcess::get, image);
-}
-
-
-Future<Nothing> LocalStore::recover()
-{
-  return dispatch(process.get(), &LocalStoreProcess::recover);
-}
-
-
-Future<vector<string>> LocalStoreProcess::get(const Image& image)
-{
-  CHECK_EQ(image.type(), Image::DOCKER);
-
-  Try<DockerImage::Name> dockerName = parseName(image.docker().name());
-  if (dockerName.isError()) {
-    return Failure("Unable to parse docker image name: " + dockerName.error());
-  }
-
-  return metadataManager->get(dockerName.get())
-    .then(defer(self(), &Self::_get, dockerName.get(), lambda::_1))
-    .then([](const DockerImage& dockerImage) {
-      vector<string> layers;
-      foreach (const string& layer, dockerImage.layer_ids()) {
-        layers.push_back(layer);
-      }
-
-      return layers;
-    });
-}
-
-
-Future<DockerImage> LocalStoreProcess::_get(
-    const DockerImage::Name& name,
-    const Option<DockerImage>& image)
-{
-  if (image.isSome()) {
-    return image.get();
-  }
-
-  string tarPath = paths::getLocalImageTarPath(
-      flags.docker_store_discovery_local_dir,
-      stringify(name));
-
-  if (!os::exists(tarPath)) {
-    VLOG(1) << "Unable to find image in local store with path: " << tarPath;
-    return Failure("No Docker image tar archive found");
-  }
-
-  // Create a temporary staging directory.
-  Try<string> staging =
-    os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
-  if (staging.isError()) {
-    return Failure("Failed to create a staging directory");
-  }
-
-  return untarImage(tarPath, staging.get())
-    .then(defer(self(), &Self::putImage, name, staging.get()));
-}
-
-
-Future<Nothing> LocalStoreProcess::recover()
-{
-  return metadataManager->recover();
-}
-
-
-Future<Nothing> LocalStoreProcess::untarImage(
-    const string& tarPath,
-    const string& staging)
-{
-  VLOG(1) << "Untarring image at: " << tarPath;
-
-  // Untar store_discovery_local_dir/name.tar into staging/.
-  vector<string> argv = {
-    "tar",
-    "-C",
-    staging,
-    "-x",
-    "-f",
-    tarPath
-  };
-
-  Try<Subprocess> s = subprocess(
-      "tar",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"));
-
-  if (s.isError()) {
-    return Failure("Failed to create tar subprocess: " + s.error());
-  }
-
-  return s.get().status()
-    .then([=](const Option<int>& status) -> Future<Nothing> {
-      if (status.isNone()) {
-        return Failure("Failed to reap status for tar subprocess in " +
-                        tarPath);
-      }
-      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-          return Failure("Untar image failed with exit code: " +
-                          WSTRINGIFY(status.get()));
-      }
-
-      return Nothing();
-    });
-}
-
-
-Future<DockerImage> LocalStoreProcess::putImage(
-    const DockerImage::Name& name,
-    const string& staging)
-{
-  Try<string> value = os::read(paths::getLocalImageRepositoriesPath(staging));
-  if (value.isError()) {
-    return Failure("Failed to read repository JSON: " + value.error());
-  }
-
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
-  if (json.isError()) {
-    return Failure("Failed to parse JSON: " + json.error());
-  }
-
-  Result<JSON::Object> repositoryValue =
-    json.get().find<JSON::Object>(name.repository());
-  if (repositoryValue.isError()) {
-    return Failure("Failed to find repository: " + repositoryValue.error());
-  } else if (repositoryValue.isNone()) {
-    return Failure("Repository '" + name.repository() + "' is not found");
-  }
-
-  JSON::Object repositoryJson = repositoryValue.get();
-
-  // We don't use JSON find here because a tag might contain a '.'.
-  std::map<string, JSON::Value>::const_iterator entry =
-    repositoryJson.values.find(name.tag());
-  if (entry == repositoryJson.values.end()) {
-    return Failure("Tag '" + name.tag() + "' is not found");
-  } else if (!entry->second.is<JSON::String>()) {
-    return Failure("Tag JSON value expected to be JSON::String");
-  }
-
-  string layerId = entry->second.as<JSON::String>().value;
-
-  Try<string> manifest =
-    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
-  if (manifest.isError()) {
-    return Failure("Failed to read manifest: " + manifest.error());
-  }
-
-  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
-  if (manifestJson.isError()) {
-    return Failure("Failed to parse manifest: " + manifestJson.error());
-  }
-
-  list<string> layerIds;
-  layerIds.push_back(layerId);
-  Result<string> parentId = getParentId(staging, layerId);
-  while(parentId.isSome()) {
-    layerIds.push_front(parentId.get());
-    parentId = getParentId(staging, parentId.get());
-  }
-  if (parentId.isError()) {
-    return Failure("Failed to obtain parent layer id: " + parentId.error());
-  }
-
-  return putLayers(staging, layerIds)
-    .then([=]() -> Future<DockerImage> {
-      return metadataManager->put(name, layerIds);
-    });
-}
-
-
-Result<string> LocalStoreProcess::getParentId(
-    const string& staging,
-    const string& layerId)
-{
-  Try<string> manifest =
-    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
-  if (manifest.isError()) {
-    return Error("Failed to read manifest: " + manifest.error());
-  }
-
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
-  if (json.isError()) {
-    return Error("Failed to parse manifest: " + json.error());
-  }
-
-  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
-  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
-    return None();
-  } else if (parentId.isError()) {
-    return Error("Failed to read parent of layer: " + parentId.error());
-  }
-  return parentId.get().value;
-}
-
-
-Future<Nothing> LocalStoreProcess::putLayers(
-    const string& staging,
-    const list<string>& layerIds)
-{
-  list<Future<Nothing>> futures{ Nothing() };
-  foreach (const string& layer, layerIds) {
-    futures.push_back(
-        futures.back().then(
-          defer(self(), &Self::putLayer, staging, layer)));
-  }
-
-  return collect(futures)
-    .then([]() -> Future<Nothing> { return Nothing(); });
-}
-
-
-Future<Nothing> LocalStoreProcess::putLayer(
-    const string& staging,
-    const string& id)
-{
-  // We untar the layer from source into a staging directory, then
-  // move the layer into store. We do this instead of untarring
-  // directly to store to make sure we don't end up with partially
-  // untarred layer rootfs.
-
-  // Check if image layer rootfs is already in store.
-  if (os::exists(paths::getImageLayerRootfsPath(flags.docker_store_dir, id))) {
-    VLOG(1) << "Image layer '" << id << "' rootfs already in store. "
-            << "Skipping put.";
-    return Nothing();
-  }
-
-  const string imageLayerPath =
-    paths::getImageLayerPath(flags.docker_store_dir, id);
-  if (!os::exists(imageLayerPath)) {
-    Try<Nothing> mkdir = os::mkdir(imageLayerPath);
-    if (mkdir.isError()) {
-      return Failure("Failed to create Image layer directory '" +
-                     imageLayerPath + "': " + mkdir.error());
-    }
-  }
-
-  // Image layer has been untarred but is not present in the store directory.
-  string localRootfsPath = paths::getLocalImageLayerRootfsPath(staging, id);
-  if (os::exists(localRootfsPath)) {
-    LOG(WARNING) << "Image layer '" << id << "' rootfs present at but not in "
-                 << "store directory '" << localRootfsPath << "'. Removing "
-                 << "staged rootfs and untarring layer again.";
-    Try<Nothing> rmdir = os::rmdir(localRootfsPath);
-    if (rmdir.isError()) {
-      return Failure("Failed to remove incomplete staged rootfs for layer '" +
-                     id + "': " + rmdir.error());
-    }
-  }
-
-  Try<Nothing> mkdir = os::mkdir(localRootfsPath);
-  if (mkdir.isError()) {
-    return Failure("Failed to create rootfs path '" + localRootfsPath + "': " +
-                   mkdir.error());
-  }
-  // Untar staging/id/layer.tar into staging/id/rootfs.
-  vector<string> argv = {
-    "tar",
-    "-C",
-    localRootfsPath,
-    "-x",
-    "-f",
-    paths::getLocalImageLayerTarPath(staging, id)
-  };
-
-  Try<Subprocess> s = subprocess(
-      "tar",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"));
-  if (s.isError()) {
-    return Failure("Failed to create tar subprocess: " + s.error());
-  }
-
-  return s.get().status()
-    .then([=](const Option<int>& status) -> Future<Nothing> {
-      if (status.isNone()) {
-        return Failure("Failed to reap subprocess to untar image");
-      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-        return Failure("Untar failed with exit code: " +
-                        WSTRINGIFY(status.get()));
-      }
-
-      return moveLayer(staging, id);
-    });
-}
-
-
-Future<Nothing> LocalStoreProcess::moveLayer(
-    const string& staging,
-    const string& id)
-{
-  Try<Nothing> status = os::rename(
-      paths::getLocalImageLayerRootfsPath(staging, id),
-      paths::getImageLayerRootfsPath(flags.docker_store_dir, id));
-
-  if (status.isError()) {
-    return Failure("Failed to move layer to store directory: "
-                   + status.error());
-  }
-
-  return Nothing();
-}
-
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/local_store.hpp b/src/slave/containerizer/provisioner/docker/local_store.hpp
deleted file mode 100644
index 5f6152b..0000000
--- a/src/slave/containerizer/provisioner/docker/local_store.hpp
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * 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_LOCAL_STORE_HPP__
-#define __MESOS_DOCKER_LOCAL_STORE_HPP__
-
-#include "slave/containerizer/provisioner/store.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-// Forward declaration.
-class LocalStoreProcess;
-
-
-/**
- * LocalStore assumes Docker images are stored in a local directory
- * (configured with flags.docker_discovery_local_dir), with all the
- * images saved as tar with the name as the image name with tag (e.g:
- * ubuntu:14.04.tar).
- */
-class LocalStore : public slave::Store
-{
-public:
-  static Try<process::Owned<Store>> create(const Flags& flags);
-
-  virtual ~LocalStore();
-
-  virtual process::Future<Nothing> recover();
-
-  virtual process::Future<std::vector<std::string>> get(const Image& image);
-
-private:
-  explicit LocalStore(process::Owned<LocalStoreProcess> _process);
-
-  LocalStore& operator=(const LocalStore&) = delete; // Not assignable.
-  LocalStore(const LocalStore&) = delete; // Not copyable.
-
-  process::Owned<LocalStoreProcess> process;
-};
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_LOCAL_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/message.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/message.hpp b/src/slave/containerizer/provisioner/docker/message.hpp
index c1596df..6368bf4 100644
--- a/src/slave/containerizer/provisioner/docker/message.hpp
+++ b/src/slave/containerizer/provisioner/docker/message.hpp
@@ -29,9 +29,9 @@ namespace internal {
 namespace slave {
 namespace docker {
 
-inline DockerImage::Name parseName(const std::string& value)
+inline Image::Name parseName(const std::string& value)
 {
-  DockerImage::Name imageName;
+  Image::Name imageName;
   Option<std::string> registry = None();
   std::vector<std::string> components = strings::split(value, "/");
   if (components.size() > 2) {
@@ -53,7 +53,7 @@ inline DockerImage::Name parseName(const std::string& value)
 
 inline std::ostream& operator<<(
     std::ostream& stream,
-    const DockerImage::Name& name)
+    const Image::Name& name)
 {
   if (name.has_registry()) {
     return stream << name.registry() << "/" << name.repository() << ":"

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/message.proto
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/message.proto b/src/slave/containerizer/provisioner/docker/message.proto
index d771968..bbac2e6 100644
--- a/src/slave/containerizer/provisioner/docker/message.proto
+++ b/src/slave/containerizer/provisioner/docker/message.proto
@@ -25,7 +25,7 @@ package mesos.internal.slave.docker;
  * The layerIds are ordered, with the root layer id (no parent layer id) first
  * and the leaf layer id last.
  */
-message DockerImage {
+message Image {
   message Name {
     optional string registry = 1;
     required string repository = 2;
@@ -38,6 +38,7 @@ message DockerImage {
   repeated string layer_ids = 2;
 }
 
-message DockerImages {
-  repeated DockerImage images = 1;
+
+message Images {
+  repeated Image images = 1;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/metadata_manager.cpp b/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
index 197931c..2b2de52 100644
--- a/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
+++ b/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
+
 #include <vector>
 
 #include <glog/logging.h>
@@ -33,7 +35,6 @@
 
 #include "slave/containerizer/provisioner/docker/paths.hpp"
 #include "slave/containerizer/provisioner/docker/message.hpp"
-#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
 
 #include "slave/state.hpp"
 
@@ -48,48 +49,41 @@ namespace internal {
 namespace slave {
 namespace docker {
 
-
 class MetadataManagerProcess : public process::Process<MetadataManagerProcess>
 {
 public:
-  ~MetadataManagerProcess() {}
+  MetadataManagerProcess(const Flags& _flags) : flags(_flags) {}
 
-  static Try<process::Owned<MetadataManagerProcess>> create(
-      const Flags& flags);
+  ~MetadataManagerProcess() {}
 
-  Future<DockerImage> put(
-      const DockerImage::Name& name,
-      const std::list<std::string>& layerIds);
+  Future<Nothing> recover();
 
-  Future<Option<DockerImage>> get(const DockerImage::Name& name);
+  Future<Image> put(
+      const Image::Name& name,
+      const std::vector<std::string>& layerIds);
 
-  Future<Nothing> recover();
+  Future<Option<Image>> get(const Image::Name& name);
 
   // TODO(chenlily): Implement removal of unreferenced images.
 
 private:
-  MetadataManagerProcess(const Flags& flags);
-
   // Write out metadata manager state to persistent store.
   Try<Nothing> persist();
 
   const Flags flags;
 
   // This is a lookup table for images that are stored in memory. It is keyed
-  // by the name of the DockerImage.
-  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
-  hashmap<std::string, DockerImage> storedImages;
+  // by the name of the Image.
+  // For example, "ubuntu:14.04" -> ubuntu14:04 Image.
+  hashmap<std::string, Image> storedImages;
 };
 
 
 Try<Owned<MetadataManager>> MetadataManager::create(const Flags& flags)
 {
-  Try<Owned<MetadataManagerProcess>> process =
-    MetadataManagerProcess::create(flags);
-  if (process.isError()) {
-    return Error("Failed to create Metadata Manager: " + process.error());
-  }
-  return Owned<MetadataManager>(new MetadataManager(process.get()));
+  Owned<MetadataManagerProcess> process(new MetadataManagerProcess(flags));
+
+  return Owned<MetadataManager>(new MetadataManager(process));
 }
 
 
@@ -113,42 +107,31 @@ Future<Nothing> MetadataManager::recover()
 }
 
 
-Future<DockerImage> MetadataManager::put(
-    const DockerImage::Name& name,
-    const list<string>& layerIds)
+Future<Image> MetadataManager::put(
+    const Image::Name& name,
+    const vector<string>& layerIds)
 {
   return dispatch(
-      process.get(), &MetadataManagerProcess::put, name, layerIds);
+      process.get(),
+      &MetadataManagerProcess::put,
+      name,
+      layerIds);
 }
 
 
-Future<Option<DockerImage>> MetadataManager::get(const DockerImage::Name& name)
+Future<Option<Image>> MetadataManager::get(const Image::Name& name)
 {
   return dispatch(process.get(), &MetadataManagerProcess::get, name);
 }
 
 
-MetadataManagerProcess::MetadataManagerProcess(const Flags& flags)
-  : flags(flags) {}
-
-
-Try<Owned<MetadataManagerProcess>> MetadataManagerProcess::create(
-    const Flags& flags)
-{
-  Owned<MetadataManagerProcess> metadataManager =
-    Owned<MetadataManagerProcess>(new MetadataManagerProcess(flags));
-
-  return metadataManager;
-}
-
-
-Future<DockerImage> MetadataManagerProcess::put(
-    const DockerImage::Name& name,
-    const list<string>& layerIds)
+Future<Image> MetadataManagerProcess::put(
+    const Image::Name& name,
+    const vector<string>& layerIds)
 {
   const string imageName = stringify(name);
 
-  DockerImage dockerImage;
+  Image dockerImage;
   dockerImage.mutable_name()->CopyFrom(name);
   foreach (const string& layerId, layerIds) {
     dockerImage.add_layer_ids(layerId);
@@ -158,15 +141,15 @@ Future<DockerImage> MetadataManagerProcess::put(
 
   Try<Nothing> status = persist();
   if (status.isError()) {
-    return Failure("Failed to save state of Docker images" + status.error());
+    return Failure("Failed to save state of Docker images: " + status.error());
   }
 
   return dockerImage;
 }
 
 
-Future<Option<DockerImage>> MetadataManagerProcess::get(
-    const DockerImage::Name& name)
+Future<Option<Image>> MetadataManagerProcess::get(
+    const Image::Name& name)
 {
   const string imageName = stringify(name);
 
@@ -180,13 +163,13 @@ Future<Option<DockerImage>> MetadataManagerProcess::get(
 
 Try<Nothing> MetadataManagerProcess::persist()
 {
-  DockerImages images;
+  Images images;
 
-  foreachvalue (const DockerImage& image, storedImages) {
+  foreachvalue (const Image& image, storedImages) {
     images.add_images()->CopyFrom(image);
   }
 
-  Try<Nothing> status = mesos::internal::slave::state::checkpoint(
+  Try<Nothing> status = state::checkpoint(
       paths::getStoredImagesPath(flags.docker_store_dir), images);
   if (status.isError()) {
     return Error("Failed to perform checkpoint: " + status.error());
@@ -200,21 +183,19 @@ Future<Nothing> MetadataManagerProcess::recover()
 {
   string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
 
-  storedImages.clear();
   if (!os::exists(storedImagesPath)) {
     LOG(INFO) << "No images to load from disk. Docker provisioner image "
-              << "storage path: " << storedImagesPath << " does not exist.";
+              << "storage path '" << storedImagesPath << "' does not exist";
     return Nothing();
   }
 
-  Result<DockerImages> images =
-    ::protobuf::read<DockerImages>(storedImagesPath);
+  Result<Images> images = ::protobuf::read<Images>(storedImagesPath);
   if (images.isError()) {
     return Failure("Failed to read protobuf for Docker provisioner image: " +
                    images.error());
   }
 
-  foreach (const DockerImage image, images.get().images()) {
+  foreach (const Image image, images.get().images()) {
     vector<string> missingLayerIds;
     foreach (const string layerId, image.layer_ids()) {
       const string rootfsPath =
@@ -226,21 +207,21 @@ Future<Nothing> MetadataManagerProcess::recover()
     }
 
     if (!missingLayerIds.empty()) {
-      LOG(WARNING) << "Skipped loading image: " << stringify(image.name())
-                   << " due to missing layers: " << stringify(missingLayerIds);
+      LOG(WARNING) << "Skipped loading image  '" << stringify(image.name())
+                   << "' due to missing layers: " << stringify(missingLayerIds);
       continue;
     }
 
     const string imageName = stringify(image.name());
     if (storedImages.contains(imageName)) {
-      LOG(WARNING) << "Found duplicate image in recovery for image name '" << imageName
-                   << "'";
+      LOG(WARNING) << "Found duplicate image in recovery for image name '"
+                   << imageName << "'";
     } else {
       storedImages[imageName] = image;
     }
   }
 
-  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
+  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images";
 
   return Nothing();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/metadata_manager.hpp b/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
index 647a478..885080d 100644
--- a/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
+++ b/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_METADATA_MANAGER_HPP__
-#define __MESOS_DOCKER_METADATA_MANAGER_HPP__
+#ifndef __PROVISIONER_DOCKER_METADATA_MANAGER_HPP__
+#define __PROVISIONER_DOCKER_METADATA_MANAGER_HPP__
 
 #include <list>
 #include <string>
@@ -49,7 +49,7 @@ class MetadataManagerProcess;
 /**
  * The MetadataManager tracks the Docker images cached by the
  * provisioner that are stored on disk. It keeps track of the layers
- * that Docker images are composed of and recovers DockerImage objects
+ * that Docker images are composed of and recovers Image objects
  * upon initialization by checking for dependent layers stored on disk.
  * Currently, image layers are stored indefinitely, with no garbage
  * collection of unreferenced image layers.
@@ -59,10 +59,15 @@ class MetadataManager
 public:
   static Try<process::Owned<MetadataManager>> create(const Flags& flags);
 
-   ~MetadataManager();
+  ~MetadataManager();
 
   /**
-   * Create a Image, put it in metadata manager and persist the reference
+   * Recover all stored Image and its layer references.
+   */
+  process::Future<Nothing> recover();
+
+  /**
+   * Create an Image, put it in metadata manager and persist the reference
    * store state to disk.
    *
    * @param name     the name of the Docker image to place in the reference
@@ -71,22 +76,17 @@ public:
    *                 order where the root layer's id (no parent layer) is first
    *                 and the leaf layer's id is last.
    */
-  process::Future<DockerImage> put(
-      const DockerImage::Name& name,
-      const std::list<std::string>& layerIds);
+  process::Future<Image> put(
+      const Image::Name& name,
+      const std::vector<std::string>& layerIds);
 
   /**
-   * Retrieve DockerImage based on image name if it is among the DockerImages
+   * Retrieve Image based on image name if it is among the Images
    * stored in memory.
    *
    * @param name  the name of the Docker image to retrieve
    */
-  process::Future<Option<DockerImage>> get(const DockerImage::Name& name);
-
-  /**
-   * Recover all stored DockerImage and its layer references.
-   */
-  process::Future<Nothing> recover();
+  process::Future<Option<Image>> get(const Image::Name& name);
 
 private:
   explicit MetadataManager(process::Owned<MetadataManagerProcess> process);
@@ -103,4 +103,4 @@ private:
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_METADATA_MANAGER_HPP__
+#endif // __PROVISIONER_DOCKER_METADATA_MANAGER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/paths.cpp b/src/slave/containerizer/provisioner/docker/paths.cpp
index 81a2176..5733fb7 100644
--- a/src/slave/containerizer/provisioner/docker/paths.cpp
+++ b/src/slave/containerizer/provisioner/docker/paths.cpp
@@ -33,51 +33,60 @@ string getStagingDir(const string& storeDir)
   return path::join(storeDir, "staging");
 }
 
-string getTempStaging(const string& storeDir)
+
+string getStagingTempDir(const string& storeDir)
 {
   return path::join(getStagingDir(storeDir), "XXXXXX");
 }
 
-string getLocalImageTarPath(
+
+string getImageArchiveTarPath(
     const string& discoveryDir,
     const string& name)
 {
   return path::join(discoveryDir, name + ".tar");
 }
 
-string getLocalImageRepositoriesPath(const string& staging)
+
+string getImageArchiveRepositoriesPath(const string& archivePath)
 {
-  return path::join(staging, "repositories");
+  return path::join(archivePath, "repositories");
 }
 
-std::string getLocalImageLayerPath(
-    const string& staging,
+
+std::string getImageArchiveLayerPath(
+    const string& archivePath,
     const string& layerId)
 {
-  return path::join(staging, layerId);
+  return path::join(archivePath, layerId);
 }
 
-string getLocalImageLayerManifestPath(
-    const string& staging,
+
+string getImageArchiveLayerManifestPath(
+    const string& archivePath,
     const string& layerId)
 {
-  return path::join(getLocalImageLayerPath(staging, layerId), "json");
+  return path::join(getImageArchiveLayerPath(archivePath, layerId), "json");
 }
 
-string getLocalImageLayerTarPath(
-  const string& staging,
+
+string getImageArchiveLayerTarPath(
+  const string& archivePath,
   const string& layerId)
 {
-  return path::join(getLocalImageLayerPath(staging, layerId), "layer.tar");
+  return path::join(
+      getImageArchiveLayerPath(archivePath, layerId), "layer.tar");
 }
 
-string getLocalImageLayerRootfsPath(
-    const string& staging,
+
+string getImageArchiveLayerRootfsPath(
+    const string& archivePath,
     const string& layerId)
 {
-  return path::join(getLocalImageLayerPath(staging, layerId), "rootfs");
+  return path::join(getImageArchiveLayerPath(archivePath, layerId), "rootfs");
 }
 
+
 string getImageLayerPath(
     const string& storeDir,
     const string& layerId)
@@ -85,6 +94,7 @@ string getImageLayerPath(
   return path::join(storeDir, "layers", layerId);
 }
 
+
 string getImageLayerRootfsPath(
     const string& storeDir,
     const string& layerId)
@@ -92,6 +102,7 @@ string getImageLayerRootfsPath(
   return path::join(getImageLayerPath(storeDir, layerId), "rootfs");
 }
 
+
 string getStoredImagesPath(const string& storeDir)
 {
   return path::join(storeDir, "storedImages");

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/paths.hpp b/src/slave/containerizer/provisioner/docker/paths.hpp
index 02f129f..18beb2e 100644
--- a/src/slave/containerizer/provisioner/docker/paths.hpp
+++ b/src/slave/containerizer/provisioner/docker/paths.hpp
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_PATHS_HPP__
-#define __MESOS_DOCKER_PATHS_HPP__
+#ifndef __PROVISIONER_DOCKER_PATHS_HPP__
+#define __PROVISIONER_DOCKER_PATHS_HPP__
 
 #include <list>
 #include <string>
@@ -33,48 +33,60 @@ namespace paths {
 /**
  * The Docker store file system layout is as follows:
  * Image store dir ('--docker_store_dir' slave flag)
- *    |--staging (contains temp directories for downloads and extract)
+ *    |--staging
+ *       |-- <temp_dir_archive>
+ *           |-- <layer_id>
+ *               |-- rootfs
  *    |--layers
  *       |--<layer_id>
  *           |--rootfs
- *    |--rootfses
  *    |--storedImages (file holding on cached images)
  */
 
 std::string getStagingDir(const std::string& storeDir);
 
-std::string getTempStaging(const std::string& storeDir);
 
-std::string getLocalImageTarPath(
+std::string getStagingTempDir(const std::string& storeDir);
+
+
+std::string getImageArchiveTarPath(
     const std::string& discoveryDir,
     const std::string& name);
 
-std::string getLocalImageRepositoriesPath(const std::string& staging);
 
-std::string getLocalImageLayerPath(
-    const std::string& staging,
+std::string getImageArchiveRepositoriesPath(const std::string& archivePath);
+
+
+std::string getImageArchiveLayerPath(
+    const std::string& archivePath,
     const std::string& layerId);
 
-std::string getLocalImageLayerManifestPath(
-    const std::string& staging,
+
+std::string getImageArchiveLayerManifestPath(
+    const std::string& archivePath,
     const std::string& layerId);
 
-std::string getLocalImageLayerTarPath(
-  const std::string& staging,
+
+std::string getImageArchiveLayerTarPath(
+  const std::string& archivePath,
   const std::string& layerId);
 
-std::string getLocalImageLayerRootfsPath(
-  const std::string& staging,
+
+std::string getImageArchiveLayerRootfsPath(
+  const std::string& archivePath,
   const std::string& layerId);
 
+
 std::string getImageLayerPath(
     const std::string& storeDir,
     const std::string& layerId);
 
+
 std::string getImageLayerRootfsPath(
     const std::string& storeDir,
     const std::string& layerId);
 
+
 std::string getStoredImagesPath(const std::string& storeDir);
 
 } // namespace paths {
@@ -83,4 +95,4 @@ std::string getStoredImagesPath(const std::string& storeDir);
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_PATHS_HPP__
+#endif // __PROVISIONER_DOCKER_PATHS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/puller.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/puller.cpp b/src/slave/containerizer/provisioner/docker/puller.cpp
new file mode 100644
index 0000000..cb05324
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/puller.cpp
@@ -0,0 +1,46 @@
+/**
+ * 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/provisioner/docker/puller.hpp"
+
+#include "slave/containerizer/provisioner/docker/local_puller.hpp"
+
+using std::string;
+
+using process::Owned;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+Try<Owned<Puller>> Puller::create(const Flags& flags)
+{
+  const string puller = flags.docker_puller;
+
+  if (puller == "local") {
+    return Owned<Puller>(new LocalPuller(flags));
+  }
+
+  return Error("Unknown or unsupported docker puller: " + puller);
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/puller.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/puller.hpp b/src/slave/containerizer/provisioner/docker/puller.hpp
new file mode 100644
index 0000000..105b4e7
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/puller.hpp
@@ -0,0 +1,68 @@
+/**
+ * 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 __PROVISIONER_DOCKER_PULLER_HPP__
+#define __PROVISIONER_DOCKER_PULLER_HPP__
+
+#include <list>
+#include <utility>
+
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+
+#include "slave/containerizer/provisioner/docker/message.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+class Puller
+{
+public:
+  static Try<process::Owned<Puller>> create(const Flags& flags);
+
+  virtual ~Puller() {}
+
+  /**
+   * Pull a Docker image layers into the specified directory, and
+   * return the list of layer ids in that image in the right
+   * dependency order, and also return the directory where
+   * the puller puts its changeset.
+   *
+   * @param name The name of the image.
+   * @param directory The target directory to store the layers.
+   * @return list of layers maped to its local directory ordered by its
+   *         dependency.
+   */
+  virtual process::Future<std::list<std::pair<std::string, std::string>>> pull(
+      const docker::Image::Name& name,
+      const std::string& directory) = 0;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+
+#endif // __PROVISIONER_DOCKER_PULLER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/store.cpp b/src/slave/containerizer/provisioner/docker/store.cpp
index 51f4d6b..cbb6768 100644
--- a/src/slave/containerizer/provisioner/docker/store.cpp
+++ b/src/slave/containerizer/provisioner/docker/store.cpp
@@ -16,29 +16,260 @@
  * limitations under the License.
  */
 
-#include "slave/containerizer/provisioner/docker/local_store.hpp"
 #include "slave/containerizer/provisioner/docker/store.hpp"
 
-using process::Owned;
+#include <list>
+#include <vector>
 
+#include <glog/logging.h>
+
+#include <stout/json.hpp>
+#include <stout/os.hpp>
+#include <stout/result.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/subprocess.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
+#include "slave/containerizer/provisioner/docker/paths.hpp"
+#include "slave/containerizer/provisioner/docker/puller.hpp"
+
+#include "slave/flags.hpp"
+
+using namespace process;
+
+using std::list;
+using std::pair;
 using std::string;
+using std::vector;
 
 namespace mesos {
 namespace internal {
 namespace slave {
 namespace docker {
 
+class StoreProcess : public Process<StoreProcess>
+{
+public:
+  StoreProcess(
+      const Flags& _flags,
+      const Owned<MetadataManager>& _metadataManager,
+      const Owned<Puller>& _puller)
+    : flags(_flags), metadataManager(_metadataManager), puller(_puller) {}
+
+  ~StoreProcess() {}
+
+  Future<Nothing> recover();
+
+  Future<vector<string>> get(const mesos::Image& image);
+
+private:
+  Future<Image> _get(
+      const Image::Name& name,
+      const Option<Image>& image);
+
+  Future<vector<string>> __get(const Image& image);
+
+  Future<vector<string>> moveLayers(
+      const std::string& staging,
+      const std::list<pair<string, string>>& layerPaths);
+
+  Future<Image> storeImage(
+      const Image::Name& name,
+      const std::vector<std::string>& layerIds);
+
+  Future<Nothing> moveLayer(const pair<string, string>& layerPath);
+
+  const Flags flags;
+  Owned<MetadataManager> metadataManager;
+  Owned<Puller> puller;
+};
+
+
 Try<Owned<slave::Store>> Store::create(const Flags& flags)
 {
-  hashmap<string, Try<Owned<slave::Store>>(*)(const Flags&)> creators{
-    {"local", &LocalStore::create}
-  };
+  Try<Owned<Puller>> puller = Puller::create(flags);
+  if (puller.isError()) {
+    return Error("Failed to create Docker puller: " + puller.error());
+  }
+
+  if (!os::exists(flags.docker_store_dir)) {
+    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store directory: " + mkdir.error());
+    }
+  }
+
+  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
+    Try<Nothing> mkdir =
+      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store staging directory: " +
+                   mkdir.error());
+    }
+  }
+
+  Try<Owned<MetadataManager>> metadataManager = MetadataManager::create(flags);
+  if (metadataManager.isError()) {
+    return Error(metadataManager.error());
+  }
+
+  Owned<StoreProcess> process(
+      new StoreProcess(flags, metadataManager.get(), puller.get()));
+
+  return Owned<slave::Store>(new Store(process));
+}
+
+
+Store::Store(const Owned<StoreProcess>& _process) : process(_process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+Store::~Store()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<Nothing> Store::recover()
+{
+  return dispatch(process.get(), &StoreProcess::recover);
+}
+
+
+Future<vector<string>> Store::get(const mesos::Image& image)
+{
+  return dispatch(process.get(), &StoreProcess::get, image);
+}
+
+
+Future<vector<string>> StoreProcess::get(const mesos::Image& image)
+{
+  if (image.type() != mesos::Image::DOCKER) {
+    return Failure("Docker provisioner store only supports Docker images");
+  }
+
+  Try<Image::Name> imageName = parseName(image.docker().name());
+  if (imageName.isError()) {
+    return Failure("Unable to parse docker image name: " + imageName.error());
+  }
+
+  return metadataManager->get(imageName.get())
+    .then(defer(self(), &Self::_get, imageName.get(), lambda::_1))
+    .then(defer(self(), &Self::__get, lambda::_1));
+}
+
+
+Future<Image> StoreProcess::_get(
+    const Image::Name& name,
+    const Option<Image>& image)
+{
+  if (image.isSome()) {
+    return image.get();
+  }
+
+  Try<string> staging =
+    os::mkdtemp(paths::getStagingTempDir(flags.docker_store_dir));
+  if (staging.isError()) {
+    return Failure("Failed to create a staging directory");
+  }
+
+  return puller->pull(name, staging.get())
+    .then(defer(self(), &Self::moveLayers, staging.get(), lambda::_1))
+    .then(defer(self(), &Self::storeImage, name, lambda::_1))
+    .onAny([staging]() {
+      Try<Nothing> rmdir = os::rmdir(staging.get());
+      if (rmdir.isError()) {
+        LOG(WARNING) << "Failed to remove staging directory: " << rmdir.error();
+      }
+    });
+}
+
+
+Future<vector<string>> StoreProcess::__get(const Image& image)
+{
+  vector<string> layerDirectories;
+  foreach (const string& layer, image.layer_ids()) {
+    layerDirectories.push_back(
+        paths::getImageLayerRootfsPath(
+            flags.docker_store_dir, layer));
+  }
+
+  return layerDirectories;
+}
+
+
+Future<Nothing> StoreProcess::recover()
+{
+  return metadataManager->recover();
+}
+
+
+Future<vector<string>> StoreProcess::moveLayers(
+    const string& staging,
+    const list<pair<string, string>>& layerPaths)
+{
+  list<Future<Nothing>> futures;
+  foreach (const auto& layerPath, layerPaths) {
+    futures.push_back(moveLayer(layerPath));
+  }
+
+  return collect(futures)
+    .then([layerPaths]() {
+        vector<string> layerIds;
+        foreach (const auto& layerPath, layerPaths) {
+          layerIds.push_back(layerPath.first);
+        }
+
+        return layerIds;
+    });
+}
+
+
+Future<Image> StoreProcess::storeImage(
+    const Image::Name& name,
+    const vector<string>& layerIds)
+{
+  return metadataManager->put(name, layerIds);
+}
+
+
+Future<Nothing> StoreProcess::moveLayer(const pair<string, string>& layerPath)
+{
+  if (!os::exists(layerPath.second)) {
+    return Failure("Unable to find layer '" + layerPath.first + "' in '" +
+                   layerPath.second + "'");
+  }
+
+  const string imageLayerPath =
+    paths::getImageLayerPath(flags.docker_store_dir, layerPath.first);
+
+  if (!os::exists(imageLayerPath)) {
+    Try<Nothing> mkdir = os::mkdir(imageLayerPath);
+    if (mkdir.isError()) {
+      return Failure("Failed to create layer path in store for id '" +
+                     layerPath.first + "': " + mkdir.error());
+    }
+  }
+
+  Try<Nothing> status = os::rename(
+      layerPath.second,
+      paths::getImageLayerRootfsPath(
+          flags.docker_store_dir, layerPath.first));
 
-  if (!creators.contains(flags.docker_store_discovery)) {
-    return Error("Unknown Docker store: " + flags.docker_store_discovery);
+  if (status.isError()) {
+    return Failure("Failed to move layer '" + layerPath.first +
+                   "' to store directory: " + status.error());
   }
 
-  return creators[flags.docker_store_discovery](flags);
+  return Nothing();
 }
 
 } // namespace docker {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/store.hpp b/src/slave/containerizer/provisioner/docker/store.hpp
index ae06706..b5e6c87 100644
--- a/src/slave/containerizer/provisioner/docker/store.hpp
+++ b/src/slave/containerizer/provisioner/docker/store.hpp
@@ -16,24 +16,17 @@
  * limitations under the License.
  */
 
-#ifndef __MESOS_DOCKER_STORE_HPP__
-#define __MESOS_DOCKER_STORE_HPP__
+#ifndef __PROVISIONER_DOCKER_STORE_HPP__
+#define __PROVISIONER_DOCKER_STORE_HPP__
 
 #include <string>
 
-#include <stout/hashmap.hpp>
-#include <stout/nothing.hpp>
-#include <stout/option.hpp>
 #include <stout/try.hpp>
 
 #include <process/future.hpp>
 
-#include "slave/containerizer/fetcher.hpp"
-
 #include "slave/containerizer/provisioner/store.hpp"
 
-#include "slave/containerizer/provisioner/docker/message.hpp"
-
 #include "slave/flags.hpp"
 
 namespace mesos {
@@ -41,13 +34,30 @@ namespace internal {
 namespace slave {
 namespace docker {
 
+// Forward Declarations.
+class Puller;
+class StoreProcess;
+
+
 // Store fetches the Docker images and stores them on disk.
-// TODO(tnachen): Make this store the only docker store
-// implementation, and move local and remote to different pullers.
-class Store
+class Store : public slave::Store
 {
 public:
   static Try<process::Owned<slave::Store>> create(const Flags& flags);
+
+  ~Store();
+
+  process::Future<Nothing> recover();
+
+  process::Future<std::vector<std::string>> get(const mesos::Image& image);
+
+private:
+  explicit Store(const process::Owned<StoreProcess>& _process);
+
+  Store& operator=(const Store&) = delete; // Not assignable.
+  Store(const Store&) = delete; // Not copyable.
+
+  process::Owned<StoreProcess> process;
 };
 
 } // namespace docker {
@@ -55,4 +65,4 @@ public:
 } // namespace internal {
 } // namespace mesos {
 
-#endif // __MESOS_DOCKER_STORE_HPP__
+#endif // __PROVISIONER_DOCKER_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/containerizer/provisioner/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/store.cpp b/src/slave/containerizer/provisioner/store.cpp
index 35d1199..a8bc302 100644
--- a/src/slave/containerizer/provisioner/store.cpp
+++ b/src/slave/containerizer/provisioner/store.cpp
@@ -28,6 +28,8 @@
 
 #include "slave/containerizer/provisioner/appc/store.hpp"
 
+#include "slave/containerizer/provisioner/docker/store.hpp"
+
 using namespace process;
 
 using std::string;
@@ -44,6 +46,7 @@ Try<hashmap<Image::Type, Owned<Store>>> Store::create(const Flags& flags)
 
   hashmap<Image::Type, Try<Owned<Store>>(*)(const Flags&)> creators;
   creators.put(Image::APPC, &appc::Store::create);
+  creators.put(Image::DOCKER, &docker::Store::create);
 
   hashmap<Image::Type, Owned<Store>> stores;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index bf26b98..029aa1e 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -93,21 +93,21 @@ mesos::internal::slave::Flags::Flags()
       "Directory the appc provisioner will store images in.",
       "/tmp/mesos/store/appc");
 
+  add(&Flags::docker_local_archives_dir,
+      "docker_local_archives_dir",
+      "Directory for docker local puller to look in for image archives",
+      "/tmp/mesos/images/docker");
+
+  add(&Flags::docker_puller,
+      "docker_puller",
+      "Strategy for docker puller to fetch images",
+      "local");
+
   add(&Flags::docker_store_dir,
       "docker_store_dir",
       "Directory the docker provisioner will store images in",
       "/tmp/mesos/store/docker");
 
-  add(&Flags::docker_store_discovery,
-      "docker_store_discovery",
-      "Strategy for docker store to fetch images",
-      "local");
-
-  add(&Flags::docker_store_discovery_local_dir,
-      "docker_store_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/9fb62cec/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 1ce123c..f76f0f6 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -53,10 +53,9 @@ public:
   std::string image_provisioner_backend;
   std::string appc_store_dir;
 
-  std::string docker_provisioner_backend;
+  std::string docker_local_archives_dir;
+  std::string docker_puller;
   std::string docker_store_dir;
-  std::string docker_store_discovery;
-  std::string docker_store_discovery_local_dir;
 
   std::string default_role;
   Option<std::string> attributes;

http://git-wip-us.apache.org/repos/asf/mesos/blob/9fb62cec/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index 3a817c0..d895eb9 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -38,6 +38,7 @@
 #include <process/ssl/gtest.hpp>
 
 #include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
+#include "slave/containerizer/provisioner/docker/paths.hpp"
 #include "slave/containerizer/provisioner/docker/registry_client.hpp"
 #include "slave/containerizer/provisioner/docker/store.hpp"
 #include "slave/containerizer/provisioner/docker/token_manager.hpp"
@@ -50,8 +51,6 @@ using std::map;
 using std::string;
 using std::vector;
 
-using namespace mesos::internal::slave::docker::registry;
-
 using process::Clock;
 using process::Future;
 using process::Owned;
@@ -61,6 +60,8 @@ using process::network::Socket;
 using namespace process;
 using namespace mesos::internal::slave;
 using namespace mesos::internal::slave::docker;
+using namespace mesos::internal::slave::docker::paths;
+using namespace mesos::internal::slave::docker::registry;
 
 using ManifestResponse = RegistryClient::ManifestResponse;
 
@@ -68,7 +69,6 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-
 /**
  * Provides token operations and defaults.
  */
@@ -695,9 +695,11 @@ TEST_F(RegistryClientTest, BadRequest)
   ASSERT_TRUE(strings::contains(resultFuture.failure(), "Error2"));
 }
 
+
 #endif // USE_SSL_SOCKET
 
-class DockerProvisionerLocalStoreTest : public TemporaryDirectoryTest
+
+class ProvisionerDockerLocalStoreTest : public TemporaryDirectoryTest
 {
 public:
   void verifyLocalDockerImage(
@@ -707,19 +709,21 @@ public:
     string layersPath = path::join(flags.docker_store_dir, "layers");
 
     // Verify contents of the image in store directory.
-    EXPECT_TRUE(os::exists(path::join(layersPath, "123", "rootfs")));
-    EXPECT_TRUE(os::exists(path::join(layersPath, "456", "rootfs")));
+    string layerPath1 = getImageLayerRootfsPath(flags.docker_store_dir, "123");
+    string layerPath2 = getImageLayerRootfsPath(flags.docker_store_dir, "456");
+    EXPECT_TRUE(os::exists(layerPath1));
+    EXPECT_TRUE(os::exists(layerPath2));
     EXPECT_SOME_EQ(
         "foo 123",
-        os::read(path::join(layersPath, "123", "rootfs" , "temp")));
+        os::read(path::join(layerPath1 , "temp")));
     EXPECT_SOME_EQ(
         "bar 456",
-        os::read(path::join(layersPath, "456", "rootfs", "temp")));
+        os::read(path::join(layerPath2, "temp")));
 
     // Verify the Docker Image provided.
     vector<string> expectedLayers;
-    expectedLayers.push_back("123");
-    expectedLayers.push_back("456");
+    expectedLayers.push_back(layerPath1);
+    expectedLayers.push_back(layerPath2);
     EXPECT_EQ(expectedLayers, layers);
   }
 
@@ -734,19 +738,19 @@ protected:
     ASSERT_SOME(os::mkdir(image));
 
     JSON::Value repositories = JSON::parse(
-      "{"
-      "  \"abc\": {"
-      "    \"latest\": \"456\""
-      "  }"
-      "}").get();
+        "{"
+        "  \"abc\": {"
+        "    \"latest\": \"456\""
+        "  }"
+        "}").get();
     ASSERT_SOME(
         os::write(path::join(image, "repositories"), stringify(repositories)));
 
     ASSERT_SOME(os::mkdir(path::join(image, "123")));
     JSON::Value manifest123 = JSON::parse(
-      "{"
-      "  \"parent\": \"\""
-      "}").get();
+        "{"
+        "  \"parent\": \"\""
+        "}").get();
     ASSERT_SOME(os::write(
         path::join(image, "123", "json"), stringify(manifest123)));
     ASSERT_SOME(os::mkdir(path::join(image, "123", "layer")));
@@ -762,9 +766,9 @@ protected:
 
     ASSERT_SOME(os::mkdir(path::join(image, "456")));
     JSON::Value manifest456 = JSON::parse(
-      "{"
-      "  \"parent\": \"123\""
-      "}").get();
+        "{"
+        "  \"parent\": \"123\""
+        "}").get();
     ASSERT_SOME(
         os::write(path::join(image, "456", "json"), stringify(manifest456)));
     ASSERT_SOME(os::mkdir(path::join(image, "456", "layer")));
@@ -783,10 +787,11 @@ protected:
   }
 };
 
+
 // This test verifies that a locally stored Docker image in the form of a
 // tar achive created from a 'docker save' command can be unpacked and
 // stored in the proper locations accessible to the Docker provisioner.
-TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
+TEST_F(ProvisionerDockerLocalStoreTest, LocalStoreTestWithTar)
 {
   string imageDir = path::join(os::getcwd(), "images");
   string image = path::join(imageDir, "abc:latest");
@@ -794,12 +799,11 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
   ASSERT_SOME(os::mkdir(image));
 
   slave::Flags flags;
-  flags.docker_store_discovery = "local";
+  flags.docker_puller = "local";
   flags.docker_store_dir = path::join(os::getcwd(), "store");
-  flags.docker_store_discovery_local_dir = imageDir;
+  flags.docker_local_archives_dir = imageDir;
 
-  Try<Owned<slave::Store>> store =
-    mesos::internal::slave::docker::Store::create(flags);
+  Try<Owned<slave::Store>> store = slave::docker::Store::create(flags);
   ASSERT_SOME(store);
 
   string sandbox = path::join(os::getcwd(), "sandbox");
@@ -815,17 +819,17 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
   verifyLocalDockerImage(flags, layers.get());
 }
 
-// This tests the ability of the reference store to recover the images it has
+
+// This tests the ability of the metadata manger to recover the images it has
 // already stored on disk when it is initialized.
-TEST_F(DockerProvisionerLocalStoreTest, MetadataManagerInitialization)
+TEST_F(ProvisionerDockerLocalStoreTest, MetadataManagerInitialization)
 {
   slave::Flags flags;
-  flags.docker_store_discovery = "local";
+  flags.docker_puller = "local";
   flags.docker_store_dir = path::join(os::getcwd(), "store");
-  flags.docker_store_discovery_local_dir = path::join(os::getcwd(), "images");
+  flags.docker_local_archives_dir = path::join(os::getcwd(), "images");
 
-  Try<Owned<slave::Store>> store =
-    mesos::internal::slave::docker::Store::create(flags);
+  Try<Owned<slave::Store>> store = slave::docker::Store::create(flags);
   ASSERT_SOME(store);
 
   string sandbox = path::join(os::getcwd(), "sandbox");
@@ -838,11 +842,13 @@ TEST_F(DockerProvisionerLocalStoreTest, MetadataManagerInitialization)
   Future<vector<string>> layers = store.get()->get(image);
   AWAIT_READY(layers);
 
-  // Store is deleted and recreated. Reference Store is initialized upon
+  // Store is deleted and recreated. Metadata Manager is initialized upon
   // creation of the store.
   store.get().reset();
-  store = mesos::internal::slave::docker::Store::create(flags);
+  store = slave::docker::Store::create(flags);
   ASSERT_SOME(store);
+  Future<Nothing> recover = store.get()->recover();
+  AWAIT_READY(recover);
 
   layers = store.get()->get(image);
   AWAIT_READY(layers);


[14/17] mesos git commit: Fix docker provisioner tests.

Posted by tn...@apache.org.
Fix docker provisioner tests.


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

Branch: refs/heads/master
Commit: f392f752fd796939184229b4d819fb1e58423290
Parents: fb2df96
Author: Timothy Chen <tn...@gmail.com>
Authored: Tue Sep 8 19:50:11 2015 +0000
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/messages/docker_provisioner.hpp             |  6 ++---
 .../containerizer/provisioner/provisioner.hpp   |  2 +-
 src/slave/containerizer/provisioners/docker.hpp |  6 ++---
 .../provisioners/docker/local_store.cpp         |  2 +-
 .../containerizer/provisioner_docker_tests.cpp  | 28 ++++++++------------
 5 files changed, 19 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f392f752/src/messages/docker_provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/messages/docker_provisioner.hpp b/src/messages/docker_provisioner.hpp
index b076fdb..684bd46 100644
--- a/src/messages/docker_provisioner.hpp
+++ b/src/messages/docker_provisioner.hpp
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-#ifndef __DOCKER_PROVISIONER_HPP__
-#define __DOCKER_PROVISIONER_HPP__
+#ifndef __MESSAGES_DOCKER_PROVISIONER_HPP__
+#define __MESSAGES_DOCKER_PROVISIONER_HPP__
 
 #include "messages/docker_provisioner.pb.h"
 
-#endif // __DOCKER_PROVISIONER_HPP__
+#endif // __MESSAGES_DOCKER_PROVISIONER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f392f752/src/slave/containerizer/provisioner/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/provisioner.hpp b/src/slave/containerizer/provisioner/provisioner.hpp
index 912fc5a..d42822b 100644
--- a/src/slave/containerizer/provisioner/provisioner.hpp
+++ b/src/slave/containerizer/provisioner/provisioner.hpp
@@ -67,7 +67,7 @@ public:
   // image and return the absolute path to the root filesystem.
   virtual process::Future<std::string> provision(
       const ContainerID& containerId,
-      const Image& image);
+      const Image& image) = 0;
 
   // Destroy a previously provisioned root filesystem. Assumes that
   // all references (e.g., mounts, open files) to the provisioned

http://git-wip-us.apache.org/repos/asf/mesos/blob/f392f752/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
index 35e23c9..d3ada9d 100644
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -96,10 +96,10 @@ struct DockerImage
 // Forward declaration.
 class DockerProvisionerProcess;
 
-class DockerProvisioner : public Provisioner
+class DockerProvisioner : public mesos::internal::slave::Provisioner
 {
 public:
-  static Try<process::Owned<Provisioner>> create(
+  static Try<process::Owned<mesos::internal::slave::Provisioner>> create(
       const Flags& flags,
       Fetcher* fetcher);
 
@@ -116,7 +116,7 @@ public:
   virtual process::Future<bool> destroy(const ContainerID& containerId);
 
 private:
-  explicit DockerProvisioner(process::Owned<DockerProvisionerProcess> process);
+  explicit DockerProvisioner(process::Owned<DockerProvisionerProcess> _process);
   DockerProvisioner(const DockerProvisioner&); // Not copyable.
   DockerProvisioner& operator=(const DockerProvisioner&); // Not assignable.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f392f752/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
index c6a9efe..80b5b06 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -187,7 +187,7 @@ Future<DockerImage> LocalStoreProcess::get(const string& name)
 {
   return refStore->get(name)
     .then(defer(self(),
-                [this, &name](
+                [this, name](
                     const Option<DockerImage>& image) -> Future<DockerImage> {
       if (image.isSome()) {
         return image.get();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f392f752/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index 3a9a6ec..e1a311b 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -701,17 +701,17 @@ public:
       const slave::Flags& flags,
       const DockerImage& dockerImage)
   {
+    string layersPath = path::join(flags.docker_store_dir, "layers");
+
     // Verify contents of the image in store directory.
-    EXPECT_TRUE(
-        os::exists(path::join(flags.docker_store_dir, "123", "rootfs")));
-    EXPECT_TRUE(
-        os::exists(path::join(flags.docker_store_dir, "456", "rootfs")));
+    EXPECT_TRUE(os::exists(path::join(layersPath, "123", "rootfs")));
+    EXPECT_TRUE(os::exists(path::join(layersPath, "456", "rootfs")));
     EXPECT_SOME_EQ(
         "foo 123",
-        os::read(path::join(flags.docker_store_dir, "123", "rootfs" , "temp")));
+        os::read(path::join(layersPath, "123", "rootfs" , "temp")));
     EXPECT_SOME_EQ(
         "bar 456",
-        os::read(path::join(flags.docker_store_dir, "456", "rootfs", "temp")));
+        os::read(path::join(layersPath, "456", "rootfs", "temp")));
 
     // Verify the Docker Image provided.
     EXPECT_EQ(dockerImage.imageName, "abc");
@@ -804,15 +804,10 @@ TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
 
   string sandbox = path::join(os::getcwd(), "sandbox");
   ASSERT_SOME(os::mkdir(sandbox));
-  Future<DockerImage> dockerImage = store.get()->put("abc", sandbox);
+  Future<DockerImage> dockerImage = store.get()->get("abc");
   AWAIT_READY(dockerImage);
 
   verifyLocalDockerImage(flags, dockerImage.get());
-
-  Future<Option<DockerImage>> dockerImageOption = store.get()->get("abc");
-  AWAIT_READY(dockerImageOption);
-  ASSERT_SOME(dockerImageOption.get());
-  verifyLocalDockerImage(flags, dockerImageOption.get().get());
 }
 
 // This tests the ability of the reference store to recover the images it has
@@ -832,7 +827,7 @@ TEST_F(DockerProvisionerLocalStoreTest, ReferenceStoreInitialization)
 
   string sandbox = path::join(os::getcwd(), "sandbox");
   ASSERT_SOME(os::mkdir(sandbox));
-  Future<DockerImage> dockerImage = store.get()->put("abc", sandbox);
+  Future<DockerImage> dockerImage = store.get()->get("abc");
   AWAIT_READY(dockerImage);
 
   // Store is deleted and recreated. Reference Store is initialized upon
@@ -841,10 +836,9 @@ TEST_F(DockerProvisionerLocalStoreTest, ReferenceStoreInitialization)
   store = Store::create(flags, &fetcher);
   ASSERT_SOME(store);
 
-  Future<Option<DockerImage>> dockerImageOption = store.get()->get("abc");
-  AWAIT_READY(dockerImageOption);
-  ASSERT_SOME(dockerImageOption.get());
-  verifyLocalDockerImage(flags, dockerImageOption.get().get());
+  dockerImage = store.get()->get("abc");
+  AWAIT_READY(dockerImage);
+  verifyLocalDockerImage(flags, dockerImage.get());
 }
 
 } // namespace tests {


[09/17] mesos git commit: Refactor docker provisioner paths.

Posted by tn...@apache.org.
Refactor docker provisioner paths.


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

Branch: refs/heads/master
Commit: c5537a1331875d3e52f8b9c7e41350dfb8120fc1
Parents: f96e0df
Author: Timothy Chen <tn...@gmail.com>
Authored: Fri Sep 4 18:09:51 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 .../containerizer/provisioners/docker/paths.cpp     |  2 +-
 .../containerizer/provisioners/docker/paths.hpp     | 16 +++++++---------
 src/slave/flags.cpp                                 |  6 ------
 src/slave/flags.hpp                                 |  1 -
 4 files changed, 8 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c5537a13/src/slave/containerizer/provisioners/docker/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.cpp b/src/slave/containerizer/provisioners/docker/paths.cpp
index ca35e0a..5a54ba1 100644
--- a/src/slave/containerizer/provisioners/docker/paths.cpp
+++ b/src/slave/containerizer/provisioners/docker/paths.cpp
@@ -82,7 +82,7 @@ string getImageLayerPath(
     const string& storeDir,
     const string& layerId)
 {
-  return path::join(storeDir, layerId);
+  return path::join(storeDir, "layers", layerId);
 }
 
 string getImageLayerRootfsPath(

http://git-wip-us.apache.org/repos/asf/mesos/blob/c5537a13/src/slave/containerizer/provisioners/docker/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/paths.hpp b/src/slave/containerizer/provisioners/docker/paths.hpp
index 4aae0d0..0ad3b74 100644
--- a/src/slave/containerizer/provisioners/docker/paths.hpp
+++ b/src/slave/containerizer/provisioners/docker/paths.hpp
@@ -32,15 +32,13 @@ namespace paths {
 
 /**
  * The Docker store file system layout is as follows:
- * <root>
- * |-- Local image discovery dir ('--docker_discovery_local_dir' slave flag)
- *    |--<name>.tar
- * |
- * |-- Image store dir ('--docker_store_dir' slave flag)
- *    |--staging
- *    |--<image_id>
- *        |--rootfs
- *    |--storedImages
+ * Image store dir ('--docker_store_dir' slave flag)
+ *    |--staging (contains temp directories for downloads and extract)
+ *    |--layers
+ *       |--<layer_id>
+ *           |--rootfs
+ *    |--rootfses
+ *    |--storedImages (file holding on cached images)
  */
 
 std::string getStagingDir(const std::string& storeDir);

http://git-wip-us.apache.org/repos/asf/mesos/blob/c5537a13/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 3ce787f..b0af3d8 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -103,12 +103,6 @@ mesos::internal::slave::Flags::Flags()
       "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",

http://git-wip-us.apache.org/repos/asf/mesos/blob/c5537a13/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 7df8062..758d495 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -56,7 +56,6 @@ public:
   std::string docker_store;
   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;


[15/17] mesos git commit: Add Docker local store recover.

Posted by tn...@apache.org.
Add Docker local store recover.


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

Branch: refs/heads/master
Commit: fb2df960a805132e58624ee0d73eaa098d57d67a
Parents: 11dc584
Author: Timothy Chen <tn...@gmail.com>
Authored: Mon Sep 7 23:07:55 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   8 +-
 src/slave/containerizer/provisioners/docker.cpp | 109 ++++++----------
 src/slave/containerizer/provisioners/docker.hpp |   1 -
 .../provisioners/docker/local_store.cpp         | 123 ++++++++++---------
 .../provisioners/docker/local_store.hpp         |  21 ++--
 .../provisioners/docker/reference_store.cpp     |  28 ++---
 .../provisioners/docker/reference_store.hpp     |  11 +-
 .../containerizer/provisioners/docker/store.hpp |  17 ++-
 8 files changed, 147 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 65def70..cd8b2ca 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -529,6 +529,10 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/provisioner/appc/store.cpp			\
 	slave/containerizer/provisioner/backend.cpp			\
 	slave/containerizer/provisioner/backends/copy.cpp		\
+	slave/containerizer/provisioner/docker.cpp			\
+	slave/containerizer/provisioner/docker/local_store.cpp		\
+	slave/containerizer/provisioner/docker/paths.cpp		\
+	slave/containerizer/provisioner/docker/reference_store.cpp	\
 	slave/containerizer/provisioner/docker/registry_client.cpp	\
 	slave/containerizer/provisioner/docker/token_manager.cpp	\
 	slave/resource_estimators/noop.cpp				\
@@ -701,10 +705,6 @@ 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/local_store.cpp
-  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/paths.cpp
-  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/reference_store.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
   EXTRA_DIST += linux/fs.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
index 32e1a3b..b1f737f 100644
--- a/src/slave/containerizer/provisioners/docker.cpp
+++ b/src/slave/containerizer/provisioners/docker.cpp
@@ -30,10 +30,14 @@
 #include <process/owned.hpp>
 #include <process/sequence.hpp>
 
-#include "linux/fs.hpp"
+#include "slave/containerizer/provisioners/backend.hpp"
+#include "slave/containerizer/provisioners/paths.hpp"
 
+#include "slave/containerizer/provisioners/docker/paths.hpp"
 #include "slave/containerizer/provisioners/docker/store.hpp"
 
+#include "slave/paths.hpp"
+
 using namespace process;
 using namespace mesos::internal::slave;
 
@@ -68,20 +72,18 @@ public:
 
 private:
   DockerProvisionerProcess(
-      const Flags& flags,
-      const process::Owned<Store>& store,
-      const process::Owned<mesos::internal::slave::Backend>& backend);
+      const string& _rootDir,
+      const Flags& _flags,
+      const process::Owned<Store>& _store,
+      const hashmap<std::string, process::Owned<Backend>>& _backends);
 
   process::Future<std::string> _provision(
+      const DockerImage& image,
       const ContainerID& containerId,
-      const DockerImage& image);
-
-  process::Future<DockerImage> fetch(
-      const std::string& name,
-      const std::string& sandbox);
+      const string& rootfs);
 
+  const string& rootDir;
   const Flags flags;
-
   process::Owned<Store> store;
   hashmap<string, process::Owned<Backend>> backends;
 
@@ -92,7 +94,6 @@ private:
   };
 
   hashmap<ContainerID, Owned<Info>> infos;
-
 };
 
 
@@ -179,24 +180,17 @@ Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  string _root =
+  string rootDir =
     slave::paths::getProvisionerDir(flags.work_dir, Image::DOCKER);
 
-  Try<Nothing> mkdir = os::mkdir(_root);
-  if (mkdir.isError()) {
-    return Error("Failed to create provisioner root directory '" +
-                 _root + "': " + mkdir.error());
-  }
-
-  Result<string> root = os::realpath(_root);
-  if (root.isError()) {
-    return Error(
-        "Failed to resolve the realpath of provisioner root directory '" +
-        _root + "': " + root.error());
+  if (!os::exists(rootDir)) {
+    Try<Nothing> mkdir = os::mkdir(rootDir);
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker provisioner root directory '" +
+                   rootDir + "': " + mkdir.error());
+    }
   }
 
-  CHECK_SOME(root); // Can't be None since we just created it.
-
   hashmap<string, Owned<Backend>> backends = Backend::create(flags);
   if (backends.empty()) {
     return Error("No usable Docker provisioner backend created");
@@ -214,6 +208,7 @@ Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
 
   return Owned<DockerProvisionerProcess>(
       new DockerProvisionerProcess(
+          rootDir,
           flags,
           store.get(),
           backends));
@@ -221,10 +216,12 @@ Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
 
 
 DockerProvisionerProcess::DockerProvisionerProcess(
+    const string& _rootDir,
     const Flags& _flags,
     const Owned<Store>& _store,
     const hashmap<string, Owned<Backend>>& _backends)
-  : flags(_flags),
+  : rootDir(_rootDir),
+    flags(_flags),
     store(_store),
     backends(_backends) {}
 
@@ -252,7 +249,7 @@ Future<Nothing> DockerProvisionerProcess::recover(
   // be destroyed by the containerizer using the normal cleanup path. See
   // MESOS-2367 for details.
   Try<hashmap<ContainerID, string>> containers =
-    provisioners::paths::listContainers(root);
+    provisioners::paths::listContainers(rootDir);
 
   if (containers.isError()) {
     return Failure("Failed to list the containers managed by Docker "
@@ -265,7 +262,7 @@ Future<Nothing> DockerProvisionerProcess::recover(
       Owned<Info> info = Owned<Info>(new Info());
 
       Try<hashmap<string, hashmap<string, string>>> rootfses =
-        provisioners::paths::listContainerRootfses(root, containerId);
+        provisioners::paths::listContainerRootfses(rootDir, containerId);
 
       if (rootfses.isError()) {
         return Failure("Unable to list rootfses belonged to container '" +
@@ -289,7 +286,7 @@ Future<Nothing> DockerProvisionerProcess::recover(
 
     // Destroy (unknown) orphan container's rootfses.
     Try<hashmap<string, hashmap<string, string>>> rootfses =
-      provisioners::paths::listContainerRootfses(root, containerId);
+      provisioners::paths::listContainerRootfses(rootDir, containerId);
 
     if (rootfses.isError()) {
       return Failure("Unable to find rootfses for container '" +
@@ -340,7 +337,7 @@ Future<string> DockerProvisionerProcess::provision(
 
   string rootfsId = UUID::random().toString();
   string rootfs = provisioners::paths::getContainerRootfsDir(
-      root, containerId, flags.docker_backend, rootfsId);
+      rootDir, containerId, flags.docker_backend, rootfsId);
 
   if (!infos.contains(containerId)) {
     infos.put(containerId, Owned<Info>(new Info()));
@@ -348,63 +345,29 @@ Future<string> DockerProvisionerProcess::provision(
 
   infos[containerId]->rootfses[flags.docker_backend].put(rootfsId, rootfs);
 
-
-  return fetch(image.docker().name())
-    .then(defer(self(),
-                &Self::_provision,
-                containerId,
-                lambda::_1));
+  return store->get(image.docker().name())
+    .then(defer(self(), &Self::_provision, lambda::_1, containerId, rootfs));
 }
 
 
 Future<string> DockerProvisionerProcess::_provision(
+    const DockerImage& image,
     const ContainerID& containerId,
-    const DockerImage& image)
+    const string& rootfs)
 {
   CHECK(backends.contains(flags.docker_backend));
 
-  // 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 << "'";
+            << " to '" << rootfs << "'";
 
   vector<string> layerPaths;
   foreach (const string& layerId, image.layers) {
-    layerPaths.push_back(path::join(flags.docker_store_dir, layerId, "rootfs"));
+    layerPaths.push_back(
+        paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId));
   }
 
-
-  return backends[flags.docker_backend]->provision(layerPaths, base)
-    .then([rootfs]() -> 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)
-{
-  return store->get(name);
+  return backends[flags.docker_backend]->provision(layerPaths, rootfs)
+    .then([rootfs]() -> Future<string> { return rootfs; });
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
index 850ce85..35e23c9 100644
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -41,7 +41,6 @@
 #include <mesos/resources.hpp>
 
 #include "slave/containerizer/provisioner.hpp"
-#include "slave/containerizer/provisioners/backend.hpp"
 
 #include "slave/flags.hpp"
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
index aec7df9..c6a9efe 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -62,10 +62,13 @@ public:
 
   process::Future<DockerImage> get(const std::string& name);
 
+  process::Future<Nothing> recover();
+
 private:
   LocalStoreProcess(
-      const Flags& flags,
-      Owned<ReferenceStore> _refStore);
+      const Flags& _flags,
+      Owned<ReferenceStore> _refStore)
+    : flags(_flags), refStore(_refStore) {}
 
   process::Future<Nothing> untarImage(
       const std::string& tarPath,
@@ -73,7 +76,7 @@ private:
 
   process::Future<DockerImage> putImage(
       const std::string& name,
-      const std::string& staging)
+      const std::string& staging);
 
   Result<std::string> getParentId(
       const std::string& staging,
@@ -81,18 +84,17 @@ private:
 
   process::Future<Nothing> putLayers(
       const std::string& staging,
-      const std::list<std::string>& layers)
+      const std::list<std::string>& layers);
 
   process::Future<Nothing> putLayer(
       const std::string& staging,
-      const std::string& id)
+      const std::string& id);
 
   process::Future<Nothing> moveLayer(
       const std::string& staging,
-      const std::string& id)
+      const std::string& id);
 
   const Flags flags;
-
   process::Owned<ReferenceStore> refStore;
 };
 
@@ -117,42 +119,17 @@ Try<Owned<Store>> LocalStore::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  if (!os::exists(flags.docker_store_dir)) {
-    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker store directory: " + mkdir.error());
-    }
-  }
-
-  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
-    Try<Nothing> mkdir =
-      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker store staging directory: " +
-                   mkdir.error());
-    }
-  }
-
   Try<Owned<LocalStoreProcess>> process =
     LocalStoreProcess::create(flags, fetcher);
   if (process.isError()) {
     return Error(process.error());
   }
 
-  Try<Owned<ReferenceStore>> refStore = ReferenceStore::create(flags);
-  if (refStore.isError()) {
-    return Error(refStore);
-  }
-
-  return Owned<Store>(new LocalStore(process.get(), refStore.get()));
+  return Owned<Store>(new LocalStore(process.get()));
 }
 
 
-LocalStore::LocalStore(
-    Owned<LocalStoreProcess> process,
-    Owned<ReferenceStore> refStore)
-  : process(process),
-    _refStore(refStore)
+LocalStore::LocalStore(Owned<LocalStoreProcess> _process) : process(_process)
 {
   process::spawn(CHECK_NOTNULL(process.get()));
 }
@@ -165,49 +142,81 @@ LocalStore::~LocalStore()
 }
 
 
-Future<Option<DockerImage>> LocalStore::get(const string& name)
+Future<DockerImage> LocalStore::get(const string& name)
 {
   return dispatch(process.get(), &LocalStoreProcess::get, name);
 }
 
 
+Future<Nothing> LocalStore::recover()
+{
+  return dispatch(process.get(), &LocalStoreProcess::recover);
+}
+
+
 Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  return Owned<LocalStoreProcess>(new LocalStoreProcess(flags));
-}
+  if (!os::exists(flags.docker_store_dir)) {
+    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store directory: " + mkdir.error());
+    }
+  }
+
+  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
+    Try<Nothing> mkdir =
+      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store staging directory: " +
+                   mkdir.error());
+    }
+  }
 
+  Try<Owned<ReferenceStore>> refStore = ReferenceStore::create(flags);
+  if (refStore.isError()) {
+    return Error(refStore.error());
+  }
 
-LocalStoreProcess::LocalStoreProcess(const Flags& flags)
-  : flags(flags), refStore(ReferenceStore::create(flags).get()) {}
+  return Owned<LocalStoreProcess>(new LocalStoreProcess(flags, refStore.get()));
+}
 
 
 Future<DockerImage> LocalStoreProcess::get(const string& name)
 {
-  Option<DockerImage> image = refStore->get(name);
-  if (image.isSome()) {
-    return image.get();
-  }
+  return refStore->get(name)
+    .then(defer(self(),
+                [this, &name](
+                    const Option<DockerImage>& image) -> Future<DockerImage> {
+      if (image.isSome()) {
+        return image.get();
+      }
 
-  string tarPath =
-    paths::getLocalImageTarPath(flags.docker_discovery_local_dir, name);
-  if (!os::exists(tarPath)) {
-    return Failure("No Docker image tar archive found");
-  }
+      string tarPath =
+        paths::getLocalImageTarPath(flags.docker_discovery_local_dir, name);
+      if (!os::exists(tarPath)) {
+        return Failure("No Docker image tar archive found");
+      }
 
-  // Create a temporary staging directory.
-  Try<string> staging =
-    os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
-  if (staging.isError()) {
-    return Failure("Failed to create a staging directory");
-  }
+      // Create a temporary staging directory.
+      Try<string> staging =
+        os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
+      if (staging.isError()) {
+        return Failure("Failed to create a staging directory");
+      }
 
-  return untarImage(tarPath, staging.get())
-    .then(defer(self(), &Self::putImage, name, staging.get()));
+      return untarImage(tarPath, staging.get())
+        .then(defer(self(), &Self::putImage, name, staging.get()));
+    }));
 }
 
 
+Future<Nothing> LocalStoreProcess::recover()
+{
+  return refStore->recover();
+}
+
 Future<Nothing> LocalStoreProcess::untarImage(
     const string& tarPath,
     const string& staging)
@@ -376,7 +385,7 @@ Future<Nothing> LocalStoreProcess::putLayer(
 
   const string imageLayerPath =
     paths::getImageLayerPath(flags.docker_store_dir, id);
-  if (!os::exists()) {
+  if (!os::exists(imageLayerPath)) {
     Try<Nothing> mkdir = os::mkdir(imageLayerPath);
     if (mkdir.isError()) {
       return Failure("Failed to create Image layer directory '" +

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.hpp b/src/slave/containerizer/provisioners/docker/local_store.hpp
index 2f0c9f1..b650b5e 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.hpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.hpp
@@ -26,8 +26,17 @@ namespace internal {
 namespace slave {
 namespace docker {
 
+// Forward declaration.
 class LocalStoreProcess;
+class ReferenceStore;
 
+
+/**
+ * LocalStore assumes Docker images are stored in a local directory
+ * (configured with flags.docker_discovery_local_dir), with all the
+ * images saved as tar with the name as the image name with tag (e.g:
+ * ubuntu:14.04.tar).
+ */
 class LocalStore : public Store
 {
 public:
@@ -37,18 +46,12 @@ public:
       const Flags& flags,
       Fetcher* fetcher);
 
-  /**
-   * Put assumes the image tar archive is located in the directory specified in
-   * the slave flag docker_discovery_local_dir and is named with <name>.tar .
-   */
-  virtual process::Future<DockerImage> put(
-      const std::string& name,
-      const std::string& sandbox);
+  virtual process::Future<DockerImage> get(const std::string& name);
 
-  virtual process::Future<Option<DockerImage>> get(const std::string& name);
+  virtual process::Future<Nothing> recover();
 
 private:
-  explicit LocalStore(process::Owned<LocalStoreProcess> process);
+  explicit LocalStore(process::Owned<LocalStoreProcess> _process);
 
   LocalStore(const LocalStore&); // Not copyable.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker/reference_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.cpp b/src/slave/containerizer/provisioners/docker/reference_store.cpp
index 1567248..4b72319 100644
--- a/src/slave/containerizer/provisioners/docker/reference_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/reference_store.cpp
@@ -54,18 +54,15 @@ class ReferenceStoreProcess : public process::Process<ReferenceStoreProcess>
 public:
   ~ReferenceStoreProcess() {}
 
-  // Explicitly use 'initialize' since we are overloading below.
-  using process::ProcessBase::initialize;
-
-  void initialize();
-
   static Try<process::Owned<ReferenceStoreProcess>> create(const Flags& flags);
 
-  process::Future<DockerImage> put(
+  Future<DockerImage> put(
       const std::string& name,
       const std::list<std::string>& layers);
 
-  process::Future<Option<DockerImage>> get(const std::string& name);
+  Future<Option<DockerImage>> get(const std::string& name);
+
+  Future<Nothing> recover();
 
   // TODO(chenlily): Implement removal of unreferenced images.
 
@@ -109,9 +106,9 @@ ReferenceStore::~ReferenceStore()
 }
 
 
-void ReferenceStore::initialize()
+Future<Nothing> ReferenceStore::recover()
 {
-  process::dispatch(process.get(), &ReferenceStoreProcess::initialize);
+  return process::dispatch(process.get(), &ReferenceStoreProcess::recover);
 }
 
 
@@ -194,7 +191,7 @@ Try<Nothing> ReferenceStoreProcess::persist()
 }
 
 
-void ReferenceStoreProcess::initialize()
+Future<Nothing> ReferenceStoreProcess::recover()
 {
   string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
 
@@ -202,15 +199,14 @@ void ReferenceStoreProcess::initialize()
   if (!os::exists(storedImagesPath)) {
     LOG(INFO) << "No images to load from disk. Docker provisioner image "
               << "storage path: " << storedImagesPath << " does not exist.";
-    return;
+    return Nothing();
   }
 
   Result<DockerProvisionerImages> images =
     ::protobuf::read<DockerProvisionerImages>(storedImagesPath);
   if (images.isError()) {
-    LOG(ERROR) << "Failed to read protobuf for Docker provisioner image: "
-               << images.error();
-    return;
+    return Failure("Failed to read protobuf for Docker provisioner image: " +
+                   images.error());
   }
 
   for (int i = 0; i < images.get().images_size(); i++) {
@@ -224,7 +220,7 @@ void ReferenceStoreProcess::initialize()
       layers.push_back(layerId);
 
       if (!os::exists(
-              paths::getImageLayerPath(flags.docker_store_dir, layerId))) {
+              paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId))) {
         missingLayers.push_back(layerId);
       }
     }
@@ -243,6 +239,8 @@ void ReferenceStoreProcess::initialize()
   }
 
   LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
+
+  return Nothing();
 }
 
 } // namespace docker {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker/reference_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.hpp b/src/slave/containerizer/provisioners/docker/reference_store.hpp
index 66b7573..be652ae 100644
--- a/src/slave/containerizer/provisioners/docker/reference_store.hpp
+++ b/src/slave/containerizer/provisioners/docker/reference_store.hpp
@@ -56,12 +56,6 @@ class ReferenceStore
 public:
   ~ReferenceStore();
 
-  /**
-   * Recover all Docker Images that are on disk by checking if all
-   * layer dependencies for that layer are present on disk.
-   */
-  void initialize();
-
   static Try<process::Owned<ReferenceStore>> create(const Flags& flags);
 
   /**
@@ -85,6 +79,11 @@ public:
    */
   process::Future<Option<DockerImage>> get(const std::string& name);
 
+  /**
+   * Recover all stored DockerImage and its layer references.
+   */
+  process::Future<Nothing> recover();
+
 private:
   explicit ReferenceStore(process::Owned<ReferenceStoreProcess> process);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb2df960/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index b9cb770..a9201d5 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -49,14 +49,19 @@ public:
   virtual ~Store() {}
 
   /**
-  * Get image by name.
-  *
-  * @param name The name of the Docker image to retrieve from store.
-  *
-  * @return The DockerImage that holds the Docker layers.
-  */
+   * Get image by name.
+   *
+   * @param name The name of the Docker image to retrieve from store.
+   *
+   * @return The DockerImage that holds the Docker layers.
+   */
   virtual process::Future<DockerImage> get(const std::string& name) = 0;
 
+  /**
+   * Recover all stored images
+   */
+  virtual process::Future<Nothing> recover() = 0;
+
   // TODO(chenlily): Implement removing an image.
 
 protected:


[11/17] mesos git commit: Restructure Docker provisioner and protos.

Posted by tn...@apache.org.
Restructure Docker provisioner and protos.


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

Branch: refs/heads/master
Commit: 4dedbf4aec2183af94ee083d2f6fb22057f9dfc8
Parents: 2cc0dfb
Author: Timothy Chen <tn...@gmail.com>
Authored: Fri Sep 11 00:19:42 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   9 +-
 src/messages/docker_provisioner.hpp             |  24 -
 src/messages/docker_provisioner.proto           |  35 --
 src/slave/containerizer/provisioner.cpp         |   1 +
 .../provisioner/docker/local_store.cpp          | 460 ++++++++++++++++++
 .../provisioner/docker/local_store.hpp          |  64 +++
 .../provisioner/docker/message.hpp              |  71 +++
 .../provisioner/docker/message.proto            |  43 ++
 .../provisioner/docker/metadata_manager.cpp     | 251 ++++++++++
 .../provisioner/docker/metadata_manager.hpp     | 106 +++++
 .../containerizer/provisioner/docker/paths.cpp  | 104 ++++
 .../containerizer/provisioner/docker/paths.hpp  |  86 ++++
 .../containerizer/provisioner/docker/store.cpp  |  47 ++
 .../containerizer/provisioner/docker/store.hpp  |  58 +++
 .../containerizer/provisioner/provisioner.hpp   |   2 +-
 src/slave/containerizer/provisioners/docker.cpp | 425 -----------------
 src/slave/containerizer/provisioners/docker.hpp | 166 -------
 .../provisioners/docker/local_store.cpp         | 471 -------------------
 .../provisioners/docker/local_store.hpp         |  66 ---
 .../provisioners/docker/metadata_manager.cpp    | 254 ----------
 .../provisioners/docker/metadata_manager.hpp    | 104 ----
 .../containerizer/provisioners/docker/paths.cpp | 104 ----
 .../containerizer/provisioners/docker/paths.hpp |  86 ----
 .../containerizer/provisioners/docker/store.hpp |  76 ---
 src/slave/flags.cpp                             |  12 +-
 src/slave/flags.hpp                             |   7 +-
 .../containerizer/provisioner_docker_tests.cpp  |  59 ++-
 27 files changed, 1336 insertions(+), 1855 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index a2a64dc..818d62d 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -250,8 +250,9 @@ BUILT_SOURCES += $(STATE_PROTOS)
 CLEANFILES += $(STATE_PROTOS)
 
 DOCKER_PROVISIONER_PROTOS =                                             \
-  messages/docker_provisioner.pb.cc                                     \
-  messages/docker_provisioner.pb.h
+  slave/containerizer/provisioner/docker/message.pb.cc			\
+  slave/containerizer/provisioner/docker/message.pb.h
+
 
 BUILT_SOURCES += $(DOCKER_PROVISIONER_PROTOS)
 CLEANFILES += $(DOCKER_PROVISIONER_PROTOS)
@@ -529,11 +530,11 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/provisioner/appc/store.cpp			\
 	slave/containerizer/provisioner/backend.cpp			\
 	slave/containerizer/provisioner/backends/copy.cpp		\
-	slave/containerizer/provisioner/docker.cpp			\
 	slave/containerizer/provisioner/docker/local_store.cpp		\
 	slave/containerizer/provisioner/docker/metadata_manager.cpp	\
 	slave/containerizer/provisioner/docker/paths.cpp		\
 	slave/containerizer/provisioner/docker/registry_client.cpp	\
+	slave/containerizer/provisioner/docker/store.cpp		\
 	slave/containerizer/provisioner/docker/token_manager.cpp	\
 	slave/resource_estimators/noop.cpp				\
 	usage/usage.cpp							\
@@ -831,7 +832,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backends/copy.hpp		\
 	slave/containerizer/provisioner/docker.hpp			\
 	slave/containerizer/provisioner/docker/local_store.hpp		\
-	slave/containerizer/provisioners/docker/metadata_manager.hpp    \
+	slave/containerizer/provisioner/docker/metadata_manager.hpp	\
 	slave/containerizer/provisioner/docker/paths.hpp		\
 	slave/containerizer/provisioner/docker/registry_client.hpp	\
 	slave/containerizer/provisioner/docker/store.hpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/messages/docker_provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/messages/docker_provisioner.hpp b/src/messages/docker_provisioner.hpp
deleted file mode 100644
index 684bd46..0000000
--- a/src/messages/docker_provisioner.hpp
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * 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 __MESSAGES_DOCKER_PROVISIONER_HPP__
-#define __MESSAGES_DOCKER_PROVISIONER_HPP__
-
-#include "messages/docker_provisioner.pb.h"
-
-#endif // __MESSAGES_DOCKER_PROVISIONER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/messages/docker_provisioner.proto
----------------------------------------------------------------------
diff --git a/src/messages/docker_provisioner.proto b/src/messages/docker_provisioner.proto
deleted file mode 100644
index 9de6707..0000000
--- a/src/messages/docker_provisioner.proto
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * 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.
- */
-
-import "mesos/mesos.proto";
-
-package mesos.internal;
-
-/**
- * A Docker Image name and the layer ids of the layers that comprise the image.
- * The layerIds are ordered, with the root layer id (no parent layer id) first
- * and the leaf layer id last.
- */
-message DockerProvisionerImages {
-  message Image {
-    required string name = 1;
-    repeated string layer_ids = 2;
-  }
-
-  repeated Image images = 1;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner.cpp b/src/slave/containerizer/provisioner.cpp
index 1ff3f10..6a0faac 100644
--- a/src/slave/containerizer/provisioner.cpp
+++ b/src/slave/containerizer/provisioner.cpp
@@ -23,6 +23,7 @@
 #include "slave/containerizer/provisioner.hpp"
 
 #include "slave/containerizer/provisioners/appc/provisioner.hpp"
+
 #include "slave/containerizer/provisioners/docker/provisioner.hpp"
 
 using namespace process;

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/local_store.cpp b/src/slave/containerizer/provisioner/docker/local_store.cpp
new file mode 100644
index 0000000..6a73dbb
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/local_store.cpp
@@ -0,0 +1,460 @@
+/**
+ * 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 <list>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include <stout/json.hpp>
+#include <stout/os.hpp>
+#include <stout/result.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "slave/containerizer/provisioner/docker/local_store.hpp"
+#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
+#include "slave/containerizer/provisioner/docker/paths.hpp"
+#include "slave/containerizer/provisioner/docker/store.hpp"
+
+#include "slave/flags.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+class LocalStoreProcess : public process::Process<LocalStoreProcess>
+{
+public:
+  LocalStoreProcess(
+      const Flags& _flags,
+      Owned<MetadataManager> _metadataManager)
+    : flags(_flags), metadataManager(_metadataManager) {}
+
+  ~LocalStoreProcess() {}
+
+  static Try<process::Owned<LocalStoreProcess>> create(const Flags& flags);
+
+  process::Future<vector<string>> get(const Image& image);
+
+  process::Future<Nothing> recover();
+
+private:
+  process::Future<DockerImage> _get(
+      const DockerImage::Name& name,
+      const Option<DockerImage>& image);
+
+  process::Future<Nothing> untarImage(
+      const std::string& tarPath,
+      const std::string& staging);
+
+  process::Future<DockerImage> putImage(
+      const DockerImage::Name& name,
+      const std::string& staging);
+
+  Result<std::string> getParentId(
+      const std::string& staging,
+      const std::string& layerId);
+
+  process::Future<Nothing> putLayers(
+      const std::string& staging,
+      const std::list<std::string>& layerIds);
+
+  process::Future<Nothing> putLayer(
+      const std::string& staging,
+      const std::string& id);
+
+  process::Future<Nothing> moveLayer(
+      const std::string& staging,
+      const std::string& id);
+
+  const Flags flags;
+  process::Owned<MetadataManager> metadataManager;
+};
+
+
+Try<Owned<slave::Store>> LocalStore::create(const Flags& flags)
+{
+  if (!os::exists(flags.docker_store_dir)) {
+    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store directory: " + mkdir.error());
+    }
+  }
+
+  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
+    Try<Nothing> mkdir =
+      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
+    if (mkdir.isError()) {
+      return Error("Failed to create Docker store staging directory: " +
+                   mkdir.error());
+    }
+  }
+
+  Try<Owned<MetadataManager>> metadataManager = MetadataManager::create(flags);
+  if (metadataManager.isError()) {
+    return Error(metadataManager.error());
+  }
+
+  Owned<LocalStoreProcess> process(
+      new LocalStoreProcess(flags, metadataManager.get()));
+
+  return Owned<slave::Store>(new LocalStore(process));
+}
+
+
+LocalStore::LocalStore(Owned<LocalStoreProcess> _process) : process(_process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+LocalStore::~LocalStore()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<vector<string>> LocalStore::get(const Image& image)
+{
+  return dispatch(process.get(), &LocalStoreProcess::get, image);
+}
+
+
+Future<Nothing> LocalStore::recover()
+{
+  return dispatch(process.get(), &LocalStoreProcess::recover);
+}
+
+
+Future<vector<string>> LocalStoreProcess::get(const Image& image)
+{
+  CHECK_EQ(image.type(), Image::DOCKER);
+
+  Try<DockerImage::Name> dockerName = parseName(image.docker().name());
+  if (dockerName.isError()) {
+    return Failure("Unable to parse docker image name: " + dockerName.error());
+  }
+
+  return metadataManager->get(dockerName.get())
+    .then(defer(self(), &Self::_get, dockerName.get(), lambda::_1))
+    .then([](const DockerImage& dockerImage) {
+      vector<string> layers;
+      foreach (const string& layer, dockerImage.layer_ids()) {
+        layers.push_back(layer);
+      }
+
+      return layers;
+    });
+}
+
+
+Future<DockerImage> LocalStoreProcess::_get(
+    const DockerImage::Name& name,
+    const Option<DockerImage>& image)
+{
+  if (image.isSome()) {
+    return image.get();
+  }
+
+  string tarPath = paths::getLocalImageTarPath(
+      flags.docker_store_discovery_local_dir,
+      stringify(name));
+
+  if (!os::exists(tarPath)) {
+    VLOG(1) << "Unable to find image in local store with path: " << tarPath;
+    return Failure("No Docker image tar archive found");
+  }
+
+  // Create a temporary staging directory.
+  Try<string> staging =
+    os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
+  if (staging.isError()) {
+    return Failure("Failed to create a staging directory");
+  }
+
+  return untarImage(tarPath, staging.get())
+    .then(defer(self(), &Self::putImage, name, staging.get()));
+}
+
+
+Future<Nothing> LocalStoreProcess::recover()
+{
+  return metadataManager->recover();
+}
+
+
+Future<Nothing> LocalStoreProcess::untarImage(
+    const string& tarPath,
+    const string& staging)
+{
+  VLOG(1) << "Untarring image at: " << tarPath;
+
+  // Untar store_discovery_local_dir/name.tar into staging/.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    staging,
+    "-x",
+    "-f",
+    tarPath
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([=](const Option<int>& status) -> Future<Nothing> {
+      if (status.isNone()) {
+        return Failure("Failed to reap status for tar subprocess in " +
+                        tarPath);
+      }
+      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+          return Failure("Untar image failed with exit code: " +
+                          WSTRINGIFY(status.get()));
+      }
+
+      return Nothing();
+    });
+}
+
+
+Future<DockerImage> LocalStoreProcess::putImage(
+    const DockerImage::Name& name,
+    const string& staging)
+{
+  Try<string> value = os::read(paths::getLocalImageRepositoriesPath(staging));
+  if (value.isError()) {
+    return Failure("Failed to read repository JSON: " + value.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
+  if (json.isError()) {
+    return Failure("Failed to parse JSON: " + json.error());
+  }
+
+  Result<JSON::Object> repositoryValue =
+    json.get().find<JSON::Object>(name.repository());
+  if (repositoryValue.isError()) {
+    return Failure("Failed to find repository: " + repositoryValue.error());
+  } else if (repositoryValue.isNone()) {
+    return Failure("Repository '" + name.repository() + "' is not found");
+  }
+
+  JSON::Object repositoryJson = repositoryValue.get();
+
+  // We don't use JSON find here because a tag might contain a '.'.
+  std::map<string, JSON::Value>::const_iterator entry =
+    repositoryJson.values.find(name.tag());
+  if (entry == repositoryJson.values.end()) {
+    return Failure("Tag '" + name.tag() + "' is not found");
+  } else if (!entry->second.is<JSON::String>()) {
+    return Failure("Tag JSON value expected to be JSON::String");
+  }
+
+  string layerId = entry->second.as<JSON::String>().value;
+
+  Try<string> manifest =
+    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
+  if (manifest.isError()) {
+    return Failure("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
+  if (manifestJson.isError()) {
+    return Failure("Failed to parse manifest: " + manifestJson.error());
+  }
+
+  list<string> layerIds;
+  layerIds.push_back(layerId);
+  Result<string> parentId = getParentId(staging, layerId);
+  while(parentId.isSome()) {
+    layerIds.push_front(parentId.get());
+    parentId = getParentId(staging, parentId.get());
+  }
+  if (parentId.isError()) {
+    return Failure("Failed to obtain parent layer id: " + parentId.error());
+  }
+
+  return putLayers(staging, layerIds)
+    .then([=]() -> Future<DockerImage> {
+      return metadataManager->put(name, layerIds);
+    });
+}
+
+
+Result<string> LocalStoreProcess::getParentId(
+    const string& staging,
+    const string& layerId)
+{
+  Try<string> manifest =
+    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
+  if (manifest.isError()) {
+    return Error("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
+  if (json.isError()) {
+    return Error("Failed to parse manifest: " + json.error());
+  }
+
+  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
+  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
+    return None();
+  } else if (parentId.isError()) {
+    return Error("Failed to read parent of layer: " + parentId.error());
+  }
+  return parentId.get().value;
+}
+
+
+Future<Nothing> LocalStoreProcess::putLayers(
+    const string& staging,
+    const list<string>& layerIds)
+{
+  list<Future<Nothing>> futures{ Nothing() };
+  foreach (const string& layer, layerIds) {
+    futures.push_back(
+        futures.back().then(
+          defer(self(), &Self::putLayer, staging, layer)));
+  }
+
+  return collect(futures)
+    .then([]() -> Future<Nothing> { return Nothing(); });
+}
+
+
+Future<Nothing> LocalStoreProcess::putLayer(
+    const string& staging,
+    const string& id)
+{
+  // We untar the layer from source into a staging directory, then
+  // move the layer into store. We do this instead of untarring
+  // directly to store to make sure we don't end up with partially
+  // untarred layer rootfs.
+
+  // Check if image layer rootfs is already in store.
+  if (os::exists(paths::getImageLayerRootfsPath(flags.docker_store_dir, id))) {
+    VLOG(1) << "Image layer '" << id << "' rootfs already in store. "
+            << "Skipping put.";
+    return Nothing();
+  }
+
+  const string imageLayerPath =
+    paths::getImageLayerPath(flags.docker_store_dir, id);
+  if (!os::exists(imageLayerPath)) {
+    Try<Nothing> mkdir = os::mkdir(imageLayerPath);
+    if (mkdir.isError()) {
+      return Failure("Failed to create Image layer directory '" +
+                     imageLayerPath + "': " + mkdir.error());
+    }
+  }
+
+  // Image layer has been untarred but is not present in the store directory.
+  string localRootfsPath = paths::getLocalImageLayerRootfsPath(staging, id);
+  if (os::exists(localRootfsPath)) {
+    LOG(WARNING) << "Image layer '" << id << "' rootfs present at but not in "
+                 << "store directory '" << localRootfsPath << "'. Removing "
+                 << "staged rootfs and untarring layer again.";
+    Try<Nothing> rmdir = os::rmdir(localRootfsPath);
+    if (rmdir.isError()) {
+      return Failure("Failed to remove incomplete staged rootfs for layer '" +
+                     id + "': " + rmdir.error());
+    }
+  }
+
+  Try<Nothing> mkdir = os::mkdir(localRootfsPath);
+  if (mkdir.isError()) {
+    return Failure("Failed to create rootfs path '" + localRootfsPath + "': " +
+                   mkdir.error());
+  }
+  // Untar staging/id/layer.tar into staging/id/rootfs.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    localRootfsPath,
+    "-x",
+    "-f",
+    paths::getLocalImageLayerTarPath(staging, id)
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([=](const Option<int>& status) -> Future<Nothing> {
+      if (status.isNone()) {
+        return Failure("Failed to reap subprocess to untar image");
+      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+        return Failure("Untar failed with exit code: " +
+                        WSTRINGIFY(status.get()));
+      }
+
+      return moveLayer(staging, id);
+    });
+}
+
+
+Future<Nothing> LocalStoreProcess::moveLayer(
+    const string& staging,
+    const string& id)
+{
+  Try<Nothing> status = os::rename(
+      paths::getLocalImageLayerRootfsPath(staging, id),
+      paths::getImageLayerRootfsPath(flags.docker_store_dir, id));
+
+  if (status.isError()) {
+    return Failure("Failed to move layer to store directory: "
+                   + status.error());
+  }
+
+  return Nothing();
+}
+
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/local_store.hpp b/src/slave/containerizer/provisioner/docker/local_store.hpp
new file mode 100644
index 0000000..5f6152b
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/local_store.hpp
@@ -0,0 +1,64 @@
+/**
+ * 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_LOCAL_STORE_HPP__
+#define __MESOS_DOCKER_LOCAL_STORE_HPP__
+
+#include "slave/containerizer/provisioner/store.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Forward declaration.
+class LocalStoreProcess;
+
+
+/**
+ * LocalStore assumes Docker images are stored in a local directory
+ * (configured with flags.docker_discovery_local_dir), with all the
+ * images saved as tar with the name as the image name with tag (e.g:
+ * ubuntu:14.04.tar).
+ */
+class LocalStore : public slave::Store
+{
+public:
+  static Try<process::Owned<Store>> create(const Flags& flags);
+
+  virtual ~LocalStore();
+
+  virtual process::Future<Nothing> recover();
+
+  virtual process::Future<std::vector<std::string>> get(const Image& image);
+
+private:
+  explicit LocalStore(process::Owned<LocalStoreProcess> _process);
+
+  LocalStore& operator=(const LocalStore&) = delete; // Not assignable.
+  LocalStore(const LocalStore&) = delete; // Not copyable.
+
+  process::Owned<LocalStoreProcess> process;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_LOCAL_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/message.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/message.hpp b/src/slave/containerizer/provisioner/docker/message.hpp
new file mode 100644
index 0000000..c1596df
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/message.hpp
@@ -0,0 +1,71 @@
+/**
+ * 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 __MESSAGES_DOCKER_PROVISIONER_HPP__
+#define __MESSAGES_DOCKER_PROVISIONER_HPP__
+
+#include <stout/strings.hpp>
+
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include "slave/containerizer/provisioner/docker/message.pb.h"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+inline DockerImage::Name parseName(const std::string& value)
+{
+  DockerImage::Name imageName;
+  Option<std::string> registry = None();
+  std::vector<std::string> components = strings::split(value, "/");
+  if (components.size() > 2) {
+    imageName.set_registry(value.substr(0, value.find_last_of("/")));
+  }
+
+  std::size_t found = components.back().find_last_of(':');
+  if (found == std::string::npos) {
+    imageName.set_repository(components.back());
+    imageName.set_tag("latest");
+  } else {
+    imageName.set_repository(components.back().substr(0, found));
+    imageName.set_tag(components.back().substr(found + 1));
+  }
+
+  return imageName;
+}
+
+
+inline std::ostream& operator<<(
+    std::ostream& stream,
+    const DockerImage::Name& name)
+{
+  if (name.has_registry()) {
+    return stream << name.registry() << "/" << name.repository() << ":"
+                  << name.tag();
+  }
+
+  return stream << name.repository() << ":" << name.tag();
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESSAGES_DOCKER_PROVISIONER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/message.proto
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/message.proto b/src/slave/containerizer/provisioner/docker/message.proto
new file mode 100644
index 0000000..d771968
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/message.proto
@@ -0,0 +1,43 @@
+/**
+ * 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.
+ */
+
+import "mesos/mesos.proto";
+
+package mesos.internal.slave.docker;
+
+/**
+ * A Docker Image name and the layer ids of the layers that comprise the image.
+ * The layerIds are ordered, with the root layer id (no parent layer id) first
+ * and the leaf layer id last.
+ */
+message DockerImage {
+  message Name {
+    optional string registry = 1;
+    required string repository = 2;
+    required string tag = 3;
+  }
+
+  required Name name = 1;
+
+  // The order of the layers represents the dependency between layers.
+  repeated string layer_ids = 2;
+}
+
+message DockerImages {
+  repeated DockerImage images = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/metadata_manager.cpp b/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
new file mode 100644
index 0000000..197931c
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/metadata_manager.cpp
@@ -0,0 +1,251 @@
+/**
+ * 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 <vector>
+
+#include <glog/logging.h>
+
+#include <stout/foreach.hpp>
+#include <stout/hashset.hpp>
+#include <stout/os.hpp>
+#include <stout/protobuf.hpp>
+
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/owned.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "slave/containerizer/provisioner/docker/paths.hpp"
+#include "slave/containerizer/provisioner/docker/message.hpp"
+#include "slave/containerizer/provisioner/docker/metadata_manager.hpp"
+
+#include "slave/state.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+
+class MetadataManagerProcess : public process::Process<MetadataManagerProcess>
+{
+public:
+  ~MetadataManagerProcess() {}
+
+  static Try<process::Owned<MetadataManagerProcess>> create(
+      const Flags& flags);
+
+  Future<DockerImage> put(
+      const DockerImage::Name& name,
+      const std::list<std::string>& layerIds);
+
+  Future<Option<DockerImage>> get(const DockerImage::Name& name);
+
+  Future<Nothing> recover();
+
+  // TODO(chenlily): Implement removal of unreferenced images.
+
+private:
+  MetadataManagerProcess(const Flags& flags);
+
+  // Write out metadata manager state to persistent store.
+  Try<Nothing> persist();
+
+  const Flags flags;
+
+  // This is a lookup table for images that are stored in memory. It is keyed
+  // by the name of the DockerImage.
+  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
+  hashmap<std::string, DockerImage> storedImages;
+};
+
+
+Try<Owned<MetadataManager>> MetadataManager::create(const Flags& flags)
+{
+  Try<Owned<MetadataManagerProcess>> process =
+    MetadataManagerProcess::create(flags);
+  if (process.isError()) {
+    return Error("Failed to create Metadata Manager: " + process.error());
+  }
+  return Owned<MetadataManager>(new MetadataManager(process.get()));
+}
+
+
+MetadataManager::MetadataManager(Owned<MetadataManagerProcess> process)
+  : process(process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+MetadataManager::~MetadataManager()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<Nothing> MetadataManager::recover()
+{
+  return process::dispatch(process.get(), &MetadataManagerProcess::recover);
+}
+
+
+Future<DockerImage> MetadataManager::put(
+    const DockerImage::Name& name,
+    const list<string>& layerIds)
+{
+  return dispatch(
+      process.get(), &MetadataManagerProcess::put, name, layerIds);
+}
+
+
+Future<Option<DockerImage>> MetadataManager::get(const DockerImage::Name& name)
+{
+  return dispatch(process.get(), &MetadataManagerProcess::get, name);
+}
+
+
+MetadataManagerProcess::MetadataManagerProcess(const Flags& flags)
+  : flags(flags) {}
+
+
+Try<Owned<MetadataManagerProcess>> MetadataManagerProcess::create(
+    const Flags& flags)
+{
+  Owned<MetadataManagerProcess> metadataManager =
+    Owned<MetadataManagerProcess>(new MetadataManagerProcess(flags));
+
+  return metadataManager;
+}
+
+
+Future<DockerImage> MetadataManagerProcess::put(
+    const DockerImage::Name& name,
+    const list<string>& layerIds)
+{
+  const string imageName = stringify(name);
+
+  DockerImage dockerImage;
+  dockerImage.mutable_name()->CopyFrom(name);
+  foreach (const string& layerId, layerIds) {
+    dockerImage.add_layer_ids(layerId);
+  }
+
+  storedImages[imageName] = dockerImage;
+
+  Try<Nothing> status = persist();
+  if (status.isError()) {
+    return Failure("Failed to save state of Docker images" + status.error());
+  }
+
+  return dockerImage;
+}
+
+
+Future<Option<DockerImage>> MetadataManagerProcess::get(
+    const DockerImage::Name& name)
+{
+  const string imageName = stringify(name);
+
+  if (!storedImages.contains(imageName)) {
+    return None();
+  }
+
+  return storedImages[imageName];
+}
+
+
+Try<Nothing> MetadataManagerProcess::persist()
+{
+  DockerImages images;
+
+  foreachvalue (const DockerImage& image, storedImages) {
+    images.add_images()->CopyFrom(image);
+  }
+
+  Try<Nothing> status = mesos::internal::slave::state::checkpoint(
+      paths::getStoredImagesPath(flags.docker_store_dir), images);
+  if (status.isError()) {
+    return Error("Failed to perform checkpoint: " + status.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> MetadataManagerProcess::recover()
+{
+  string storedImagesPath = paths::getStoredImagesPath(flags.docker_store_dir);
+
+  storedImages.clear();
+  if (!os::exists(storedImagesPath)) {
+    LOG(INFO) << "No images to load from disk. Docker provisioner image "
+              << "storage path: " << storedImagesPath << " does not exist.";
+    return Nothing();
+  }
+
+  Result<DockerImages> images =
+    ::protobuf::read<DockerImages>(storedImagesPath);
+  if (images.isError()) {
+    return Failure("Failed to read protobuf for Docker provisioner image: " +
+                   images.error());
+  }
+
+  foreach (const DockerImage image, images.get().images()) {
+    vector<string> missingLayerIds;
+    foreach (const string layerId, image.layer_ids()) {
+      const string rootfsPath =
+        paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId);
+
+      if (!os::exists(rootfsPath)) {
+        missingLayerIds.push_back(layerId);
+      }
+    }
+
+    if (!missingLayerIds.empty()) {
+      LOG(WARNING) << "Skipped loading image: " << stringify(image.name())
+                   << " due to missing layers: " << stringify(missingLayerIds);
+      continue;
+    }
+
+    const string imageName = stringify(image.name());
+    if (storedImages.contains(imageName)) {
+      LOG(WARNING) << "Found duplicate image in recovery for image name '" << imageName
+                   << "'";
+    } else {
+      storedImages[imageName] = image;
+    }
+  }
+
+  LOG(INFO) << "Loaded " << storedImages.size() << " Docker images.";
+
+  return Nothing();
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/metadata_manager.hpp b/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
new file mode 100644
index 0000000..647a478
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/metadata_manager.hpp
@@ -0,0 +1,106 @@
+/**
+ * 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_METADATA_MANAGER_HPP__
+#define __MESOS_DOCKER_METADATA_MANAGER_HPP__
+
+#include <list>
+#include <string>
+
+#include <stout/hashmap.hpp>
+#include <stout/json.hpp>
+#include <stout/option.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include "slave/containerizer/provisioner/provisioner.hpp"
+
+#include "slave/containerizer/provisioner/docker/message.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Forward Declaration.
+class MetadataManagerProcess;
+
+/**
+ * The MetadataManager tracks the Docker images cached by the
+ * provisioner that are stored on disk. It keeps track of the layers
+ * that Docker images are composed of and recovers DockerImage objects
+ * upon initialization by checking for dependent layers stored on disk.
+ * Currently, image layers are stored indefinitely, with no garbage
+ * collection of unreferenced image layers.
+ */
+class MetadataManager
+{
+public:
+  static Try<process::Owned<MetadataManager>> create(const Flags& flags);
+
+   ~MetadataManager();
+
+  /**
+   * Create a Image, put it in metadata manager and persist the reference
+   * store state to disk.
+   *
+   * @param name     the name of the Docker image to place in the reference
+   *                 store.
+   * @param layerIds the list of layer ids that comprise the Docker image in
+   *                 order where the root layer's id (no parent layer) is first
+   *                 and the leaf layer's id is last.
+   */
+  process::Future<DockerImage> put(
+      const DockerImage::Name& name,
+      const std::list<std::string>& layerIds);
+
+  /**
+   * Retrieve DockerImage based on image name if it is among the DockerImages
+   * stored in memory.
+   *
+   * @param name  the name of the Docker image to retrieve
+   */
+  process::Future<Option<DockerImage>> get(const DockerImage::Name& name);
+
+  /**
+   * Recover all stored DockerImage and its layer references.
+   */
+  process::Future<Nothing> recover();
+
+private:
+  explicit MetadataManager(process::Owned<MetadataManagerProcess> process);
+
+  MetadataManager(const MetadataManager&); // Not copyable.
+  MetadataManager& operator=(const MetadataManager&); // Not assignable.
+
+  process::Owned<MetadataManagerProcess> process;
+};
+
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_METADATA_MANAGER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/paths.cpp b/src/slave/containerizer/provisioner/docker/paths.cpp
new file mode 100644
index 0000000..81a2176
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/paths.cpp
@@ -0,0 +1,104 @@
+/**
+ * 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/provisioner/docker/paths.hpp"
+
+#include <stout/path.hpp>
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+namespace paths {
+
+string getStagingDir(const string& storeDir)
+{
+  return path::join(storeDir, "staging");
+}
+
+string getTempStaging(const string& storeDir)
+{
+  return path::join(getStagingDir(storeDir), "XXXXXX");
+}
+
+string getLocalImageTarPath(
+    const string& discoveryDir,
+    const string& name)
+{
+  return path::join(discoveryDir, name + ".tar");
+}
+
+string getLocalImageRepositoriesPath(const string& staging)
+{
+  return path::join(staging, "repositories");
+}
+
+std::string getLocalImageLayerPath(
+    const string& staging,
+    const string& layerId)
+{
+  return path::join(staging, layerId);
+}
+
+string getLocalImageLayerManifestPath(
+    const string& staging,
+    const string& layerId)
+{
+  return path::join(getLocalImageLayerPath(staging, layerId), "json");
+}
+
+string getLocalImageLayerTarPath(
+  const string& staging,
+  const string& layerId)
+{
+  return path::join(getLocalImageLayerPath(staging, layerId), "layer.tar");
+}
+
+string getLocalImageLayerRootfsPath(
+    const string& staging,
+    const string& layerId)
+{
+  return path::join(getLocalImageLayerPath(staging, layerId), "rootfs");
+}
+
+string getImageLayerPath(
+    const string& storeDir,
+    const string& layerId)
+{
+  return path::join(storeDir, "layers", layerId);
+}
+
+string getImageLayerRootfsPath(
+    const string& storeDir,
+    const string& layerId)
+{
+  return path::join(getImageLayerPath(storeDir, layerId), "rootfs");
+}
+
+string getStoredImagesPath(const string& storeDir)
+{
+  return path::join(storeDir, "storedImages");
+}
+
+} // namespace paths {
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/paths.hpp b/src/slave/containerizer/provisioner/docker/paths.hpp
new file mode 100644
index 0000000..02f129f
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/paths.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 __MESOS_DOCKER_PATHS_HPP__
+#define __MESOS_DOCKER_PATHS_HPP__
+
+#include <list>
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+namespace paths {
+
+/**
+ * The Docker store file system layout is as follows:
+ * Image store dir ('--docker_store_dir' slave flag)
+ *    |--staging (contains temp directories for downloads and extract)
+ *    |--layers
+ *       |--<layer_id>
+ *           |--rootfs
+ *    |--rootfses
+ *    |--storedImages (file holding on cached images)
+ */
+
+std::string getStagingDir(const std::string& storeDir);
+
+std::string getTempStaging(const std::string& storeDir);
+
+std::string getLocalImageTarPath(
+    const std::string& discoveryDir,
+    const std::string& name);
+
+std::string getLocalImageRepositoriesPath(const std::string& staging);
+
+std::string getLocalImageLayerPath(
+    const std::string& staging,
+    const std::string& layerId);
+
+std::string getLocalImageLayerManifestPath(
+    const std::string& staging,
+    const std::string& layerId);
+
+std::string getLocalImageLayerTarPath(
+  const std::string& staging,
+  const std::string& layerId);
+
+std::string getLocalImageLayerRootfsPath(
+  const std::string& staging,
+  const std::string& layerId);
+
+std::string getImageLayerPath(
+    const std::string& storeDir,
+    const std::string& layerId);
+
+std::string getImageLayerRootfsPath(
+    const std::string& storeDir,
+    const std::string& layerId);
+
+std::string getStoredImagesPath(const std::string& storeDir);
+
+} // namespace paths {
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_PATHS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/store.cpp b/src/slave/containerizer/provisioner/docker/store.cpp
new file mode 100644
index 0000000..51f4d6b
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/store.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 "slave/containerizer/provisioner/docker/local_store.hpp"
+#include "slave/containerizer/provisioner/docker/store.hpp"
+
+using process::Owned;
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+Try<Owned<slave::Store>> Store::create(const Flags& flags)
+{
+  hashmap<string, Try<Owned<slave::Store>>(*)(const Flags&)> creators{
+    {"local", &LocalStore::create}
+  };
+
+  if (!creators.contains(flags.docker_store_discovery)) {
+    return Error("Unknown Docker store: " + flags.docker_store_discovery);
+  }
+
+  return creators[flags.docker_store_discovery](flags);
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/docker/store.hpp b/src/slave/containerizer/provisioner/docker/store.hpp
new file mode 100644
index 0000000..ae06706
--- /dev/null
+++ b/src/slave/containerizer/provisioner/docker/store.hpp
@@ -0,0 +1,58 @@
+/**
+ * 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_STORE_HPP__
+#define __MESOS_DOCKER_STORE_HPP__
+
+#include <string>
+
+#include <stout/hashmap.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+
+#include "slave/containerizer/fetcher.hpp"
+
+#include "slave/containerizer/provisioner/store.hpp"
+
+#include "slave/containerizer/provisioner/docker/message.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Store fetches the Docker images and stores them on disk.
+// TODO(tnachen): Make this store the only docker store
+// implementation, and move local and remote to different pullers.
+class Store
+{
+public:
+  static Try<process::Owned<slave::Store>> create(const Flags& flags);
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioner/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/provisioner.hpp b/src/slave/containerizer/provisioner/provisioner.hpp
index d42822b..912fc5a 100644
--- a/src/slave/containerizer/provisioner/provisioner.hpp
+++ b/src/slave/containerizer/provisioner/provisioner.hpp
@@ -67,7 +67,7 @@ public:
   // image and return the absolute path to the root filesystem.
   virtual process::Future<std::string> provision(
       const ContainerID& containerId,
-      const Image& image) = 0;
+      const Image& image);
 
   // Destroy a previously provisioned root filesystem. Assumes that
   // all references (e.g., mounts, open files) to the provisioned

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
deleted file mode 100644
index b5c5a7b..0000000
--- a/src/slave/containerizer/provisioners/docker.cpp
+++ /dev/null
@@ -1,425 +0,0 @@
-/**
- * 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 "slave/containerizer/provisioners/backend.hpp"
-#include "slave/containerizer/provisioners/paths.hpp"
-
-#include "slave/containerizer/provisioners/docker/paths.hpp"
-#include "slave/containerizer/provisioners/docker/store.hpp"
-
-#include "slave/paths.hpp"
-
-using namespace process;
-using namespace mesos::internal::slave;
-
-using std::list;
-using std::string;
-using std::vector;
-
-using mesos::slave::ContainerState;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-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);
-
-  process::Future<bool> destroy(const ContainerID& containerId);
-
-private:
-  DockerProvisionerProcess(
-      const string& _rootDir,
-      const Flags& _flags,
-      const process::Owned<Store>& _store,
-      const hashmap<std::string, process::Owned<Backend>>& _backends);
-
-  process::Future<std::string> _provision(
-      const DockerImage& image,
-      const ContainerID& containerId,
-      const string& rootfs);
-
-  const string& rootDir;
-  const Flags flags;
-  process::Owned<Store> store;
-  hashmap<string, process::Owned<Backend>> backends;
-
-  struct Info
-  {
-    // Mappings: backend -> rootfsId -> rootfsPath.
-    hashmap<string, hashmap<string, string>> rootfses;
-  };
-
-  hashmap<ContainerID, Owned<Info>> infos;
-};
-
-
-Try<ImageName> ImageName::create(const std::string& name)
-{
-  ImageName imageName;
-  Option<string> registry = None();
-  std::vector<std::string> components = strings::split(name, "/");
-  if (components.size() > 2) {
-    registry = name.substr(0, name.find_last_of("/"));
-  }
-
-  std::size_t found = components.back().find_last_of(':');
-  if (found == std::string::npos) {
-    imageName.repository = components.back();
-    imageName.tag = "latest";
-  } else {
-    imageName.repository = components.back().substr(0, found);
-    imageName.tag = components.back().substr(found + 1);
-  }
-
-  return imageName;
-}
-
-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)
-{
-  return dispatch(
-      process.get(),
-      &DockerProvisionerProcess::provision,
-      containerId,
-      image);
-}
-
-
-Future<bool> DockerProvisioner::destroy(const ContainerID& containerId)
-{
-  return dispatch(
-      process.get(),
-      &DockerProvisionerProcess::destroy,
-      containerId);
-}
-
-
-Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  string rootDir =
-    slave::paths::getProvisionerDir(flags.work_dir, Image::DOCKER);
-
-  if (!os::exists(rootDir)) {
-    Try<Nothing> mkdir = os::mkdir(rootDir);
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker provisioner root directory '" +
-                   rootDir + "': " + mkdir.error());
-    }
-  }
-
-  const hashmap<string, Owned<Backend>> backends = Backend::create(flags);
-  if (backends.empty()) {
-    return Error("No usable Docker provisioner backend created");
-  }
-
-  if (!backends.contains(flags.docker_backend)) {
-    return Error("The specified Docker provisioner backend '" +
-                 flags.docker_backend + "'is unsupported");
-  }
-
-  Try<Owned<Store>> store = Store::create(flags, fetcher);
-  if (store.isError()) {
-    return Error("Failed to create image store: " + store.error());
-  }
-
-  return Owned<DockerProvisionerProcess>(
-      new DockerProvisionerProcess(
-          rootDir,
-          flags,
-          store.get(),
-          backends));
-}
-
-
-DockerProvisionerProcess::DockerProvisionerProcess(
-    const string& _rootDir,
-    const Flags& _flags,
-    const Owned<Store>& _store,
-    const hashmap<string, Owned<Backend>>& _backends)
-  : rootDir(_rootDir),
-    flags(_flags),
-    store(_store),
-    backends(_backends) {}
-
-
-Future<Nothing> DockerProvisionerProcess::recover(
-    const list<ContainerState>& states,
-    const hashset<ContainerID>& orphans)
-{
-  // TODO(tnachen): Consider merging this with
-  // AppcProvisionerProcess::recover.
-
-  // Register living containers, including the ones that do not
-  // provision Docker images.
-  hashset<ContainerID> alive;
-
-  foreach (const ContainerState& state, states) {
-    if (state.executor_info().has_container() &&
-        state.executor_info().container().type() == ContainerInfo::MESOS) {
-      alive.insert(state.container_id());
-    }
-  }
-
-  // List provisioned containers; recover living ones; destroy unknown orphans.
-  // Note that known orphan containers are recovered as well and they will
-  // be destroyed by the containerizer using the normal cleanup path. See
-  // MESOS-2367 for details.
-  Try<hashmap<ContainerID, string>> containers =
-    provisioners::paths::listContainers(rootDir);
-
-  if (containers.isError()) {
-    return Failure("Failed to list the containers managed by Docker "
-                   "provisioner: " + containers.error());
-  }
-
-  // If no container has been launched the 'containers' directory will be empty.
-  foreachkey (const ContainerID& containerId, containers.get()) {
-    if (alive.contains(containerId) || orphans.contains(containerId)) {
-      Owned<Info> info = Owned<Info>(new Info());
-
-      Try<hashmap<string, hashmap<string, string>>> rootfses =
-        provisioners::paths::listContainerRootfses(rootDir, containerId);
-
-      if (rootfses.isError()) {
-        return Failure("Unable to list rootfses belonged to container '" +
-                       containerId.value() + "': " + rootfses.error());
-      }
-
-      foreachkey (const string& backend, rootfses.get()) {
-        if (!backends.contains(backend)) {
-          return Failure("Found rootfses managed by an unrecognized backend: " +
-                         backend);
-        }
-
-        info->rootfses.put(backend, rootfses.get()[backend]);
-      }
-
-      VLOG(1) << "Recovered container " << containerId;
-      infos.put(containerId, info);
-
-      continue;
-    }
-
-    // Destroy (unknown) orphan container's rootfses.
-    Try<hashmap<string, hashmap<string, string>>> rootfses =
-      provisioners::paths::listContainerRootfses(rootDir, containerId);
-
-    if (rootfses.isError()) {
-      return Failure("Unable to find rootfses for container '" +
-                     containerId.value() + "': " + rootfses.error());
-    }
-
-    foreachkey (const string& backend, rootfses.get()) {
-      if (!backends.contains(backend)) {
-        return Failure("Found rootfses managed by an unrecognized backend: " +
-                       backend);
-      }
-
-      foreachvalue (const string& rootfs, rootfses.get()[backend]) {
-        VLOG(1) << "Destroying orphan rootfs " << rootfs;
-
-        // Not waiting for the destruction and we don't care about
-        // the return value.
-        backends.get(backend).get()->destroy(rootfs)
-          .onFailed([rootfs](const std::string& error) {
-            LOG(WARNING) << "Failed to destroy orphan rootfs '" << rootfs
-                         << "': "<< error;
-          });
-      }
-    }
-  }
-
-  LOG(INFO) << "Recovered Docker provisioner rootfses";
-
-  return store->recover()
-    .then([]() -> Future<Nothing> {
-      LOG(INFO) << "Recovered Docker image store";
-      return Nothing();
-    });
-}
-
-
-Future<string> DockerProvisionerProcess::provision(
-    const ContainerID& containerId,
-    const Image& image)
-{
-  if (image.type() != Image::DOCKER) {
-    return Failure("Unsupported container image type");
-  }
-
-  if (!image.has_docker()) {
-    return Failure("Missing Docker image info");
-  }
-
-  string rootfsId = UUID::random().toString();
-  string rootfs = provisioners::paths::getContainerRootfsDir(
-      rootDir, containerId, flags.docker_backend, rootfsId);
-
-  if (!infos.contains(containerId)) {
-    infos.put(containerId, Owned<Info>(new Info()));
-  }
-
-  infos[containerId]->rootfses[flags.docker_backend].put(rootfsId, rootfs);
-
-  Try<ImageName> imageName = ImageName::create(image.docker().name());
-  if (imageName.isError()) {
-    return Failure("Unable to able to parse Docker image name '" +
-                   image.docker().name() + "': " + imageName.error());
-  }
-
-  return store->get(imageName.get())
-    .then(defer(self(), &Self::_provision, lambda::_1, containerId, rootfs));
-}
-
-
-Future<string> DockerProvisionerProcess::_provision(
-    const DockerImage& image,
-    const ContainerID& containerId,
-    const string& rootfs)
-{
-  CHECK(backends.contains(flags.docker_backend));
-
-  LOG(INFO) << "Provisioning rootfs for container '" << containerId << "'"
-            << " to '" << rootfs << "'";
-
-  vector<string> layerPaths;
-  foreach (const string& layerId, image.layerIds) {
-    layerPaths.push_back(
-        paths::getImageLayerRootfsPath(flags.docker_store_dir, layerId));
-  }
-
-  return backends[flags.docker_backend]->provision(layerPaths, rootfs)
-    .then([rootfs]() -> Future<string> { return rootfs; });
-}
-
-
-Future<bool> DockerProvisionerProcess::destroy(
-    const ContainerID& containerId)
-{
-  // TODO(tnachen): Consider merging this with
-  // AppcProvisionerProcess::destroy.
-  if (!infos.contains(containerId)) {
-    LOG(INFO) << "Ignoring destroy request for unknown container: "
-              << containerId;
-
-    return false;
-  }
-
-  // Unregister the container first. If destroy() fails, we can rely on
-  // recover() to retry it later.
-  Owned<Info> info = infos[containerId];
-  infos.erase(containerId);
-
-  list<Future<bool>> futures;
-  foreachkey (const string& backend, info->rootfses) {
-    foreachvalue (const string& rootfs, info->rootfses[backend]) {
-      if (!backends.contains(backend)) {
-        return Failure("Cannot destroy rootfs '" + rootfs +
-                       "' provisioned by an unknown backend '" + backend + "'");
-      }
-
-      LOG(INFO) << "Destroying container rootfs for container '"
-                << containerId << "' at '" << rootfs << "'";
-
-      futures.push_back(
-          backends.get(backend).get()->destroy(rootfs));
-    }
-  }
-
-  return collect(futures)
-    .then([=](const list<bool>& results) -> Future<bool> {
-      return true;
-    });
-}
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
deleted file mode 100644
index cd7911c..0000000
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * 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_HPP__
-#define __MESOS_DOCKER_HPP__
-
-#include <list>
-#include <ostream>
-#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/strings.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/flags.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-// Forward declaration.
-class Store;
-
-/**
- * Represents Docker Image Name, which composes of a repository and a
- * tag.
- */
-struct ImageName
-{
-  static Try<ImageName> create(const std::string& name);
-
-  ImageName(
-      const std::string& _repository,
-      const std::string& _tag,
-      const Option<std::string>& _registry = None())
-    : repository(_repository),
-      tag(_tag),
-      registry(_registry) {}
-
-  ImageName() {}
-
-  /**
-   * The string representation of this image.
-   */
-  std::string name() const
-  {
-    if (registry.isSome()) {
-      return registry.get() +  "/" + repository + ":" + tag;
-    }
-
-    return repository + ":" + tag;
-  }
-
-  /**
-   * Repository of this image (e.g, ubuntu).
-   */
-  std::string repository;
-
-  /**
-   * Tag of this image (e.g: 14.04).
-   */
-  std::string tag;
-
-  /**
-   * Custom registry that the image points to.
-   */
-  Option<std::string> registry;
-};
-
-
-inline std::ostream& operator<<(std::ostream& stream, const ImageName& image)
-{
-  return stream << image.name();
-}
-
-
-/**
- * Represents a Docker Image that holds its name and all of its layers
- * sorted by its dependency.
- */
-struct DockerImage
-{
-  DockerImage() {}
-
-  DockerImage(
-      const ImageName& _imageName,
-      const std::list<std::string>& _layerIds)
-  : imageName(_imageName), layerIds(_layerIds) {}
-
-  ImageName imageName;
-  std::list<std::string> layerIds;
-};
-
-// Forward declaration.
-class DockerProvisionerProcess;
-
-/**
- * Docker Provisioner is responsible to provision rootfs for
- * containers with Docker images.
- */
-class DockerProvisioner : public mesos::internal::slave::Provisioner
-{
-public:
-  static Try<process::Owned<mesos::internal::slave::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);
-
-  virtual process::Future<bool> destroy(const ContainerID& containerId);
-
-private:
-  explicit DockerProvisioner(process::Owned<DockerProvisionerProcess> _process);
-
-  DockerProvisioner& operator=(const DockerProvisioner&) = delete; // Not assignable.
-  DockerProvisioner(const DockerProvisioner&) = delete; // Not copyable.
-
-  process::Owned<DockerProvisionerProcess> process;
-};
-
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
deleted file mode 100644
index ec0420e..0000000
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ /dev/null
@@ -1,471 +0,0 @@
-/**
- * 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/local_store.hpp"
-
-#include <list>
-#include <vector>
-
-#include <glog/logging.h>
-
-#include <stout/json.hpp>
-#include <stout/os.hpp>
-#include <stout/result.hpp>
-
-#include <process/collect.hpp>
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-
-#include "common/status_utils.hpp"
-
-#include "slave/containerizer/fetcher.hpp"
-
-#include "slave/containerizer/provisioners/docker/metadata_manager.hpp"
-#include "slave/containerizer/provisioners/docker/paths.hpp"
-#include "slave/containerizer/provisioners/docker/store.hpp"
-
-#include "slave/flags.hpp"
-
-using namespace process;
-
-using std::list;
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-class LocalStoreProcess : public process::Process<LocalStoreProcess>
-{
-public:
-  ~LocalStoreProcess() {}
-
-  static Try<process::Owned<LocalStoreProcess>> create(
-      const Flags& flags,
-      Fetcher* fetcher);
-
-  process::Future<DockerImage> get(const ImageName& name);
-
-  process::Future<Nothing> recover();
-
-private:
-  LocalStoreProcess(
-      const Flags& _flags,
-      Owned<MetadataManager> _metadataManager)
-    : flags(_flags), metadataManager(_metadataManager) {}
-
-  process::Future<Nothing> untarImage(
-      const std::string& tarPath,
-      const std::string& staging);
-
-  process::Future<DockerImage> putImage(
-      const ImageName& name,
-      const std::string& staging);
-
-  Result<std::string> getParentId(
-      const std::string& staging,
-      const std::string& layerId);
-
-  process::Future<Nothing> putLayers(
-      const std::string& staging,
-      const std::list<std::string>& layerIds);
-
-  process::Future<Nothing> putLayer(
-      const std::string& staging,
-      const std::string& id);
-
-  process::Future<Nothing> moveLayer(
-      const std::string& staging,
-      const std::string& id);
-
-  const Flags flags;
-  process::Owned<MetadataManager> metadataManager;
-};
-
-
-Try<Owned<Store>> Store::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  hashmap<string, Try<Owned<Store>>(*)(const Flags&, Fetcher*)> creators{
-    {"local", &LocalStore::create}
-  };
-
-  if (!creators.contains(flags.docker_store)) {
-    return Error("Unknown Docker store: " + flags.docker_store);
-  }
-
-  return creators[flags.docker_store](flags, fetcher);
-}
-
-
-Try<Owned<Store>> LocalStore::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  Try<Owned<LocalStoreProcess>> process =
-    LocalStoreProcess::create(flags, fetcher);
-  if (process.isError()) {
-    return Error(process.error());
-  }
-
-  return Owned<Store>(new LocalStore(process.get()));
-}
-
-
-LocalStore::LocalStore(Owned<LocalStoreProcess> _process) : process(_process)
-{
-  process::spawn(CHECK_NOTNULL(process.get()));
-}
-
-
-LocalStore::~LocalStore()
-{
-  process::terminate(process.get());
-  process::wait(process.get());
-}
-
-
-Future<DockerImage> LocalStore::get(const ImageName& name)
-{
-  return dispatch(process.get(), &LocalStoreProcess::get, name);
-}
-
-
-Future<Nothing> LocalStore::recover()
-{
-  return dispatch(process.get(), &LocalStoreProcess::recover);
-}
-
-
-Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  if (!os::exists(flags.docker_store_dir)) {
-    Try<Nothing> mkdir = os::mkdir(flags.docker_store_dir);
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker store directory: " + mkdir.error());
-    }
-  }
-
-  if (!os::exists(paths::getStagingDir(flags.docker_store_dir))) {
-    Try<Nothing> mkdir =
-      os::mkdir(paths::getStagingDir(flags.docker_store_dir));
-    if (mkdir.isError()) {
-      return Error("Failed to create Docker store staging directory: " +
-                   mkdir.error());
-    }
-  }
-
-  Try<Owned<MetadataManager>> metadataManager = MetadataManager::create(flags);
-  if (metadataManager.isError()) {
-    return Error(metadataManager.error());
-  }
-
-  return Owned<LocalStoreProcess>(
-      new LocalStoreProcess(flags, metadataManager.get()));
-}
-
-
-Future<DockerImage> LocalStoreProcess::get(const ImageName& name)
-{
-  return metadataManager->get(name)
-    .then(defer(self(),
-                [this, name](
-                    const Option<DockerImage>& image) -> Future<DockerImage> {
-      if (image.isSome()) {
-        return image.get();
-      }
-
-      string tarPath = paths::getLocalImageTarPath(
-          flags.docker_discovery_local_dir,
-          name.name());
-
-      if (!os::exists(tarPath)) {
-        VLOG(1) << "Unable to find image in local store with path: " << tarPath;
-        return Failure("No Docker image tar archive found");
-      }
-
-      // Create a temporary staging directory.
-      Try<string> staging =
-        os::mkdtemp(paths::getTempStaging(flags.docker_store_dir));
-      if (staging.isError()) {
-        return Failure("Failed to create a staging directory");
-      }
-
-      return untarImage(tarPath, staging.get())
-        .then(defer(self(), &Self::putImage, name, staging.get()));
-    }));
-}
-
-
-Future<Nothing> LocalStoreProcess::recover()
-{
-  return metadataManager->recover();
-}
-
-Future<Nothing> LocalStoreProcess::untarImage(
-    const string& tarPath,
-    const string& staging)
-{
-  VLOG(1) << "Untarring image at: " << tarPath;
-
-  // Untar discovery_local_dir/name.tar into staging/.
-  vector<string> argv = {
-    "tar",
-    "-C",
-    staging,
-    "-x",
-    "-f",
-    tarPath
-  };
-
-  Try<Subprocess> s = subprocess(
-      "tar",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"));
-
-  if (s.isError()) {
-    return Failure("Failed to create tar subprocess: " + s.error());
-  }
-
-  return s.get().status()
-    .then([=](const Option<int>& status) -> Future<Nothing> {
-      if (status.isNone()) {
-        return Failure("Failed to reap status for tar subprocess in " +
-                        tarPath);
-      }
-      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-          return Failure("Untar image failed with exit code: " +
-                          WSTRINGIFY(status.get()));
-      }
-
-      return Nothing();
-    });
-}
-
-
-Future<DockerImage> LocalStoreProcess::putImage(
-    const ImageName& name,
-    const string& staging)
-{
-  Try<string> value = os::read(paths::getLocalImageRepositoriesPath(staging));
-  if (value.isError()) {
-    return Failure("Failed to read repository JSON: " + value.error());
-  }
-
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
-  if (json.isError()) {
-    return Failure("Failed to parse JSON: " + json.error());
-  }
-
-  Result<JSON::Object> repositoryValue =
-    json.get().find<JSON::Object>(name.repository);
-  if (repositoryValue.isError()) {
-    return Failure("Failed to find repository: " + repositoryValue.error());
-  } else if (repositoryValue.isNone()) {
-    return Failure("Repository '" + name.repository + "' is not found");
-  }
-
-  JSON::Object repositoryJson = repositoryValue.get();
-
-  // We don't use JSON find here because a tag might contain a '.'.
-  std::map<string, JSON::Value>::const_iterator entry =
-    repositoryJson.values.find(name.tag);
-  if (entry == repositoryJson.values.end()) {
-    return Failure("Tag '" + name.tag + "' is not found");
-  } else if (!entry->second.is<JSON::String>()) {
-    return Failure("Tag JSON value expected to be JSON::String");
-  }
-
-  string layerId = entry->second.as<JSON::String>().value;
-
-  Try<string> manifest =
-    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
-  if (manifest.isError()) {
-    return Failure("Failed to read manifest: " + manifest.error());
-  }
-
-  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
-  if (manifestJson.isError()) {
-    return Failure("Failed to parse manifest: " + manifestJson.error());
-  }
-
-  list<string> layerIds;
-  layerIds.push_back(layerId);
-  Result<string> parentId = getParentId(staging, layerId);
-  while(parentId.isSome()) {
-    layerIds.push_front(parentId.get());
-    parentId = getParentId(staging, parentId.get());
-  }
-  if (parentId.isError()) {
-    return Failure("Failed to obtain parent layer id: " + parentId.error());
-  }
-
-  return putLayers(staging, layerIds)
-    .then([=]() -> Future<DockerImage> {
-      return metadataManager->put(name, layerIds);
-    });
-}
-
-
-Result<string> LocalStoreProcess::getParentId(
-    const string& staging,
-    const string& layerId)
-{
-  Try<string> manifest =
-    os::read(paths::getLocalImageLayerManifestPath(staging, layerId));
-  if (manifest.isError()) {
-    return Error("Failed to read manifest: " + manifest.error());
-  }
-
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
-  if (json.isError()) {
-    return Error("Failed to parse manifest: " + json.error());
-  }
-
-  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
-  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
-    return None();
-  } else if (parentId.isError()) {
-    return Error("Failed to read parent of layer: " + parentId.error());
-  }
-  return parentId.get().value;
-}
-
-
-Future<Nothing> LocalStoreProcess::putLayers(
-    const string& staging,
-    const list<string>& layerIds)
-{
-  list<Future<Nothing>> futures{ Nothing() };
-  foreach (const string& layer, layerIds) {
-    futures.push_back(
-        futures.back().then(
-          defer(self(), &Self::putLayer, staging, layer)));
-  }
-
-  return collect(futures)
-    .then([]() -> Future<Nothing> { return Nothing(); });
-}
-
-
-Future<Nothing> LocalStoreProcess::putLayer(
-    const string& staging,
-    const string& id)
-{
-  // We untar the layer from source into a staging directory, then
-  // move the layer into store. We do this instead of untarring
-  // directly to store to make sure we don't end up with partially
-  // untarred layer rootfs.
-
-  // Check if image layer rootfs is already in store.
-  if (os::exists(paths::getImageLayerRootfsPath(flags.docker_store_dir, id))) {
-    VLOG(1) << "Image layer '" << id << "' rootfs already in store. "
-            << "Skipping put.";
-    return Nothing();
-  }
-
-  const string imageLayerPath =
-    paths::getImageLayerPath(flags.docker_store_dir, id);
-  if (!os::exists(imageLayerPath)) {
-    Try<Nothing> mkdir = os::mkdir(imageLayerPath);
-    if (mkdir.isError()) {
-      return Failure("Failed to create Image layer directory '" +
-                     imageLayerPath + "': " + mkdir.error());
-    }
-  }
-
-  // Image layer has been untarred but is not present in the store directory.
-  string localRootfsPath = paths::getLocalImageLayerRootfsPath(staging, id);
-  if (os::exists(localRootfsPath)) {
-    LOG(WARNING) << "Image layer '" << id << "' rootfs present at but not in "
-                 << "store directory '" << localRootfsPath << "'. Removing "
-                 << "staged rootfs and untarring layer again.";
-    Try<Nothing> rmdir = os::rmdir(localRootfsPath);
-    if (rmdir.isError()) {
-      return Failure("Failed to remove incomplete staged rootfs for layer '" +
-                     id + "': " + rmdir.error());
-    }
-  }
-
-  Try<Nothing> mkdir = os::mkdir(localRootfsPath);
-  if (mkdir.isError()) {
-    return Failure("Failed to create rootfs path '" + localRootfsPath + "': " +
-                   mkdir.error());
-  }
-  // Untar staging/id/layer.tar into staging/id/rootfs.
-  vector<string> argv = {
-    "tar",
-    "-C",
-    localRootfsPath,
-    "-x",
-    "-f",
-    paths::getLocalImageLayerTarPath(staging, id)
-  };
-
-  Try<Subprocess> s = subprocess(
-      "tar",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"));
-  if (s.isError()) {
-    return Failure("Failed to create tar subprocess: " + s.error());
-  }
-
-  return s.get().status()
-    .then([=](const Option<int>& status) -> Future<Nothing> {
-      if (status.isNone()) {
-        return Failure("Failed to reap subprocess to untar image");
-      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-        return Failure("Untar failed with exit code: " +
-                        WSTRINGIFY(status.get()));
-      }
-
-      return moveLayer(staging, id);
-    });
-}
-
-
-Future<Nothing> LocalStoreProcess::moveLayer(
-    const string& staging,
-    const string& id)
-{
-  Try<Nothing> status = os::rename(
-      paths::getLocalImageLayerRootfsPath(staging, id),
-      paths::getImageLayerRootfsPath(flags.docker_store_dir, id));
-
-  if (status.isError()) {
-    return Failure("Failed to move layer to store directory: "
-                   + status.error());
-  }
-
-  return Nothing();
-}
-
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4dedbf4a/src/slave/containerizer/provisioners/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.hpp b/src/slave/containerizer/provisioners/docker/local_store.hpp
deleted file mode 100644
index 64a6fc0..0000000
--- a/src/slave/containerizer/provisioners/docker/local_store.hpp
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * 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_LOCAL_STORE_HPP__
-#define __MESOS_DOCKER_LOCAL_STORE_HPP__
-
-#include "slave/containerizer/provisioners/docker/store.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-// Forward declaration.
-class LocalStoreProcess;
-
-
-/**
- * LocalStore assumes Docker images are stored in a local directory
- * (configured with flags.docker_discovery_local_dir), with all the
- * images saved as tar with the name as the image name with tag (e.g:
- * ubuntu:14.04.tar).
- */
-class LocalStore : public Store
-{
-public:
-  static Try<process::Owned<Store>> create(
-      const Flags& flags,
-      Fetcher* fetcher);
-
-  virtual ~LocalStore();
-
-  virtual process::Future<DockerImage> get(const ImageName& name);
-
-  virtual process::Future<Nothing> recover();
-
-private:
-  explicit LocalStore(process::Owned<LocalStoreProcess> _process);
-
-  LocalStore& operator=(const LocalStore&) = delete; // Not assignable.
-  LocalStore(const LocalStore&) = delete; // Not copyable.
-
-  process::Owned<LocalStoreProcess> process;
-};
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MESOS_DOCKER_LOCAL_STORE_HPP__


[02/17] mesos git commit: Docker provisioner local store unit tests.

Posted by tn...@apache.org.
Docker provisioner local store unit tests.

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


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

Branch: refs/heads/master
Commit: 85c1cd3a51405ae7481f62b834ca76644c6f882a
Parents: e8906a1
Author: Lily Chen <li...@mesosphere.io>
Authored: Thu Aug 13 11:23:36 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   1 +
 .../containerizer/provisioner_docker_tests.cpp  | 164 +++++++++++++++++++
 2 files changed, 165 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/85c1cd3a/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 0b49a3b..40a6427 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1770,6 +1770,7 @@ mesos_tests_DEPENDENCIES = # Initialized to allow += below.
 if OS_LINUX
   mesos_tests_SOURCES += tests/containerizer/cgroups_isolator_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/cgroups_tests.cpp
+  mesos_tests_SOURCES += tests/containerizer/docker_provisioner_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/filesystem_isolator_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/fs_tests.cpp
   mesos_tests_SOURCES += tests/containerizer/launch_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/85c1cd3a/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index 1b0c304..3927009 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -21,6 +21,12 @@
 
 #include <stout/duration.hpp>
 
+#include <stout/gtest.hpp>
+#include <stout/json.hpp>
+#include <stout/os.hpp>
+#include <stout/path.hpp>
+#include <stout/stringify.hpp>
+
 #include <process/address.hpp>
 #include <process/clock.hpp>
 #include <process/future.hpp>
@@ -35,6 +41,7 @@
 #include "slave/containerizer/provisioner/docker/token_manager.hpp"
 
 #include "tests/mesos.hpp"
+#include "tests/utils.hpp"
 
 using std::map;
 using std::string;
@@ -48,6 +55,10 @@ using process::Owned;
 
 using process::network::Socket;
 
+using namespace process;
+using namespace mesos::internal::slave;
+using namespace mesos::internal::slave::docker;
+
 using ManifestResponse = RegistryClient::ManifestResponse;
 
 namespace mesos {
@@ -683,6 +694,159 @@ TEST_F(RegistryClientTest, BadRequest)
 
 #endif // USE_SSL_SOCKET
 
+class DockerProvisionerLocalStoreTest : public TemporaryDirectoryTest
+{
+public:
+  void verifyLocalDockerImage(
+      const slave::Flags& flags,
+      const DockerImage& dockerImage)
+  {
+    // Verify contents of the image in store directory.
+    EXPECT_TRUE(
+        os::exists(path::join(flags.docker_store_dir, "123", "rootfs")));
+    EXPECT_TRUE(
+        os::exists(path::join(flags.docker_store_dir, "456", "rootfs")));
+    EXPECT_SOME_EQ(
+        "foo 123",
+        os::read(path::join(flags.docker_store_dir, "123", "rootfs" , "temp")));
+    EXPECT_SOME_EQ(
+        "bar 456",
+        os::read(path::join(flags.docker_store_dir, "456", "rootfs", "temp")));
+
+    // Verify the docker Image provided.
+    EXPECT_EQ(dockerImage.imageName, "abc");
+    list<string> expectedLayers;
+    expectedLayers.push_back("123");
+    expectedLayers.push_back("456");
+    EXPECT_EQ(dockerImage.layers, expectedLayers);
+  }
+
+protected:
+  virtual void SetUp()
+  {
+    TemporaryDirectoryTest::SetUp();
+
+    string imageDir = path::join(os::getcwd(), "images");
+    string image = path::join(imageDir, "abc");
+    ASSERT_SOME(os::mkdir(imageDir));
+    ASSERT_SOME(os::mkdir(image));
+
+    JSON::Value repositories = JSON::parse(
+      "{"
+      "  \"abc\": {"
+      "    \"latest\": \"456\""
+      "  }"
+      "}").get();
+    ASSERT_SOME(
+        os::write(path::join(image, "repositories"), stringify(repositories)));
+
+    ASSERT_SOME(os::mkdir(path::join(image, "123")));
+    JSON::Value manifest123 = JSON::parse(
+      "{"
+      "  \"parent\": \"\""
+      "}").get();
+    ASSERT_SOME(os::write(
+        path::join(image, "123", "json"), stringify(manifest123)));
+    ASSERT_SOME(os::mkdir(path::join(image, "123", "layer")));
+    ASSERT_SOME(
+        os::write(path::join(image, "123", "layer", "temp"), "foo 123"));
+
+    // Must change directory to avoid carrying over /path/to/archive during tar.
+    const string cwd = os::getcwd();
+    ASSERT_SOME(os::chdir(path::join(image, "123", "layer")));
+    ASSERT_SOME(os::tar(".", "../layer.tar"));
+    ASSERT_SOME(os::chdir(cwd));
+    ASSERT_SOME(os::rmdir(path::join(image, "123", "layer")));
+
+    ASSERT_SOME(os::mkdir(path::join(image, "456")));
+    JSON::Value manifest456 = JSON::parse(
+      "{"
+      "  \"parent\": \"123\""
+      "}").get();
+    ASSERT_SOME(
+        os::write(path::join(image, "456", "json"), stringify(manifest456)));
+    ASSERT_SOME(os::mkdir(path::join(image, "456", "layer")));
+    ASSERT_SOME(
+        os::write(path::join(image, "456", "layer", "temp"), "bar 456"));
+
+    ASSERT_SOME(os::chdir(path::join(image, "456", "layer")));
+    ASSERT_SOME(os::tar(".", "../layer.tar"));
+    ASSERT_SOME(os::chdir(cwd));
+    ASSERT_SOME(os::rmdir(path::join(image, "456", "layer")));
+
+    ASSERT_SOME(os::chdir(image));
+    ASSERT_SOME(os::tar(".", "../abc.tar"));
+    ASSERT_SOME(os::chdir(cwd));
+    ASSERT_SOME(os::rmdir(image));
+  }
+};
+
+// This test verifies that a locally stored Docker image in the form of a
+// tar achive created from a 'docker save' command can be unpacked and
+// stored in the proper locations accessible to the Docker provisioner.
+TEST_F(DockerProvisionerLocalStoreTest, LocalStoreTestWithTar)
+{
+  string imageDir = path::join(os::getcwd(), "images");
+  string image = path::join(imageDir, "abc");
+  ASSERT_SOME(os::mkdir(imageDir));
+  ASSERT_SOME(os::mkdir(image));
+
+  slave::Flags flags;
+  flags.docker_store = "local";
+  flags.docker_store_dir = path::join(os::getcwd(), "store");
+  flags.docker_discovery_local_dir = imageDir;
+
+  // Fetcher is not relevant to local store. It is passed through from the
+  // provisioner interface.
+  Fetcher fetcher;
+  Try<Owned<Store>> store = Store::create(flags, &fetcher);
+  ASSERT_SOME(store);
+
+  string sandbox = path::join(os::getcwd(), "sandbox");
+  ASSERT_SOME(os::mkdir(sandbox));
+  Future<DockerImage> dockerImage = store.get()->put("abc", sandbox);
+  AWAIT_READY(dockerImage);
+
+  verifyLocalDockerImage(flags, dockerImage.get());
+
+  Future<Option<DockerImage>> dockerImageOption = store.get()->get("abc");
+  AWAIT_READY(dockerImageOption);
+  ASSERT_SOME(dockerImageOption.get());
+  verifyLocalDockerImage(flags, dockerImageOption.get().get());
+}
+
+// This tests the ability of the reference store to recover the images it has
+// already stored on disk when it is initialized.
+TEST_F(DockerProvisionerLocalStoreTest, ReferenceStoreInitialization)
+{
+  slave::Flags flags;
+  flags.docker_store = "local";
+  flags.docker_store_dir = path::join(os::getcwd(), "store");
+  flags.docker_discovery_local_dir = path::join(os::getcwd(), "images");
+
+  // Fetcher is not relevant to local store. It is passed through from the
+  // provisioner interface.
+  Fetcher fetcher;
+  Try<Owned<Store>> store = Store::create(flags, &fetcher);
+  ASSERT_SOME(store);
+
+  string sandbox = path::join(os::getcwd(), "sandbox");
+  ASSERT_SOME(os::mkdir(sandbox));
+  Future<DockerImage> dockerImage = store.get()->put("abc", sandbox);
+  AWAIT_READY(dockerImage);
+
+  // Store is deleted and recreated. Reference Store is initialized upon
+  // creation of the store.
+  store.get().reset();
+  store = Store::create(flags, &fetcher);
+  ASSERT_SOME(store);
+
+  Future<Option<DockerImage>> dockerImageOption = store.get()->get("abc");
+  AWAIT_READY(dockerImageOption);
+  ASSERT_SOME(dockerImageOption.get());
+  verifyLocalDockerImage(flags, dockerImageOption.get().get());
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


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

Posted by tn...@apache.org.
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(


[17/17] mesos git commit: Refactor docker provisioner store to act as read-through cache.

Posted by tn...@apache.org.
Refactor docker provisioner store to act as read-through cache.

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


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

Branch: refs/heads/master
Commit: f96e0dfc0d4a3f4100d3f9bc6c087dcd4c0b8c6a
Parents: 19d787b
Author: Timothy Chen <tn...@apache.org>
Authored: Tue Sep 1 15:02:36 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/provisioners/docker.cpp | 61 ++++++++++----
 src/slave/containerizer/provisioners/docker.hpp | 42 +---------
 .../provisioners/docker/local_store.cpp         | 84 +++++---------------
 .../containerizer/provisioners/docker/store.hpp | 19 +----
 src/tests/containerizer/provisioner.hpp         |  8 +-
 5 files changed, 72 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f96e0dfc/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
index 888f17a..bac29d3 100644
--- a/src/slave/containerizer/provisioners/docker.cpp
+++ b/src/slave/containerizer/provisioners/docker.cpp
@@ -47,6 +47,45 @@ namespace internal {
 namespace slave {
 namespace docker {
 
+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);
+
+  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;
+};
+
+
 ImageName::ImageName(const std::string& name)
 {
   registry = None();
@@ -107,15 +146,13 @@ Future<Nothing> DockerProvisioner::recover(
 
 Future<string> DockerProvisioner::provision(
     const ContainerID& containerId,
-    const Image& image,
-    const std::string& sandbox)
+    const Image& image)
 {
   return dispatch(
       process.get(),
       &DockerProvisionerProcess::provision,
       containerId,
-      image,
-      sandbox);
+      image);
 }
 
 
@@ -173,8 +210,7 @@ Future<Nothing> DockerProvisionerProcess::recover(
 
 Future<string> DockerProvisionerProcess::provision(
     const ContainerID& containerId,
-    const Image& image,
-    const string& sandbox)
+    const Image& image)
 {
   if (image.type() != Image::DOCKER) {
     return Failure("Unsupported container image type");
@@ -184,7 +220,7 @@ Future<string> DockerProvisionerProcess::provision(
     return Failure("Missing Docker image info");
   }
 
-  return fetch(image.docker().name(), sandbox)
+  return fetch(image.docker().name())
     .then(defer(self(),
                 &Self::_provision,
                 containerId,
@@ -234,16 +270,9 @@ Future<string> DockerProvisionerProcess::_provision(
 
 // Fetch an image and all dependencies.
 Future<DockerImage> DockerProvisionerProcess::fetch(
-    const string& name,
-    const string& sandbox)
+    const string& name)
 {
-  return store->get(name)
-    .then([=](const Option<DockerImage>& image) -> Future<DockerImage> {
-      if (image.isSome()) {
-        return image.get();
-      }
-      return store->put(name, sandbox);
-    });
+  return store->get(name);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f96e0dfc/src/slave/containerizer/provisioners/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.hpp b/src/slave/containerizer/provisioners/docker.hpp
index 9cca662..cda83cb 100644
--- a/src/slave/containerizer/provisioners/docker.hpp
+++ b/src/slave/containerizer/provisioners/docker.hpp
@@ -112,8 +112,7 @@ public:
 
   virtual process::Future<std::string> provision(
       const ContainerID& containerId,
-      const Image& image,
-      const std::string& sandbox);
+      const Image& image);
 
   virtual process::Future<bool> destroy(const ContainerID& containerId);
 
@@ -126,45 +125,6 @@ private:
 };
 
 
-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 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f96e0dfc/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
index d46f4bc..58af655 100644
--- a/src/slave/containerizer/provisioners/docker/local_store.cpp
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -60,11 +60,7 @@ public:
       const Flags& flags,
       Fetcher* fetcher);
 
-  process::Future<DockerImage> put(
-      const std::string& name,
-      const std::string& sandbox);
-
-  process::Future<Option<DockerImage>> get(const std::string& name);
+  process::Future<DockerImage> get(const std::string& name);
 
 private:
   LocalStoreProcess(const Flags& flags);
@@ -75,8 +71,7 @@ private:
 
   process::Future<DockerImage> putImage(
       const std::string& name,
-      const std::string& staging,
-      const std::string& sandbox);
+      const std::string& staging)
 
   Result<std::string> getParentId(
       const std::string& staging,
@@ -84,18 +79,15 @@ private:
 
   process::Future<Nothing> putLayers(
       const std::string& staging,
-      const std::list<std::string>& layers,
-      const std::string& sandbox);
+      const std::list<std::string>& layers)
 
   process::Future<Nothing> untarLayer(
       const std::string& staging,
-      const std::string& id,
-      const std::string& sandbox);
+      const std::string& id)
 
   process::Future<Nothing> moveLayer(
       const std::string& staging,
-      const std::string& id,
-      const std::string& sandbox);
+      const std::string& id)
 
   const Flags flags;
 
@@ -147,14 +139,6 @@ LocalStore::~LocalStore()
 }
 
 
-Future<DockerImage> LocalStore::put(
-    const string& name,
-    const string& sandbox)
-{
-  return dispatch(process.get(), &LocalStoreProcess::put, name, sandbox);
-}
-
-
 Future<Option<DockerImage>> LocalStore::get(const string& name)
 {
   return dispatch(process.get(), &LocalStoreProcess::get, name);
@@ -173,10 +157,13 @@ LocalStoreProcess::LocalStoreProcess(const Flags& flags)
   : flags(flags), refStore(ReferenceStore::create(flags).get()) {}
 
 
-Future<DockerImage> LocalStoreProcess::put(
-    const string& name,
-    const string& sandbox)
+Future<DockerImage> LocalStoreProcess::get(const string& name)
 {
+  Option<DockerImage> image = refStore->get(name);
+  if (image.isSome()) {
+    return image.get();
+  }
+
   string tarPath =
     paths::getLocalImageTarPath(flags.docker_discovery_local_dir, name);
   if (!os::exists(tarPath)) {
@@ -195,7 +182,7 @@ Future<DockerImage> LocalStoreProcess::put(
   }
 
   return untarImage(tarPath, staging.get())
-    .then(defer(self(), &Self::putImage, name, staging.get(), sandbox));
+    .then(defer(self(), &Self::putImage, name, staging.get()));
 }
 
 
@@ -244,8 +231,7 @@ Future<Nothing> LocalStoreProcess::untarImage(
 
 Future<DockerImage> LocalStoreProcess::putImage(
     const std::string& name,
-    const string& staging,
-    const string& sandbox)
+    const string& staging)
 {
   ImageName imageName(name);
 
@@ -302,7 +288,7 @@ Future<DockerImage> LocalStoreProcess::putImage(
     return Failure("Failed to obtain parent layer id: " + parentId.error());
   }
 
-  return putLayers(staging, layers, sandbox)
+  return putLayers(staging, layers)
     .then([=]() -> Future<DockerImage> {
       return refStore->put(name, layers);
     });
@@ -336,14 +322,13 @@ Result<string> LocalStoreProcess::getParentId(
 
 Future<Nothing> LocalStoreProcess::putLayers(
     const string& staging,
-    const list<string>& layers,
-    const string& sandbox)
+    const list<string>& layers)
 {
   list<Future<Nothing>> futures{ Nothing() };
   foreach (const string& layer, layers) {
     futures.push_back(
         futures.back().then(
-          defer(self(), &Self::untarLayer, staging, layer, sandbox)));
+          defer(self(), &Self::untarLayer, staging, layer)));
   }
 
   return collect(futures)
@@ -353,8 +338,7 @@ Future<Nothing> LocalStoreProcess::putLayers(
 
 Future<Nothing> LocalStoreProcess::untarLayer(
     const string& staging,
-    const string& id,
-    const string& sandbox)
+    const string& id)
 {
   // Check if image layer is already in store.
   if (os::exists(paths::getImageLayerPath(flags.docker_store_dir, id))) {
@@ -368,7 +352,7 @@ Future<Nothing> LocalStoreProcess::untarLayer(
   if (os::exists(localRootfsPath)) {
     LOG(WARNING) << "Image layer rootfs present at but not in store directory: "
                  << localRootfsPath << "Skipping untarLayer.";
-    return moveLayer(staging, id, sandbox);
+    return moveLayer(staging, id);
   }
 
   os::mkdir(localRootfsPath);
@@ -401,36 +385,15 @@ Future<Nothing> LocalStoreProcess::untarLayer(
                         WSTRINGIFY(status.get()));
       }
 
-      return moveLayer(staging, id, sandbox);
+      return moveLayer(staging, id);
     });
 }
 
 
 Future<Nothing> LocalStoreProcess::moveLayer(
     const string& staging,
-    const string& id,
-    const string& sandbox){
-
-  Try<int> out = os::open(
-      path::join(sandbox, "stdout"),
-      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
-      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
-
-  if (out.isError()) {
-    return Failure("Failed to create 'stdout' file: " + out.error());
-  }
-
-  // Repeat for stderr.
-  Try<int> err = os::open(
-      path::join(sandbox, "stderr"),
-      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
-      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
-
-  if (err.isError()) {
-    os::close(out.get());
-    return Failure("Failed to create 'stderr' file: " + err.error());
-  }
-
+    const string& id)
+{
   if (!os::exists(flags.docker_store_dir)) {
     VLOG(1) << "Creating docker store directory";
     os::mkdir(flags.docker_store_dir);
@@ -452,11 +415,6 @@ Future<Nothing> LocalStoreProcess::moveLayer(
 }
 
 
-Future<Option<DockerImage>> LocalStoreProcess::get(const string& name)
-{
-  return refStore->get(name);
-}
-
 } // namespace docker {
 } // namespace slave {
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f96e0dfc/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index 2eda083..0520a2c 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -49,28 +49,13 @@ public:
   virtual ~Store() {}
 
   /**
-   * Put an image into the store. Returns the DockerImage containing
-   * the manifest, hash of the image, and the path to the extracted
-   * image.
-   *
-   * @param name The name of the Docker image being stored.
-   * @param sandbox The path of the directory in which the stderr and
-   *     stdout logs will be placed.
-   *
-   * @return The DockerImage placed in the store.
-   */
-  virtual process::Future<DockerImage> put(
-      const std::string& name,
-      const std::string& sandbox) = 0;
-
-  /**
   * Get image by name.
   *
   * @param name The name of the Docker image to retrieve from store.
   *
-  * @return The DockerImage or none if image is not in the store.
+  * @return The DockerImage that holds the Docker layers.
   */
-  virtual process::Future<Option<DockerImage>> get(const std::string& name) = 0;
+  virtual process::Future<DockerImage> get(const std::string& name) = 0;
 
   // TODO(chenlily): Implement removing an image.
 

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


[08/17] mesos git commit: Support destroy and recover for Docker provisioner.

Posted by tn...@apache.org.
Support destroy and recover for Docker provisioner.


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

Branch: refs/heads/master
Commit: 59404cd3e08bc9c4c497655ad9cee2ac392247e2
Parents: c5537a1
Author: Timothy Chen <tn...@gmail.com>
Authored: Sat Sep 5 18:10:56 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:05 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/provisioners/docker.cpp | 197 ++++++++++++++++---
 1 file changed, 169 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/59404cd3/src/slave/containerizer/provisioners/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker.cpp b/src/slave/containerizer/provisioners/docker.cpp
index bac29d3..32e1a3b 100644
--- a/src/slave/containerizer/provisioners/docker.cpp
+++ b/src/slave/containerizer/provisioners/docker.cpp
@@ -35,6 +35,7 @@
 #include "slave/containerizer/provisioners/docker/store.hpp"
 
 using namespace process;
+using namespace mesos::internal::slave;
 
 using std::list;
 using std::string;
@@ -82,7 +83,16 @@ private:
   const Flags flags;
 
   process::Owned<Store> store;
-  process::Owned<mesos::internal::slave::Backend> backend;
+  hashmap<string, process::Owned<Backend>> backends;
+
+  struct Info
+  {
+    // Mappings: backend -> rootfsId -> rootfsPath.
+    hashmap<string, hashmap<string, string>> rootfses;
+  };
+
+  hashmap<ContainerID, Owned<Info>> infos;
+
 };
 
 
@@ -169,10 +179,32 @@ Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  Try<Nothing> mkdir = os::mkdir(flags.docker_rootfs_dir);
+  string _root =
+    slave::paths::getProvisionerDir(flags.work_dir, Image::DOCKER);
+
+  Try<Nothing> mkdir = os::mkdir(_root);
   if (mkdir.isError()) {
-    return Error("Failed to create provisioner rootfs directory '" +
-                 flags.docker_rootfs_dir + "': " + mkdir.error());
+    return Error("Failed to create provisioner root directory '" +
+                 _root + "': " + mkdir.error());
+  }
+
+  Result<string> root = os::realpath(_root);
+  if (root.isError()) {
+    return Error(
+        "Failed to resolve the realpath of provisioner root directory '" +
+        _root + "': " + root.error());
+  }
+
+  CHECK_SOME(root); // Can't be None since we just created it.
+
+  hashmap<string, Owned<Backend>> backends = Backend::create(flags);
+  if (backends.empty()) {
+    return Error("No usable Docker provisioner backend created");
+  }
+
+  if (!backends.contains(flags.docker_backend)) {
+    return Error("The specified Docker provisioner backend '" +
+                 flags.docker_backend + "'is unsupported");
   }
 
   Try<Owned<Store>> store = Store::create(flags, fetcher);
@@ -180,31 +212,117 @@ Try<Owned<DockerProvisionerProcess>> DockerProvisionerProcess::create(
     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]));
+          backends));
 }
 
 
 DockerProvisionerProcess::DockerProvisionerProcess(
     const Flags& _flags,
     const Owned<Store>& _store,
-    const Owned<mesos::internal::slave::Backend>& _backend)
+    const hashmap<string, Owned<Backend>>& _backends)
   : flags(_flags),
     store(_store),
-    backend(_backend) {}
+    backends(_backends) {}
 
 
 Future<Nothing> DockerProvisionerProcess::recover(
     const list<ContainerState>& states,
     const hashset<ContainerID>& orphans)
 {
-  return Nothing();
+  // TODO(tnachen): Consider merging this with
+  // AppcProvisionerProcess::recover.
+
+  // Register living containers, including the ones that do not
+  // provision Docker images.
+  hashset<ContainerID> alive;
+
+  foreach (const ContainerState& state, states) {
+    if (state.executor_info().has_container() &&
+        state.executor_info().container().type() == ContainerInfo::MESOS) {
+      alive.insert(state.container_id());
+    }
+  }
+
+  // List provisioned containers; recover living ones; destroy unknown orphans.
+  // Note that known orphan containers are recovered as well and they will
+  // be destroyed by the containerizer using the normal cleanup path. See
+  // MESOS-2367 for details.
+  Try<hashmap<ContainerID, string>> containers =
+    provisioners::paths::listContainers(root);
+
+  if (containers.isError()) {
+    return Failure("Failed to list the containers managed by Docker "
+                   "provisioner: " + containers.error());
+  }
+
+  // If no container has been launched the 'containers' directory will be empty.
+  foreachkey (const ContainerID& containerId, containers.get()) {
+    if (alive.contains(containerId) || orphans.contains(containerId)) {
+      Owned<Info> info = Owned<Info>(new Info());
+
+      Try<hashmap<string, hashmap<string, string>>> rootfses =
+        provisioners::paths::listContainerRootfses(root, containerId);
+
+      if (rootfses.isError()) {
+        return Failure("Unable to list rootfses belonged to container '" +
+                       containerId.value() + "': " + rootfses.error());
+      }
+
+      foreachkey (const string& backend, rootfses.get()) {
+        if (!backends.contains(backend)) {
+          return Failure("Found rootfses managed by an unrecognized backend: " +
+                         backend);
+        }
+
+        info->rootfses.put(backend, rootfses.get()[backend]);
+      }
+
+      VLOG(1) << "Recovered container " << containerId;
+      infos.put(containerId, info);
+
+      continue;
+    }
+
+    // Destroy (unknown) orphan container's rootfses.
+    Try<hashmap<string, hashmap<string, string>>> rootfses =
+      provisioners::paths::listContainerRootfses(root, containerId);
+
+    if (rootfses.isError()) {
+      return Failure("Unable to find rootfses for container '" +
+                     containerId.value() + "': " + rootfses.error());
+    }
+
+    foreachkey (const string& backend, rootfses.get()) {
+      if (!backends.contains(backend)) {
+        return Failure("Found rootfses managed by an unrecognized backend: " +
+                       backend);
+      }
+
+      foreachvalue (const string& rootfs, rootfses.get()[backend]) {
+        VLOG(1) << "Destroying orphan rootfs " << rootfs;
+
+        // Not waiting for the destruction and we don't care about
+        // the return value.
+        backends.get(backend).get()->destroy(rootfs)
+          .onFailed([rootfs](const std::string& error) {
+            LOG(WARNING) << "Failed to destroy orphan rootfs '" << rootfs
+                         << "': "<< error;
+          });
+      }
+    }
+  }
+
+  LOG(INFO) << "Recovered Docker provisioner rootfses";
+
+  return store->recover()
+    .then([]() -> Future<Nothing> {
+      LOG(INFO) << "Recovered Docker image store";
+      return Nothing();
+    });
 }
 
 
@@ -220,6 +338,17 @@ Future<string> DockerProvisionerProcess::provision(
     return Failure("Missing Docker image info");
   }
 
+  string rootfsId = UUID::random().toString();
+  string rootfs = provisioners::paths::getContainerRootfsDir(
+      root, containerId, flags.docker_backend, rootfsId);
+
+  if (!infos.contains(containerId)) {
+    infos.put(containerId, Owned<Info>(new Info()));
+  }
+
+  infos[containerId]->rootfses[flags.docker_backend].put(rootfsId, rootfs);
+
+
   return fetch(image.docker().name())
     .then(defer(self(),
                 &Self::_provision,
@@ -232,6 +361,8 @@ Future<string> DockerProvisionerProcess::_provision(
     const ContainerID& containerId,
     const DockerImage& image)
 {
+  CHECK(backends.contains(flags.docker_backend));
+
   // Create root directory.
   string base = path::join(flags.docker_rootfs_dir,
                            stringify(containerId));
@@ -252,7 +383,8 @@ Future<string> DockerProvisionerProcess::_provision(
     layerPaths.push_back(path::join(flags.docker_store_dir, layerId, "rootfs"));
   }
 
-  return backend->provision(layerPaths, base)
+
+  return backends[flags.docker_backend]->provision(layerPaths, base)
     .then([rootfs]() -> 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
@@ -279,31 +411,40 @@ Future<DockerImage> DockerProvisionerProcess::fetch(
 Future<bool> DockerProvisionerProcess::destroy(
     const ContainerID& containerId)
 {
-  string base = path::join(flags.docker_rootfs_dir, stringify(containerId));
+  // TODO(tnachen): Consider merging this with
+  // AppcProvisionerProcess::destroy.
+  if (!infos.contains(containerId)) {
+    LOG(INFO) << "Ignoring destroy request for unknown container: "
+              << containerId;
 
-  if (!os::exists(base)) {
     return false;
   }
 
-  LOG(INFO) << "Destroying container rootfs for container '"
-            << containerId << "'";
+  // Unregister the container first. If destroy() fails, we can rely on
+  // recover() to retry it later.
+  Owned<Info> info = infos[containerId];
+  infos.erase(containerId);
+
+  list<Future<bool>> futures;
+  foreachkey (const string& backend, info->rootfses) {
+    foreachvalue (const string& rootfs, info->rootfses[backend]) {
+      if (!backends.contains(backend)) {
+        return Failure("Cannot destroy rootfs '" + rootfs +
+                       "' provisioned by an unknown backend '" + backend + "'");
+      }
 
-  Try<fs::MountInfoTable> mountTable = fs::MountInfoTable::read();
-  if (mountTable.isError()) {
-    return Failure("Failed to read mount table: " + mountTable.error());
-  }
+      LOG(INFO) << "Destroying container rootfs for container '"
+                << containerId << "' at '" << rootfs << "'";
 
-  foreach (const fs::MountInfoTable::Entry& entry, mountTable.get().entries) {
-    if (strings::startsWith(entry.target, base)) {
-      Try<Nothing> unmount = fs::unmount(entry.target, MNT_DETACH);
-      if (unmount.isError()) {
-        return Failure("Failed to unmount mount table target: " +
-                        unmount.error());
-      }
+      futures.push_back(
+          backends.get(backend).get()->destroy(rootfs));
     }
   }
 
-  return backend->destroy(base);
+  return collect(futures)
+    .then([=](const list<bool>& results) -> Future<bool> {
+      return true;
+    });
 }
 
 } // namespace docker {


[06/17] mesos git commit: Move docker provisioner local store into dedicated folders.

Posted by tn...@apache.org.
Move docker provisioner local store into dedicated folders.

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


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

Branch: refs/heads/master
Commit: 31b62d6e6992dfd268543eb5c05ee3a19ac466ce
Parents: 85c1cd3
Author: Lily Chen <li...@mesosphere.io>
Authored: Fri Aug 14 13:20:20 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   3 +-
 .../provisioners/docker/local_store.cpp         | 469 +++++++++++++++++++
 .../provisioners/docker/local_store.hpp         |  65 +++
 .../containerizer/provisioners/docker/store.cpp | 413 ----------------
 .../containerizer/provisioners/docker/store.hpp |  89 ----
 5 files changed, 536 insertions(+), 503 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/31b62d6e/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 40a6427..0b7018c 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -700,8 +700,8 @@ if OS_LINUX
   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/local_store.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/reference_store.cpp
-  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/store.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
   EXTRA_DIST += linux/fs.cpp
@@ -827,6 +827,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backends/bind.hpp		\
 	slave/containerizer/provisioner/backends/copy.hpp		\
 	slave/containerizer/provisioner/docker.hpp			\
+	slave/containerizer/provisioners/docker/local_store.hpp         \
 	slave/containerizer/provisioner/docker/reference_store.hpp	\
 	slave/containerizer/provisioner/docker/registry_client.hpp	\
 	slave/containerizer/provisioner/docker/store.hpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/31b62d6e/src/slave/containerizer/provisioners/docker/local_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.cpp b/src/slave/containerizer/provisioners/docker/local_store.cpp
new file mode 100644
index 0000000..5f80b43
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/local_store.cpp
@@ -0,0 +1,469 @@
+/**
+ * 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/local_store.hpp"
+
+#include <list>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include <stout/json.hpp>
+#include <stout/os.hpp>
+#include <stout/result.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "slave/containerizer/fetcher.hpp"
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/provisioners/docker/store.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+class LocalStoreProcess : public process::Process<LocalStoreProcess>
+{
+public:
+  ~LocalStoreProcess() {}
+
+  static Try<process::Owned<LocalStoreProcess>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  process::Future<DockerImage> put(
+      const std::string& name,
+      const std::string& sandbox);
+
+  process::Future<Option<DockerImage>> get(const std::string& name);
+
+private:
+  LocalStoreProcess(const Flags& flags);
+
+  process::Future<Nothing> untarImage(
+      const std::string& tarPath,
+      const std::string& staging);
+
+  process::Future<DockerImage> putImage(
+      const std::string& name,
+      const std::string& staging,
+      const std::string& sandbox);
+
+  Result<std::string> getParentId(
+      const std::string& staging,
+      const std::string& layerId);
+
+  process::Future<Nothing> putLayers(
+      const std::string& staging,
+      const std::list<std::string>& layers,
+      const std::string& sandbox);
+
+  process::Future<Nothing> untarLayer(
+      const std::string& staging,
+      const std::string& id,
+      const std::string& sandbox);
+
+  process::Future<Nothing> moveLayer(
+      const std::string& staging,
+      const std::string& id,
+      const std::string& sandbox);
+
+  const Flags flags;
+
+  process::Owned<ReferenceStore> refStore;
+};
+
+
+Try<Owned<Store>> Store::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  hashmap<string, Try<Owned<Store>>(*)(const Flags&, Fetcher*)> creators{
+    {"local", &LocalStore::create}
+  };
+
+  if (!creators.contains(flags.docker_store)) {
+    return Error("Unknown Docker store: " + flags.docker_store);
+  }
+
+  return creators[flags.docker_store](flags, fetcher);
+}
+
+
+Try<Owned<Store>> LocalStore::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  Try<Owned<LocalStoreProcess>> process =
+    LocalStoreProcess::create(flags, fetcher);
+  if (process.isError()) {
+    return Error("Failed to create store: " + process.error());
+  }
+
+  return Owned<Store>(new LocalStore(process.get()));
+}
+
+
+LocalStore::LocalStore(Owned<LocalStoreProcess> process)
+  : process(process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+LocalStore::~LocalStore()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<DockerImage> LocalStore::put(
+    const string& name,
+    const string& sandbox)
+{
+  return dispatch(process.get(), &LocalStoreProcess::put, name, sandbox);
+}
+
+
+Future<Option<DockerImage>> LocalStore::get(const string& name)
+{
+  return dispatch(process.get(), &LocalStoreProcess::get, name);
+}
+
+
+Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
+    const Flags& flags,
+    Fetcher* fetcher)
+{
+  return Owned<LocalStoreProcess>(new LocalStoreProcess(flags));
+}
+
+
+LocalStoreProcess::LocalStoreProcess(const Flags& flags)
+  : flags(flags), refStore(ReferenceStore::create(flags).get()) {}
+
+
+Future<DockerImage> LocalStoreProcess::put(
+    const string& name,
+    const string& sandbox)
+{
+  string tarName = name + ".tar";
+  Try<string> tarPath = path::join(flags.docker_discovery_local_dir, tarName);
+  if (tarPath.isError()) {
+    return Failure(tarPath.error());
+  }
+  if (!os::exists(tarPath.get())) {
+    return Failure("No Docker image tar archive found: " + name);
+  }
+
+  // Create a temporary staging directory.
+  Try<string> staging = os::mkdtemp();
+  if (staging.isError()) {
+    return Failure("Failed to create a staging directory");
+  }
+
+  return untarImage(tarPath.get(), staging.get())
+    .then(defer(self(), &Self::putImage, name, staging.get(), sandbox));
+}
+
+
+Future<Nothing> LocalStoreProcess::untarImage(
+    const string& tarPath,
+    const string& staging)
+{
+  LOG(INFO) << "Untarring image at: " << tarPath;
+
+  // Untar discovery_local_dir/name.tar into staging/.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    staging,
+    "-x",
+    "-f",
+    tarPath
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([=](const Option<int>& status) -> Future<Nothing> {
+      if (status.isNone()) {
+        return Failure("Failed to reap status for tar subprocess in " +
+                        tarPath);
+      }
+      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+          return Failure("Untar image failed with exit code: " +
+                          WSTRINGIFY(status.get()));
+      }
+
+      return Nothing();
+    });
+}
+
+
+Future<DockerImage> LocalStoreProcess::putImage(
+    const std::string& name,
+    const string& staging,
+    const string& sandbox)
+{
+  ImageName imageName(name);
+  // Read repository json.
+  Try<string> repoPath = path::join(staging, "repositories");
+  if (repoPath.isError()) {
+    return Failure("Failed to create path to repository: " + repoPath.error());
+  }
+
+  Try<string> value = os::read(repoPath.get());
+  if (value.isError()) {
+    return Failure("Failed to read repository JSON: " + value.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
+  if (json.isError()) {
+    return Failure("Failed to parse JSON: " + json.error());
+  }
+
+  Result<JSON::Object> repositoryValue =
+    json.get().find<JSON::Object>(imageName.repo);
+  if (repositoryValue.isError()) {
+    return Failure("Failed to find repository: " + repositoryValue.error());
+  } else if (repositoryValue.isNone()) {
+    return Failure("Repository '" + imageName.repo + "' is not found");
+  }
+
+  JSON::Object repositoryJson = repositoryValue.get();
+
+  // We don't use JSON find here because a tag might contain a '.'.
+  std::map<string, JSON::Value>::const_iterator entry =
+    repositoryJson.values.find(imageName.tag);
+  if (entry == repositoryJson.values.end()) {
+    return Failure("Tag '" + imageName.tag + "' is not found");
+  } else if (!entry->second.is<JSON::String>()) {
+    return Failure("Tag JSON value expected to be JSON::String");
+  }
+
+  Try<string> layerPath = path::join(
+      staging,
+      entry->second.as<JSON::String>().value);
+  if (layerPath.isError()) {
+    return Failure("Failed to create path to image layer: " +
+                    layerPath.error());
+  }
+  string layerId = entry->second.as<JSON::String>().value;
+
+  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
+  if (manifest.isError()) {
+    return Failure("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
+  if (manifestJson.isError()) {
+    return Failure("Failed to parse manifest: " + manifestJson.error());
+  }
+
+  list<string> layers;
+  layers.push_back(layerId);
+  Result<string> parentId = getParentId(staging, layerId);
+  while(parentId.isSome()) {
+    layers.push_front(parentId.get());
+    parentId = getParentId(staging, parentId.get());
+  }
+  if (parentId.isError()) {
+    return Failure("Failed to obtain parent layer id: " + parentId.error());
+  }
+
+  return putLayers(staging, layers, sandbox)
+    .then([=]() -> Future<DockerImage> {
+      return refStore->put(name, layers);
+    });
+}
+
+
+Result<string> LocalStoreProcess::getParentId(
+    const string& staging,
+    const string& layerId)
+{
+  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
+  if (manifest.isError()) {
+    return Error("Failed to read manifest: " + manifest.error());
+  }
+
+  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
+  if (json.isError()) {
+    return Error("Failed to parse manifest: " + json.error());
+  }
+
+  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
+  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
+    return None();
+  } else if (parentId.isError()) {
+    return Error("Failed to read parent of layer: " + parentId.error());
+  }
+  return parentId.get().value;
+}
+
+
+Future<Nothing> LocalStoreProcess::putLayers(
+    const string& staging,
+    const list<string>& layers,
+    const string& sandbox)
+{
+  list<Future<Nothing>> futures{ Nothing() };
+  foreach (const string& layer, layers) {
+    futures.push_back(
+        futures.back().then(
+          defer(self(), &Self::untarLayer, staging, layer, sandbox)));
+  }
+
+  return collect(futures)
+    .then([]() -> Future<Nothing> { return Nothing(); });
+}
+
+
+Future<Nothing> LocalStoreProcess::untarLayer(
+    const string& staging,
+    const string& id,
+    const string& sandbox)
+{
+  // Check if image layer is already in store.
+  if (os::exists(path::join(flags.docker_store_dir, id))) {
+    VLOG(1) << "Image layer: " << id << " already in store. Skipping untar"
+            << " and putLayer.";
+    return Nothing();
+  }
+
+  // Image layer has been untarred but is not present in the store directory.
+  if (os::exists(path::join(staging, id, "rootfs"))) {
+    LOG(WARNING) << "Image layer rootfs present at but not in store directory: "
+                << path::join(staging, id) << "Skipping untarLayer.";
+    return moveLayer(staging, id, sandbox);
+  }
+
+  os::mkdir(path::join(staging, id, "rootfs"));
+  // Untar staging/id/layer.tar into staging/id/rootfs.
+  vector<string> argv = {
+    "tar",
+    "-C",
+    path::join(staging, id, "rootfs"),
+    "-x",
+    "-f",
+    path::join(staging, id, "layer.tar")
+  };
+
+  Try<Subprocess> s = subprocess(
+      "tar",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH("/dev/null"));
+  if (s.isError()) {
+    return Failure("Failed to create tar subprocess: " + s.error());
+  }
+
+  return s.get().status()
+    .then([=](const Option<int>& status) -> Future<Nothing> {
+      Try<string> layerPath = path::join(staging, id, "rootfs");
+      if (status.isNone()) {
+        return Failure("Failed to reap subprocess to untar image");
+      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+        return Failure("Untar image failed with exit code: " +
+                        WSTRINGIFY(status.get()));
+      }
+
+      return moveLayer(staging, id, sandbox);
+    });
+}
+
+
+Future<Nothing> LocalStoreProcess::moveLayer(
+    const string& staging,
+    const string& id,
+    const string& sandbox){
+
+  Try<int> out = os::open(
+      path::join(sandbox, "stdout"),
+      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
+      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
+
+  if (out.isError()) {
+    return Failure("Failed to create 'stdout' file: " + out.error());
+  }
+
+  // Repeat for stderr.
+  Try<int> err = os::open(
+      path::join(sandbox, "stderr"),
+      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
+      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
+
+  if (err.isError()) {
+    os::close(out.get());
+    return Failure("Failed to create 'stderr' file: " + err.error());
+  }
+
+  if (!os::exists(flags.docker_store_dir)) {
+    VLOG(1) << "Creating docker store directory";
+    os::mkdir(flags.docker_store_dir);
+  }
+
+  if (!os::exists(path::join(flags.docker_store_dir, id))) {
+    os::mkdir(path::join(flags.docker_store_dir, id));
+  }
+
+  Try<Nothing> status = os::rename(
+      path::join(staging, id, "rootfs"),
+      path::join(flags.docker_store_dir, id, "rootfs"));
+
+  if (status.isError()) {
+    return Failure("Failed to move layer to store directory:" + status.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<Option<DockerImage>> LocalStoreProcess::get(const string& name)
+{
+  return refStore->get(name);
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/31b62d6e/src/slave/containerizer/provisioners/docker/local_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/local_store.hpp b/src/slave/containerizer/provisioners/docker/local_store.hpp
new file mode 100644
index 0000000..41a3559
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/local_store.hpp
@@ -0,0 +1,65 @@
+/**
+ * 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_LOCAL_STORE__
+#define __MESOS_DOCKER_LOCAL_STORE__
+
+#include "slave/containerizer/provisioners/docker/store.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+class LocalStoreProcess;
+
+class LocalStore : public Store
+{
+public:
+  virtual ~LocalStore();
+
+  static Try<process::Owned<Store>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
+
+  /**
+   * Put assumes the image tar archive is located in the directory specified in
+   * the slave flag docker_discovery_local_dir and is named with <name>.tar .
+   */
+  virtual process::Future<DockerImage> put(
+      const std::string& name,
+      const std::string& sandbox);
+
+  virtual process::Future<Option<DockerImage>> get(const std::string& name);
+
+private:
+  explicit LocalStore(process::Owned<LocalStoreProcess> process);
+
+  LocalStore(const LocalStore&); // Not copyable.
+
+  LocalStore& operator=(const LocalStore&); // Not assignable.
+
+  process::Owned<LocalStoreProcess> process;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_LOCAL_STORE__

http://git-wip-us.apache.org/repos/asf/mesos/blob/31b62d6e/src/slave/containerizer/provisioners/docker/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.cpp b/src/slave/containerizer/provisioners/docker/store.cpp
deleted file mode 100644
index b902f8d..0000000
--- a/src/slave/containerizer/provisioners/docker/store.cpp
+++ /dev/null
@@ -1,413 +0,0 @@
-/**
- * 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/store.hpp"
-
-#include <list>
-
-#include <glog/logging.h>
-
-#include <stout/os.hpp>
-#include <stout/json.hpp>
-
-#include <process/collect.hpp>
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-
-#include "common/status_utils.hpp"
-
-#include "slave/containerizer/fetcher.hpp"
-#include "slave/flags.hpp"
-
-using namespace process;
-
-using std::list;
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace docker {
-
-Try<Owned<Store>> Store::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  hashmap<string, Try<Owned<Store>>(*)(const Flags&, Fetcher*)> creators{
-    {"local", &LocalStore::create}
-  };
-
-  if (!creators.contains(flags.docker_store)) {
-    return Error("Unknown Docker store: " + flags.docker_store);
-  }
-
-  return creators[flags.docker_store](flags, fetcher);
-}
-
-
-Try<Owned<Store>> LocalStore::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  Try<Owned<LocalStoreProcess>> process =
-    LocalStoreProcess::create(flags, fetcher);
-  if (process.isError()) {
-    return Error("Failed to create store: " + process.error());
-  }
-
-  return Owned<Store>(new LocalStore(process.get()));
-}
-
-
-LocalStore::LocalStore(Owned<LocalStoreProcess> process)
-  : process(process)
-{
-  process::spawn(CHECK_NOTNULL(process.get()));
-}
-
-
-LocalStore::~LocalStore()
-{
-  process::terminate(process.get());
-  process::wait(process.get());
-}
-
-
-Future<DockerImage> LocalStore::put(
-    const string& name,
-    const string& sandbox)
-{
-  return dispatch(process.get(), &LocalStoreProcess::put, name, sandbox);
-}
-
-
-Future<Option<DockerImage>> LocalStore::get(const string& name)
-{
-  return dispatch(process.get(), &LocalStoreProcess::get, name);
-}
-
-
-Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  return Owned<LocalStoreProcess>(new LocalStoreProcess(flags));
-}
-
-
-LocalStoreProcess::LocalStoreProcess(const Flags& flags)
-  : flags(flags), refStore(ReferenceStore::create(flags).get()) {}
-
-
-Future<DockerImage> LocalStoreProcess::put(
-    const string& name,
-    const string& sandbox)
-{
-  string tarName = name + ".tar";
-  Try<string> tarPath = path::join(flags.docker_discovery_local_dir, tarName);
-  if (tarPath.isError()) {
-    return Failure(tarPath.error());
-  }
-  if (!os::exists(tarPath.get())) {
-    return Failure("No Docker image tar archive found: " + name);
-  }
-
-  // Create a temporary staging directory.
-  Try<string> staging = os::mkdtemp();
-  if (staging.isError()) {
-    return Failure("Failed to create a staging directory");
-  }
-
-  return untarImage(tarPath.get(), staging.get())
-    .then(defer(self(), &Self::putImage, name, staging.get(), sandbox));
-}
-
-
-Future<Nothing> LocalStoreProcess::untarImage(
-    const string& tarPath,
-    const string& staging)
-{
-  LOG(INFO) << "Untarring image at: " << tarPath;
-
-  // Untar discovery_local_dir/name.tar into staging/.
-  vector<string> argv = {
-    "tar",
-    "-C",
-    staging,
-    "-x",
-    "-f",
-    tarPath
-  };
-
-  Try<Subprocess> s = subprocess(
-      "tar",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"));
-
-  if (s.isError()) {
-    return Failure("Failed to create tar subprocess: " + s.error());
-  }
-
-  return s.get().status()
-    .then([=](const Option<int>& status) -> Future<Nothing> {
-      if (status.isNone()) {
-        return Failure("Failed to reap status for tar subprocess in " +
-                        tarPath);
-      }
-      if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-          return Failure("Untar image failed with exit code: " +
-                          WSTRINGIFY(status.get()));
-      }
-
-      return Nothing();
-    });
-}
-
-
-Future<DockerImage> LocalStoreProcess::putImage(
-    const std::string& name,
-    const string& staging,
-    const string& sandbox)
-{
-  ImageName imageName(name);
-  // Read repository json.
-  Try<string> repoPath = path::join(staging, "repositories");
-  if (repoPath.isError()) {
-    return Failure("Failed to create path to repository: " + repoPath.error());
-  }
-
-  Try<string> value = os::read(repoPath.get());
-  if (value.isError()) {
-    return Failure("Failed to read repository JSON: " + value.error());
-  }
-
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(value.get());
-  if (json.isError()) {
-    return Failure("Failed to parse JSON: " + json.error());
-  }
-
-  Result<JSON::Object> repositoryValue =
-    json.get().find<JSON::Object>(imageName.repo);
-  if (repositoryValue.isError()) {
-    return Failure("Failed to find repository: " + repositoryValue.error());
-  } else if (repositoryValue.isNone()) {
-    return Failure("Repository '" + imageName.repo + "' is not found");
-  }
-
-  JSON::Object repositoryJson = repositoryValue.get();
-
-  // We don't use JSON find here because a tag might contain a '.'.
-  std::map<string, JSON::Value>::const_iterator entry =
-    repositoryJson.values.find(imageName.tag);
-  if (entry == repositoryJson.values.end()) {
-    return Failure("Tag '" + imageName.tag + "' is not found");
-  } else if (!entry->second.is<JSON::String>()) {
-    return Failure("Tag JSON value expected to be JSON::String");
-  }
-
-  Try<string> layerPath = path::join(
-      staging,
-      entry->second.as<JSON::String>().value);
-  if (layerPath.isError()) {
-    return Failure("Failed to create path to image layer: " +
-                    layerPath.error());
-  }
-  string layerId = entry->second.as<JSON::String>().value;
-
-  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
-  if (manifest.isError()) {
-    return Failure("Failed to read manifest: " + manifest.error());
-  }
-
-  Try<JSON::Object> manifestJson = JSON::parse<JSON::Object>(manifest.get());
-  if (manifestJson.isError()) {
-    return Failure("Failed to parse manifest: " + manifestJson.error());
-  }
-
-  list<string> layers;
-  layers.push_back(layerId);
-  Result<string> parentId = getParentId(staging, layerId);
-  while(parentId.isSome()) {
-    layers.push_front(parentId.get());
-    parentId = getParentId(staging, parentId.get());
-  }
-  if (parentId.isError()) {
-    return Failure("Failed to obtain parent layer id: " + parentId.error());
-  }
-
-  return putLayers(staging, layers, sandbox)
-    .then([=]() -> Future<DockerImage> {
-      return refStore->put(name, layers);
-    });
-}
-
-
-Result<string> LocalStoreProcess::getParentId(
-    const string& staging,
-    const string& layerId)
-{
-  Try<string> manifest = os::read(path::join(staging, layerId, "json"));
-  if (manifest.isError()) {
-    return Error("Failed to read manifest: " + manifest.error());
-  }
-
-  Try<JSON::Object> json = JSON::parse<JSON::Object>(manifest.get());
-  if (json.isError()) {
-    return Error("Failed to parse manifest: " + json.error());
-  }
-
-  Result<JSON::String> parentId = json.get().find<JSON::String>("parent");
-  if (parentId.isNone() || (parentId.isSome() && parentId.get() == "")) {
-    return None();
-  } else if (parentId.isError()) {
-    return Error("Failed to read parent of layer: " + parentId.error());
-  }
-  return parentId.get().value;
-}
-
-
-Future<Nothing> LocalStoreProcess::putLayers(
-    const string& staging,
-    const list<string>& layers,
-    const string& sandbox)
-{
-  list<Future<Nothing>> futures{ Nothing() };
-  foreach (const string& layer, layers) {
-    futures.push_back(
-        futures.back().then(
-          defer(self(), &Self::untarLayer, staging, layer, sandbox)));
-  }
-
-  return collect(futures)
-    .then([]() -> Future<Nothing> { return Nothing(); });
-}
-
-
-Future<Nothing> LocalStoreProcess::untarLayer(
-    const string& staging,
-    const string& id,
-    const string& sandbox)
-{
-  // Check if image layer is already in store.
-  if (os::exists(path::join(flags.docker_store_dir, id))) {
-    VLOG(1) << "Image layer: " << id << " already in store. Skipping untar"
-            << " and putLayer.";
-    return Nothing();
-  }
-
-  // Image layer has been untarred but is not present in the store directory.
-  if (os::exists(path::join(staging, id, "rootfs"))) {
-    LOG(WARNING) << "Image layer rootfs present at but not in store directory: "
-                << path::join(staging, id) << "Skipping untarLayer.";
-    return moveLayer(staging, id, sandbox);
-  }
-
-  os::mkdir(path::join(staging, id, "rootfs"));
-  // Untar staging/id/layer.tar into staging/id/rootfs.
-  vector<string> argv = {
-    "tar",
-    "-C",
-    path::join(staging, id, "rootfs"),
-    "-x",
-    "-f",
-    path::join(staging, id, "layer.tar")
-  };
-
-  Try<Subprocess> s = subprocess(
-      "tar",
-      argv,
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"));
-  if (s.isError()) {
-    return Failure("Failed to create tar subprocess: " + s.error());
-  }
-
-  return s.get().status()
-    .then([=](const Option<int>& status) -> Future<Nothing> {
-      Try<string> layerPath = path::join(staging, id, "rootfs");
-      if (status.isNone()) {
-        return Failure("Failed to reap subprocess to untar image");
-      } else if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
-        return Failure("Untar image failed with exit code: " +
-                        WSTRINGIFY(status.get()));
-      }
-
-      return moveLayer(staging, id, sandbox);
-    });
-}
-
-
-Future<Nothing> LocalStoreProcess::moveLayer(
-    const string& staging,
-    const string& id,
-    const string& sandbox){
-
-  Try<int> out = os::open(
-      path::join(sandbox, "stdout"),
-      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
-      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
-
-  if (out.isError()) {
-    return Failure("Failed to create 'stdout' file: " + out.error());
-  }
-
-  // Repeat for stderr.
-  Try<int> err = os::open(
-      path::join(sandbox, "stderr"),
-      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK | O_CLOEXEC,
-      S_IRUSR | S_IWUSR | S_IRGRP | S_IROTH);
-
-  if (err.isError()) {
-    os::close(out.get());
-    return Failure("Failed to create 'stderr' file: " + err.error());
-  }
-
-  if (!os::exists(flags.docker_store_dir)) {
-    VLOG(1) << "Creating docker store directory";
-    os::mkdir(flags.docker_store_dir);
-  }
-
-  if (!os::exists(path::join(flags.docker_store_dir, id))) {
-    os::mkdir(path::join(flags.docker_store_dir, id));
-  }
-
-  Try<Nothing> status = os::rename(
-      path::join(staging, id, "rootfs"),
-      path::join(flags.docker_store_dir, id, "rootfs"));
-
-  if (status.isError()) {
-    return Failure("Failed to move layer to store directory:" + status.error());
-  }
-
-  return Nothing();
-}
-
-
-Future<Option<DockerImage>> LocalStoreProcess::get(const string& name)
-{
-  return refStore->get(name);
-}
-
-} // namespace docker {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/31b62d6e/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index 256e146..2eda083 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -20,18 +20,13 @@
 #define __MESOS_DOCKER_STORE__
 
 #include <string>
-#include <vector>
 
 #include <stout/hashmap.hpp>
 #include <stout/nothing.hpp>
 #include <stout/option.hpp>
-#include <stout/result.hpp>
 #include <stout/try.hpp>
 
 #include <process/future.hpp>
-#include <process/owned.hpp>
-#include <process/process.hpp>
-#include <process/shared.hpp>
 
 #include "slave/containerizer/fetcher.hpp"
 #include "slave/containerizer/provisioners/docker.hpp"
@@ -83,90 +78,6 @@ protected:
   Store() {}
 };
 
-// Forward Declaration.
-class LocalStoreProcess;
-
-class LocalStore : public Store
-{
-public:
-  virtual ~LocalStore();
-
-  static Try<process::Owned<Store>> create(
-      const Flags& flags,
-      Fetcher* fetcher);
-
-  /**
-   * Put assumes the image tar archive is located in the directory specified in
-   * the slave flag docker_discovery_local_dir and is named with <name>.tar .
-   */
-  virtual process::Future<DockerImage> put(
-      const std::string& name,
-      const std::string& sandbox);
-
-  virtual process::Future<Option<DockerImage>> get(const std::string& name);
-
-private:
-  explicit LocalStore(process::Owned<LocalStoreProcess> process);
-
-  LocalStore(const LocalStore&); // Not copyable.
-
-  LocalStore& operator=(const LocalStore&); // Not assignable.
-
-  process::Owned<LocalStoreProcess> process;
-};
-
-
-class LocalStoreProcess : public process::Process<LocalStoreProcess>
-{
-public:
-  ~LocalStoreProcess() {}
-
-  static Try<process::Owned<LocalStoreProcess>> create(
-      const Flags& flags,
-      Fetcher* fetcher);
-
-  process::Future<DockerImage> put(
-      const std::string& name,
-      const std::string& sandbox);
-
-  process::Future<Option<DockerImage>> get(const std::string& name);
-
-private:
-  LocalStoreProcess(const Flags& flags);
-
-  process::Future<Nothing> untarImage(
-      const std::string& tarPath,
-      const std::string& staging);
-
-  process::Future<DockerImage> putImage(
-      const std::string& name,
-      const std::string& staging,
-      const std::string& sandbox);
-
-  Result<std::string> getParentId(
-      const std::string& staging,
-      const std::string& layerId);
-
-  process::Future<Nothing> putLayers(
-      const std::string& staging,
-      const std::list<std::string>& layers,
-      const std::string& sandbox);
-
-  process::Future<Nothing> untarLayer(
-      const std::string& staging,
-      const std::string& id,
-      const std::string& sandbox);
-
-  process::Future<Nothing> moveLayer(
-      const std::string& staging,
-      const std::string& id,
-      const std::string& sandbox);
-
-  const Flags flags;
-
-  process::Owned<ReferenceStore> refStore;
-};
-
 } // namespace docker {
 } // namespace slave {
 } // namespace internal {


[05/17] mesos git commit: Added Docker image reference store.

Posted by tn...@apache.org.
Added Docker image reference store.

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


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

Branch: refs/heads/master
Commit: e8906a1339acdf55eb4c03061fbac872f991f447
Parents: 86d87aa
Author: Lily Chen <li...@mesosphere.io>
Authored: Tue Jul 28 16:13:00 2015 -0700
Committer: Timothy Chen <tn...@apache.org>
Committed: Fri Sep 25 09:02:04 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |  10 +
 src/messages/docker_provisioner.hpp             |  24 ++
 src/messages/docker_provisioner.proto           |  35 +++
 .../provisioners/docker/reference_store.cpp     | 218 +++++++++++++++++++
 .../provisioners/docker/reference_store.hpp     | 137 ++++++++++++
 .../containerizer/provisioners/docker/store.cpp |  31 +--
 .../containerizer/provisioners/docker/store.hpp |   7 +-
 7 files changed, 430 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e8906a13/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 916be39..0b49a3b 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -249,6 +249,11 @@ STATE_PROTOS = messages/state.pb.cc messages/state.pb.h
 BUILT_SOURCES += $(STATE_PROTOS)
 CLEANFILES += $(STATE_PROTOS)
 
+DOCKER_PROVISIONER_PROTOS = messages/docker_provisioner.pb.cc messages/docker_provisioner.pb.h
+
+BUILT_SOURCES += $(DOCKER_PROVISIONER_PROTOS)
+CLEANFILES += $(DOCKER_PROVISIONER_PROTOS)
+
 REGISTRY_PROTOS = master/registry.pb.cc master/registry.pb.h
 
 BUILT_SOURCES += $(REGISTRY_PROTOS)
@@ -435,6 +440,7 @@ nodist_libmesos_no_3rdparty_la_SOURCES =				\
   $(CXX_PROTOS)								\
   $(FLAGS_PROTOS)							\
   $(MESSAGES_PROTOS)							\
+  $(DOCKER_PROVISIONER_PROTOS)		                                \
   $(REGISTRY_PROTOS)
 
 # TODO(tillt): Remove authentication/cram_md5/* which will enable us to
@@ -479,6 +485,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	master/validation.cpp						\
 	master/allocator/allocator.cpp					\
 	master/allocator/sorter/drf/sorter.cpp				\
+	messages/docker_provisioner.proto			        \
 	messages/flags.proto						\
 	messages/messages.cpp						\
 	messages/messages.proto						\
@@ -693,6 +700,7 @@ if OS_LINUX
   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/reference_store.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/provisioner/docker/store.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
@@ -788,6 +796,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	master/allocator/mesos/hierarchical.hpp				\
 	master/allocator/sorter/drf/sorter.hpp				\
 	master/allocator/sorter/sorter.hpp				\
+	messages/docker_provisioner.hpp				        \
 	messages/flags.hpp						\
 	messages/messages.hpp						\
 	module/manager.hpp						\
@@ -818,6 +827,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/provisioner/backends/bind.hpp		\
 	slave/containerizer/provisioner/backends/copy.hpp		\
 	slave/containerizer/provisioner/docker.hpp			\
+	slave/containerizer/provisioner/docker/reference_store.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/e8906a13/src/messages/docker_provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/messages/docker_provisioner.hpp b/src/messages/docker_provisioner.hpp
new file mode 100644
index 0000000..b076fdb
--- /dev/null
+++ b/src/messages/docker_provisioner.hpp
@@ -0,0 +1,24 @@
+/**
+ * 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 __DOCKER_PROVISIONER_HPP__
+#define __DOCKER_PROVISIONER_HPP__
+
+#include "messages/docker_provisioner.pb.h"
+
+#endif // __DOCKER_PROVISIONER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e8906a13/src/messages/docker_provisioner.proto
----------------------------------------------------------------------
diff --git a/src/messages/docker_provisioner.proto b/src/messages/docker_provisioner.proto
new file mode 100644
index 0000000..9de6707
--- /dev/null
+++ b/src/messages/docker_provisioner.proto
@@ -0,0 +1,35 @@
+/**
+ * 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.
+ */
+
+import "mesos/mesos.proto";
+
+package mesos.internal;
+
+/**
+ * A Docker Image name and the layer ids of the layers that comprise the image.
+ * The layerIds are ordered, with the root layer id (no parent layer id) first
+ * and the leaf layer id last.
+ */
+message DockerProvisionerImages {
+  message Image {
+    required string name = 1;
+    repeated string layer_ids = 2;
+  }
+
+  repeated Image images = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/e8906a13/src/slave/containerizer/provisioners/docker/reference_store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.cpp b/src/slave/containerizer/provisioners/docker/reference_store.cpp
new file mode 100644
index 0000000..b435ed4
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/reference_store.cpp
@@ -0,0 +1,218 @@
+/**
+ * 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 <vector>
+
+#include <glog/logging.h>
+
+#include <stout/foreach.hpp>
+#include <stout/hashset.hpp>
+#include <stout/os.hpp>
+#include <stout/protobuf.hpp>
+
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/owned.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "messages/docker_provisioner.hpp"
+
+#include "slave/containerizer/provisioners/docker/reference_store.hpp"
+#include "slave/state.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+Try<Owned<ReferenceStore>> ReferenceStore::create(const Flags& flags)
+{
+  Try<Owned<ReferenceStoreProcess>> process =
+    ReferenceStoreProcess::create(flags);
+  if (process.isError()) {
+    return Error("Failed to create reference store: " + process.error());
+  }
+  return Owned<ReferenceStore>(new ReferenceStore(process.get()));
+}
+
+
+ReferenceStore::ReferenceStore(Owned<ReferenceStoreProcess> process)
+  : process(process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+ReferenceStore::~ReferenceStore()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+void ReferenceStore::initialize()
+{
+  process::dispatch(process.get(), &ReferenceStoreProcess::initialize);
+}
+
+
+Future<DockerImage> ReferenceStore::put(
+    const string& name,
+    const list<string>& layers)
+{
+  return dispatch(
+      process.get(), &ReferenceStoreProcess::put, name, layers);
+}
+
+
+Future<Option<DockerImage>> ReferenceStore::get(const string& name)
+{
+  return dispatch(process.get(), &ReferenceStoreProcess::get, name);
+}
+
+
+ReferenceStoreProcess::ReferenceStoreProcess(const Flags& flags)
+  : flags(flags) {}
+
+
+Try<Owned<ReferenceStoreProcess>> ReferenceStoreProcess::create(
+    const Flags& flags)
+{
+  Owned<ReferenceStoreProcess> referenceStore =
+    Owned<ReferenceStoreProcess>(new ReferenceStoreProcess(flags));
+
+  return referenceStore;
+}
+
+
+Future<DockerImage> ReferenceStoreProcess::put(
+    const string& name,
+    const list<string>& layers)
+{
+  storedImages[name] = DockerImage(name, layers);
+
+  Try<Nothing> status = persist();
+  if (status.isError()) {
+    return Failure("Failed to save state of Docker images" + status.error());
+  }
+
+  return storedImages[name];
+}
+
+
+Future<Option<DockerImage>> ReferenceStoreProcess::get(const string& name)
+{
+  if (!storedImages.contains(name)) {
+    return None();
+  }
+
+  return storedImages[name];
+}
+
+
+Try<Nothing> ReferenceStoreProcess::persist()
+{
+  DockerProvisionerImages images;
+
+  foreachpair(
+      const string& name, const DockerImage& dockerImage, storedImages) {
+    DockerProvisionerImages::Image* image = images.add_images();
+
+    image->set_name(name);
+
+    foreach (const string& layer, dockerImage.layers) {
+      image->add_layer_ids(layer);
+    }
+  }
+
+  Try<string> path = path::join(flags.docker_store_dir, "storedImages");
+  if (path.isError()) {
+    return Error("Failure to construct path to repositories lookup: " +
+                    path.error());
+  }
+
+  Try<Nothing> status =
+    mesos::internal::slave::state::checkpoint(path.get(), images);
+  if (status.isError()) {
+    return Error("Failed to perform checkpoint: " + status.error());
+  }
+
+  return Nothing();
+}
+
+
+void ReferenceStoreProcess::initialize()
+{
+  Try<string> path = path::join(flags.docker_store_dir, "storedImages");
+
+  storedImages.clear();
+  if (!os::exists(path.get())) {
+    LOG(INFO) << "No images to load from disk. Docker provisioner image "
+              << "storage path: " << path.get() << " does not exist.";
+    return;
+  }
+
+  Result<DockerProvisionerImages> images =
+    ::protobuf::read<DockerProvisionerImages>(path.get());
+  if (images.isError()) {
+    LOG(ERROR) << "Failed to read protobuf for Docker provisioner image: "
+               << images.error();
+    return;
+  }
+
+  for (int i = 0; i < images.get().images_size(); i++) {
+    string imageName = images.get().images(i).name();
+
+    list<string> layers;
+    vector<string> missingLayers;
+    for (int j = 0; j < images.get().images(i).layer_ids_size(); j++) {
+      string layerId = images.get().images(i).layer_ids(j);
+
+      layers.push_back(layerId);
+
+      if (!os::exists(path::join(flags.docker_store_dir, layerId))) {
+        missingLayers.push_back(layerId);
+      }
+    }
+
+    if (!missingLayers.empty()) {
+      foreach (const string& layer, missingLayers) {
+        LOG(WARNING) << "Image layer: " << layer << " required for Docker "
+                     << "image: " << imageName << " is not on disk.";
+      }
+      LOG(WARNING) << "Skipped loading image: " << imageName
+                   << " due to missing layers.";
+      continue;
+    }
+
+    VLOG(1) << "Loaded Docker image: " << imageName << " from disk.";
+    storedImages[imageName] = DockerImage(imageName, layers);
+  }
+}
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e8906a13/src/slave/containerizer/provisioners/docker/reference_store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/reference_store.hpp b/src/slave/containerizer/provisioners/docker/reference_store.hpp
new file mode 100644
index 0000000..d9f7070
--- /dev/null
+++ b/src/slave/containerizer/provisioners/docker/reference_store.hpp
@@ -0,0 +1,137 @@
+/**
+ * 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_REFERENCE_STORE__
+#define __MESOS_DOCKER_REFERENCE_STORE__
+
+#include <list>
+#include <string>
+
+#include <stout/hashmap.hpp>
+#include <stout/json.hpp>
+#include <stout/option.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/try.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include "slave/containerizer/provisioners/docker.hpp"
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+namespace docker {
+
+// Forward Declaration.
+class ReferenceStoreProcess;
+
+/**
+ * The Reference Store is a way to track the Docker images used by the
+ * provisioner that are stored in on disk. It keeps track of the layers
+ * that Docker images are composed of and recovers DockerImage objects upon
+ * initialization by checking for dependent layers stored on disk.
+ * Currently, image layers are stored indefinitely, with no garbage collection
+ * of unreferenced image layers.
+ */
+class ReferenceStore
+{
+public:
+  ~ReferenceStore();
+
+  /**
+   * Recover all Docker Images that are on disk by checking if all
+   * layer dependencies for that layer are present on disk.
+   */
+  void initialize();
+
+  static Try<process::Owned<ReferenceStore>> create(const Flags& flags);
+
+  /**
+   * Create a DockerImage, put it in reference store and persist the reference
+   * store state to disk.
+   *
+   * @param name   the name of the Docker image to place in the reference store.
+   * @param layers the list of layer ids that comprise the Docker image in
+   *               order where the root layer's id (no parent layer) is first
+   *               and the leaf layer's id is last.
+   */
+  process::Future<DockerImage> put(
+      const std::string& name,
+      const std::list<std::string>& layers);
+
+  /**
+   * Retrieve DockerImage based on image name if it is among the DockerImages
+   * stored in memory.
+   *
+   * @param name  the name of the Docker image to retrieve
+   */
+  process::Future<Option<DockerImage>> get(const std::string& name);
+
+private:
+  explicit ReferenceStore(process::Owned<ReferenceStoreProcess> process);
+
+  ReferenceStore(const ReferenceStore&); // Not copyable.
+  ReferenceStore& operator=(const ReferenceStore&); // Not assignable.
+
+  process::Owned<ReferenceStoreProcess> process;
+};
+
+
+class ReferenceStoreProcess : public process::Process<ReferenceStoreProcess>
+{
+public:
+  ~ReferenceStoreProcess() {}
+
+  // Explicitly use 'initialize' since we are overloading below.
+  using process::ProcessBase::initialize;
+
+  void initialize();
+
+  static Try<process::Owned<ReferenceStoreProcess>> create(const Flags& flags);
+
+  process::Future<DockerImage> put(
+      const std::string& name,
+      const std::list<std::string>& layers);
+
+  process::Future<Option<DockerImage>> get(const std::string& name);
+
+  // TODO(chenlily): Implement removal of unreferenced images.
+
+private:
+  ReferenceStoreProcess(const Flags& flags);
+
+  // Write out reference store state to persistent store.
+  Try<Nothing> persist();
+
+  const Flags flags;
+
+  // This is a lookup table for images that are stored in memory. It is keyed
+  // by the name of the DockerImage.
+  // For example, "ubuntu:14.04" -> ubuntu14:04 DockerImage.
+  hashmap<std::string, DockerImage> storedImages;
+};
+
+} // namespace docker {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_DOCKER_REFERENCE_STORE__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e8906a13/src/slave/containerizer/provisioners/docker/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.cpp b/src/slave/containerizer/provisioners/docker/store.cpp
index 9453d6f..b902f8d 100644
--- a/src/slave/containerizer/provisioners/docker/store.cpp
+++ b/src/slave/containerizer/provisioners/docker/store.cpp
@@ -107,20 +107,12 @@ Try<Owned<LocalStoreProcess>> LocalStoreProcess::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  Owned<LocalStoreProcess> store =
-    Owned<LocalStoreProcess>(new LocalStoreProcess(flags));
-
-  Try<Nothing> restore = store->restore(flags);
-  if (restore.isError()) {
-    return Error("Failed to restore store: " + restore.error());
-  }
-
-  return store;
+  return Owned<LocalStoreProcess>(new LocalStoreProcess(flags));
 }
 
 
 LocalStoreProcess::LocalStoreProcess(const Flags& flags)
-  : flags(flags) {}
+  : flags(flags), refStore(ReferenceStore::create(flags).get()) {}
 
 
 Future<DockerImage> LocalStoreProcess::put(
@@ -263,10 +255,7 @@ Future<DockerImage> LocalStoreProcess::putImage(
 
   return putLayers(staging, layers, sandbox)
     .then([=]() -> Future<DockerImage> {
-      images[name] = DockerImage(name, layers);
-
-      // TODO(chenlily): update reference store or replace with reference store
-      return images[name];
+      return refStore->put(name, layers);
     });
 }
 
@@ -415,19 +404,7 @@ Future<Nothing> LocalStoreProcess::moveLayer(
 
 Future<Option<DockerImage>> LocalStoreProcess::get(const string& name)
 {
-  if (!images.contains(name)) {
-    return None();
-  }
-
-  return images[name];
-}
-
-
-// Recover stored image layers and update layers map.
-// TODO(chenlily): Implement restore.
-Try<Nothing> LocalStoreProcess::restore(const Flags& flags)
-{
-  return Nothing();
+  return refStore->get(name);
 }
 
 } // namespace docker {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e8906a13/src/slave/containerizer/provisioners/docker/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/docker/store.hpp b/src/slave/containerizer/provisioners/docker/store.hpp
index 043f2d7..256e146 100644
--- a/src/slave/containerizer/provisioners/docker/store.hpp
+++ b/src/slave/containerizer/provisioners/docker/store.hpp
@@ -35,6 +35,7 @@
 
 #include "slave/containerizer/fetcher.hpp"
 #include "slave/containerizer/provisioners/docker.hpp"
+#include "slave/containerizer/provisioners/docker/reference_store.hpp"
 #include "slave/flags.hpp"
 
 namespace mesos {
@@ -133,8 +134,6 @@ public:
 private:
   LocalStoreProcess(const Flags& flags);
 
-  Try<Nothing> restore(const Flags& flags);
-
   process::Future<Nothing> untarImage(
       const std::string& tarPath,
       const std::string& staging);
@@ -165,9 +164,7 @@ private:
 
   const Flags flags;
 
-  // This hashmap maps a Docker image by name to its corresponding DockerImage
-  // object.
-  hashmap<std::string, DockerImage> images;
+  process::Owned<ReferenceStore> refStore;
 };
 
 } // namespace docker {