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

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

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;