You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2015/09/17 00:54:40 UTC

[6/6] mesos git commit: Unified the implementations of image provisioners.

Unified the implementations of image provisioners.

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


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

Branch: refs/heads/master
Commit: 76861c52772f7c501697ac73637d8a56aa8c9651
Parents: c027c82
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Sep 15 18:04:15 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Sep 16 15:45:50 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   4 +-
 .../isolators/filesystem/linux.cpp              |  38 +-
 .../isolators/filesystem/linux.hpp              |   6 +-
 src/slave/containerizer/mesos/containerizer.cpp |  10 +-
 .../provisioner/appc/provisioner.cpp            | 397 -------------------
 .../provisioner/appc/provisioner.hpp            |  78 ----
 .../containerizer/provisioner/appc/store.cpp    | 156 ++++----
 .../containerizer/provisioner/appc/store.hpp    |  51 +--
 src/slave/containerizer/provisioner/paths.cpp   |  14 +-
 src/slave/containerizer/provisioner/paths.hpp   |  25 +-
 .../containerizer/provisioner/provisioner.cpp   | 394 ++++++++++++++++--
 .../containerizer/provisioner/provisioner.hpp   |  33 +-
 src/slave/containerizer/provisioner/store.cpp   |  76 ++++
 src/slave/containerizer/provisioner/store.hpp   |  75 ++++
 src/slave/flags.cpp                             |  21 +-
 src/slave/flags.hpp                             |   5 +-
 src/slave/paths.cpp                             |  17 +-
 src/slave/paths.hpp                             |  12 +-
 .../containerizer/filesystem_isolator_tests.cpp |   8 +-
 src/tests/containerizer/provisioner.hpp         |  10 +-
 .../containerizer/provisioner_appc_tests.cpp    | 144 +++----
 src/tests/paths_tests.cpp                       |  14 +-
 22 files changed, 779 insertions(+), 809 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 60cb10d..2286366 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -505,8 +505,8 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/mesos/launch.cpp				\
         slave/containerizer/provisioner/paths.cpp			\
 	slave/containerizer/provisioner/provisioner.cpp			\
+        slave/containerizer/provisioner/store.cpp			\
 	slave/containerizer/provisioner/appc/paths.cpp			\
-	slave/containerizer/provisioner/appc/provisioner.cpp		\
 	slave/containerizer/provisioner/appc/spec.cpp			\
 	slave/containerizer/provisioner/appc/store.cpp			\
 	slave/containerizer/provisioner/backend.cpp			\
@@ -796,8 +796,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/linux_launcher.hpp				\
         slave/containerizer/provisioner/paths.hpp			\
 	slave/containerizer/provisioner/provisioner.hpp			\
+	slave/containerizer/provisioner/store.hpp			\
 	slave/containerizer/provisioner/appc/paths.hpp			\
-	slave/containerizer/provisioner/appc/provisioner.hpp		\
 	slave/containerizer/provisioner/appc/spec.hpp			\
 	slave/containerizer/provisioner/appc/store.hpp			\
 	slave/containerizer/provisioner/backend.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/isolators/filesystem/linux.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/linux.cpp b/src/slave/containerizer/isolators/filesystem/linux.cpp
index dbdbf87..297a296 100644
--- a/src/slave/containerizer/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/isolators/filesystem/linux.cpp
@@ -55,7 +55,7 @@ namespace slave {
 
 Try<Isolator*> LinuxFilesystemIsolatorProcess::create(
     const Flags& flags,
-    const hashmap<Image::Type, Owned<Provisioner>>& provisioners)
+    const Owned<Provisioner>& provisioner)
 {
   Result<string> user = os::user();
   if (!user.isSome()) {
@@ -68,7 +68,7 @@ Try<Isolator*> LinuxFilesystemIsolatorProcess::create(
   }
 
   Owned<MesosIsolatorProcess> process(
-      new LinuxFilesystemIsolatorProcess(flags, provisioners));
+      new LinuxFilesystemIsolatorProcess(flags, provisioner));
 
   return new MesosIsolator(process);
 }
@@ -76,9 +76,9 @@ Try<Isolator*> LinuxFilesystemIsolatorProcess::create(
 
 LinuxFilesystemIsolatorProcess::LinuxFilesystemIsolatorProcess(
     const Flags& _flags,
-    const hashmap<Image::Type, Owned<Provisioner>>& _provisioners)
+    const Owned<Provisioner>& _provisioner)
   : flags(_flags),
-    provisioners(_provisioners) {}
+    provisioner(_provisioner) {}
 
 
 LinuxFilesystemIsolatorProcess::~LinuxFilesystemIsolatorProcess() {}
@@ -174,12 +174,7 @@ Future<Nothing> LinuxFilesystemIsolatorProcess::_recover(
     const list<ContainerState>& states,
     const hashset<ContainerID>& orphans)
 {
-  list<Future<Nothing>> futures;
-  foreachvalue (const Owned<Provisioner>& provisioner, provisioners) {
-    futures.push_back(provisioner->recover(states, orphans));
-  }
-
-  return collect(futures)
+  return provisioner->recover(states, orphans)
     .then([]() -> Future<Nothing> { return Nothing(); });
 }
 
@@ -209,13 +204,7 @@ Future<Option<ContainerPrepareInfo>> LinuxFilesystemIsolatorProcess::prepare(
 
   const Image& image = executorInfo.container().mesos().image();
 
-  if (!provisioners.contains(image.type())) {
-    return Failure(
-        "No suitable provisioner found for container image type '" +
-        stringify(image.type()) + "'");
-  }
-
-  return provisioners[image.type()]->provision(containerId, image)
+  return provisioner->provision(containerId, image)
     .then(defer(PID<LinuxFilesystemIsolatorProcess>(this),
                 &LinuxFilesystemIsolatorProcess::_prepare,
                 containerId,
@@ -252,14 +241,8 @@ Future<Option<ContainerPrepareInfo>> LinuxFilesystemIsolatorProcess::_prepare(
 
     const Image& image = volume->image();
 
-    if (!provisioners.contains(image.type())) {
-      return Failure(
-          "No suitable provisioner found for image type '" +
-          stringify(image.type()) + "' in a volume");
-    }
-
     futures.push_back(
-        provisioners[image.type()]->provision(containerId, image)
+        provisioner->provision(containerId, image)
           .then([volume](const string& path) -> Future<Nothing> {
             volume->set_host_path(path);
             return Nothing();
@@ -772,12 +755,7 @@ Future<Nothing> LinuxFilesystemIsolatorProcess::cleanup(
   }
 
   // Destroy the provisioned root filesystems.
-  list<Future<bool>> futures;
-  foreachvalue (const Owned<Provisioner>& provisioner, provisioners) {
-    futures.push_back(provisioner->destroy(containerId));
-  }
-
-  return collect(futures)
+  return provisioner->destroy(containerId)
     .then([]() -> Future<Nothing> { return Nothing(); });
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/isolators/filesystem/linux.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/linux.hpp b/src/slave/containerizer/isolators/filesystem/linux.hpp
index ff76c89..b0016e5 100644
--- a/src/slave/containerizer/isolators/filesystem/linux.hpp
+++ b/src/slave/containerizer/isolators/filesystem/linux.hpp
@@ -45,7 +45,7 @@ class LinuxFilesystemIsolatorProcess : public MesosIsolatorProcess
 public:
   static Try<mesos::slave::Isolator*> create(
       const Flags& flags,
-      const hashmap<Image::Type, process::Owned<Provisioner>>& provisioners);
+      const process::Owned<Provisioner>& provisioner);
 
   virtual ~LinuxFilesystemIsolatorProcess();
 
@@ -81,7 +81,7 @@ public:
 private:
   LinuxFilesystemIsolatorProcess(
       const Flags& flags,
-      const hashmap<Image::Type, process::Owned<Provisioner>>& provisioners);
+      const process::Owned<Provisioner>& provisioner);
 
   process::Future<Nothing> _recover(
       const std::list<mesos::slave::ContainerState>& states,
@@ -107,6 +107,7 @@ private:
       const Option<std::string>& rootfs);
 
   const Flags flags;
+  const process::Owned<Provisioner> provisioner;
 
   struct Info
   {
@@ -126,7 +127,6 @@ private:
   };
 
   hashmap<ContainerID, process::Owned<Info>> infos;
-  hashmap<Image::Type, process::Owned<Provisioner>> provisioners;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 6ab4c08..0023f1d 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -162,11 +162,9 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 
 #ifdef __linux__
   // The provisioner will be used by the 'filesystem/linux' isolator.
-  Try<hashmap<Image::Type, Owned<Provisioner>>> provisioners =
-    Provisioner::create(flags, fetcher);
-
-  if (provisioners.isError()) {
-    return Error("Failed to create provisioner(s): " + provisioners.error());
+  Try<Owned<Provisioner>> provisioner = Provisioner::create(flags, fetcher);
+  if (provisioner.isError()) {
+    return Error("Failed to create provisioner: " + provisioner.error());
   }
 #endif
 
@@ -178,7 +176,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 #ifdef __linux__
     {"filesystem/linux", lambda::bind(&LinuxFilesystemIsolatorProcess::create,
                                       lambda::_1,
-                                      provisioners.get())},
+                                      provisioner.get())},
 
     // TODO(jieyu): Deprecate this in favor of using filesystem/linux.
     {"filesystem/shared", &SharedFilesystemIsolatorProcess::create},

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/appc/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/appc/provisioner.cpp b/src/slave/containerizer/provisioner/appc/provisioner.cpp
deleted file mode 100644
index 2e77747..0000000
--- a/src/slave/containerizer/provisioner/appc/provisioner.cpp
+++ /dev/null
@@ -1,397 +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 <mesos/type_utils.hpp>
-
-#include <process/collect.hpp>
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-#include <process/process.hpp>
-
-#include <stout/foreach.hpp>
-#include <stout/hashset.hpp>
-#include <stout/os.hpp>
-#include <stout/stringify.hpp>
-#include <stout/strings.hpp>
-#include <stout/uuid.hpp>
-
-#include "slave/containerizer/provisioner/backend.hpp"
-#include "slave/containerizer/provisioner/paths.hpp"
-
-#include "slave/containerizer/provisioner/appc/paths.hpp"
-#include "slave/containerizer/provisioner/appc/provisioner.hpp"
-#include "slave/containerizer/provisioner/appc/spec.hpp"
-#include "slave/containerizer/provisioner/appc/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 appc {
-
-class AppcProvisionerProcess : public Process<AppcProvisionerProcess>
-{
-public:
-  AppcProvisionerProcess(
-      const Flags& flags,
-      const string& root,
-      const Owned<Store>& store,
-      const hashmap<string, Owned<Backend>>& backends);
-
-  Future<Nothing> recover(
-      const list<ContainerState>& states,
-      const hashset<ContainerID>& orphans);
-
-  Future<string> provision(const ContainerID& containerId, const Image& image);
-
-  Future<bool> destroy(const ContainerID& containerId);
-
-private:
-  Future<string> _provision(const vector<string>& layers, const string& rootfs);
-
-  const Flags flags;
-
-  // Absolute path to the Appc provisioner root directory. It can be derived
-  // from '--work_dir' but we keep a separate copy here because we converted
-  // it into an absolute path so managed rootfs paths match the ones in
-  // 'mountinfo' (important if mount-based backends are used).
-  const string root;
-
-  const Owned<Store> store;
-  const hashmap<string, Owned<Backend>> backends;
-
-  struct Info
-  {
-    // Mappings: backend -> rootfsId -> rootfsPath.
-    hashmap<string, hashmap<string, string>> rootfses;
-  };
-
-  hashmap<ContainerID, Owned<Info>> infos;
-};
-
-
-// NOTE: Successful creation of the provisioner means its managed
-// directory under --work_dir is also created.
-Try<Owned<Provisioner>> AppcProvisioner::create(
-    const Flags& flags,
-    Fetcher* fetcher)
-{
-  string _root =
-    slave::paths::getProvisionerDir(flags.work_dir, Image::APPC);
-
-  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());
-  }
-
-  CHECK_SOME(root); // Can't be None since we just created it.
-
-  Try<Owned<Store>> store = Store::create(flags);
-  if (store.isError()) {
-    return Error("Failed to create image store: " + store.error());
-  }
-
-  hashmap<string, Owned<Backend>> backends = Backend::create(flags);
-  if (backends.empty()) {
-    return Error("No usable provisioner backend created");
-  }
-
-  if (!backends.contains(flags.appc_provisioner_backend)) {
-    return Error("The specified provisioner backend '" +
-                 flags.appc_provisioner_backend + "'is unsupported");
-  }
-
-  return Owned<Provisioner>(new AppcProvisioner(
-      Owned<AppcProvisionerProcess>(new AppcProvisionerProcess(
-          flags,
-          root.get(),
-          store.get(),
-          backends))));
-}
-
-
-AppcProvisioner::AppcProvisioner(Owned<AppcProvisionerProcess> _process)
-  : process(_process)
-{
-  spawn(CHECK_NOTNULL(process.get()));
-}
-
-
-AppcProvisioner::~AppcProvisioner()
-{
-  terminate(process.get());
-  wait(process.get());
-}
-
-
-Future<Nothing> AppcProvisioner::recover(
-    const list<ContainerState>& states,
-    const hashset<ContainerID>& orphans)
-{
-  return dispatch(
-      process.get(),
-      &AppcProvisionerProcess::recover,
-      states,
-      orphans);
-}
-
-
-Future<string> AppcProvisioner::provision(
-    const ContainerID& containerId,
-    const Image& image)
-{
-  return dispatch(
-      process.get(),
-      &AppcProvisionerProcess::provision,
-      containerId,
-      image);
-}
-
-
-Future<bool> AppcProvisioner::destroy(const ContainerID& containerId)
-{
-  return dispatch(
-      process.get(),
-      &AppcProvisionerProcess::destroy,
-      containerId);
-}
-
-
-AppcProvisionerProcess::AppcProvisionerProcess(
-    const Flags& _flags,
-    const string& _root,
-    const Owned<Store>& _store,
-    const hashmap<string, Owned<Backend>>& _backends)
-  : flags(_flags),
-    root(_root),
-    store(_store),
-    backends(_backends) {}
-
-
-Future<Nothing> AppcProvisionerProcess::recover(
-    const list<ContainerState>& states,
-    const hashset<ContainerID>& orphans)
-{
-  // Register living containers, including the ones that do not
-  // provision Appc images.
-  hashset<ContainerID> alive;
-  foreach (const ContainerState& state, states) {
-    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 =
-    provisioner::paths::listContainers(root);
-
-  if (containers.isError()) {
-    return Failure("Failed to list the containers managed by Appc "
-                   "provisioner: " + containers.error());
-  }
-
-  // Scan the list of containers, register all of them with 'infos' but
-  // mark unknown orphans for immediate cleanup.
-  hashset<ContainerID> unknownOrphans;
-  foreachkey (const ContainerID& containerId, containers.get()) {
-    Owned<Info> info = Owned<Info>(new Info());
-
-    Try<hashmap<string, hashmap<string, string>>> rootfses =
-      provisioner::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]);
-    }
-
-    infos.put(containerId, info);
-
-    if (alive.contains(containerId) || orphans.contains(containerId)) {
-      VLOG(1) << "Recovered container " << containerId;
-      continue;
-    } else {
-      // For immediate cleanup below.
-      unknownOrphans.insert(containerId);
-    }
-  }
-
-  LOG(INFO)
-    << "Recovered living and known orphan containers for Appc provisioner";
-
-  // Destroy unknown orphan containers' rootfses.
-  list<Future<bool>> destroys;
-  foreach (const ContainerID& containerId, unknownOrphans) {
-    destroys.push_back(destroy(containerId));
-  }
-
-  Future<Nothing> cleanup = collect(destroys)
-    .then([]() -> Future<Nothing> {
-      LOG(INFO) << "Cleaned up unknown orphan containers for Appc provisioner";
-      return Nothing();
-    });
-
-  Future<Nothing> recover = store->recover()
-    .then([]() -> Future<Nothing> {
-      LOG(INFO) << "Recovered Appc image store";
-      return Nothing();
-    });
-
-
-  // A successful provisioner recovery depends on:
-  // 1) Recovery of living containers and known orphans (done above).
-  // 2) Successful cleanup of unknown orphans.
-  // 3) Successful store recovery.
-  return collect(cleanup, recover)
-    .then([=]() -> Future<Nothing> {
-      return Nothing();
-    });
-}
-
-
-Future<string> AppcProvisionerProcess::provision(
-    const ContainerID& containerId,
-    const Image& image)
-{
-  if (image.type() != Image::APPC) {
-    return Failure("Unsupported container image type: " +
-                   stringify(image.type()));
-  }
-
-  if (!image.has_appc()) {
-    return Failure("Missing Appc image info");
-  }
-
-  string rootfsId = UUID::random().toString();
-  string rootfs = provisioner::paths::getContainerRootfsDir(
-      root, containerId, flags.appc_provisioner_backend, rootfsId);
-
-  if (!infos.contains(containerId)) {
-    infos.put(containerId, Owned<Info>(new Info()));
-  }
-
-  infos[containerId]->rootfses[flags.appc_provisioner_backend].put(
-      rootfsId, rootfs);
-
-  // Get and then provision image layers from the store.
-  return store->get(image.appc())
-    .then(defer(self(), &Self::_provision, lambda::_1, rootfs));
-}
-
-
-Future<string> AppcProvisionerProcess::_provision(
-     const vector<string>& layers,
-     const string& rootfs)
-{
-  LOG(INFO) << "Provisioning image layers to rootfs '" << rootfs << "'";
-
-  CHECK(backends.contains(flags.appc_provisioner_backend));
-  return backends.get(flags.appc_provisioner_backend).get()->provision(
-      layers,
-      rootfs)
-    .then([rootfs]() -> Future<string> { return rootfs; });
-}
-
-
-Future<bool> AppcProvisionerProcess::destroy(const ContainerID& containerId)
-{
-  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));
-    }
-  }
-
-  // NOTE: We calculate 'containerDir' here so that the following
-  // lambda does not need to bind 'this'.
-  string containerDir =
-    provisioner::paths::getContainerDir(root, containerId);
-
-  // TODO(xujyan): Revisit the usefulness of this return value.
-  return collect(futures)
-    .then([containerDir]() -> Future<bool> {
-      // This should be fairly cheap as the directory should only
-      // contain a few empty sub-directories at this point.
-      //
-      // TODO(jieyu): Currently, it's possible that some directories
-      // cannot be removed due to EBUSY. EBUSY is caused by the race
-      // between cleaning up this container and new containers copying
-      // the host mount table. It's OK to ignore them. The cleanup
-      // will be retried during slave recovery.
-      Try<Nothing> rmdir = os::rmdir(containerDir);
-      if (rmdir.isError()) {
-        LOG(ERROR) << "Failed to remove the provisioned container directory "
-                   << "at '" << containerDir << "'";
-      }
-
-      return true;
-    });
-}
-
-} // namespace appc {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/appc/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/appc/provisioner.hpp b/src/slave/containerizer/provisioner/appc/provisioner.hpp
deleted file mode 100644
index e4d5b8e..0000000
--- a/src/slave/containerizer/provisioner/appc/provisioner.hpp
+++ /dev/null
@@ -1,78 +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 __APPC_PROVISIONER_HPP__
-#define __APPC_PROVISIONER_HPP__
-
-#include <list>
-#include <string>
-#include <vector>
-
-#include <process/future.hpp>
-#include <process/owned.hpp>
-
-#include <stout/hashmap.hpp>
-#include <stout/json.hpp>
-#include <stout/nothing.hpp>
-#include <stout/try.hpp>
-
-#include "slave/containerizer/provisioner/provisioner.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace appc {
-
-// Forward declaration.
-class AppcProvisionerProcess;
-
-
-class AppcProvisioner : public Provisioner
-{
-public:
-  static Try<process::Owned<Provisioner>> create(
-      const Flags& flags,
-      Fetcher* fetcher);
-
-  ~AppcProvisioner();
-
-  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 AppcProvisioner(process::Owned<AppcProvisionerProcess> process);
-
-  AppcProvisioner(const AppcProvisioner&); // Not copyable.
-  AppcProvisioner& operator=(const AppcProvisioner&); // Not assignable.
-
-  process::Owned<AppcProvisionerProcess> process;
-};
-
-} // namespace appc {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __APPC_PROVISIONER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/appc/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/appc/store.cpp b/src/slave/containerizer/provisioner/appc/store.cpp
index 327ac9b..a5ef4ea 100644
--- a/src/slave/containerizer/provisioner/appc/store.cpp
+++ b/src/slave/containerizer/provisioner/appc/store.cpp
@@ -46,6 +46,8 @@ namespace appc {
 // Defines a locally cached image (which has passed validation).
 struct CachedImage
 {
+  static Try<CachedImage> create(const string& imagePath);
+
   CachedImage(
       const AppcImageManifest& _manifest,
       const string& _id,
@@ -69,6 +71,34 @@ struct CachedImage
 };
 
 
+Try<CachedImage> CachedImage::create(const string& imagePath)
+{
+  Option<Error> error = spec::validateLayout(imagePath);
+  if (error.isSome()) {
+    return Error("Invalid image layout: " + error.get().message);
+  }
+
+  string imageId = Path(imagePath).basename();
+
+  error = spec::validateImageID(imageId);
+  if (error.isSome()) {
+    return Error("Invalid image ID: " + error.get().message);
+  }
+
+  Try<string> read = os::read(paths::getImageManifestPath(imagePath));
+  if (read.isError()) {
+    return Error("Failed to read manifest: " + read.error());
+  }
+
+  Try<AppcImageManifest> manifest = spec::parse(read.get());
+  if (manifest.isError()) {
+    return Error("Failed to parse manifest: " + manifest.error());
+  }
+
+  return CachedImage(manifest.get(), imageId, imagePath);
+}
+
+
 // Helper that implements this:
 // https://github.com/appc/spec/blob/master/spec/aci.md#dependency-matching
 static bool matches(Image::Appc requirements, const CachedImage& candidate)
@@ -114,25 +144,25 @@ static bool matches(Image::Appc requirements, const CachedImage& candidate)
 class StoreProcess : public Process<StoreProcess>
 {
 public:
-  StoreProcess(const string& root);
+  StoreProcess(const string& rootDir);
 
   ~StoreProcess() {}
 
   Future<Nothing> recover();
 
-  Future<vector<string>> get(const Image::Appc& image);
+  Future<vector<string>> get(const Image& image);
 
 private:
   // Absolute path to the root directory of the store as defined by
   // --appc_store_dir.
-  const string root;
+  const string rootDir;
 
   // Mappings: name -> id -> image.
   hashmap<string, hashmap<string, CachedImage>> images;
 };
 
 
-Try<Owned<Store>> Store::create(const Flags& flags)
+Try<Owned<slave::Store>> Store::create(const Flags& flags)
 {
   Try<Nothing> mkdir = os::mkdir(paths::getImagesDir(flags.appc_store_dir));
   if (mkdir.isError()) {
@@ -141,17 +171,19 @@ Try<Owned<Store>> Store::create(const Flags& flags)
 
   // Make sure the root path is canonical so all image paths derived
   // from it are canonical too.
-  Result<string> root = os::realpath(flags.appc_store_dir);
-  if (!root.isSome()) {
+  Result<string> rootDir = os::realpath(flags.appc_store_dir);
+  if (!rootDir.isSome()) {
     // The above mkdir call recursively creates the store directory
     // if necessary so it cannot be None here.
-    CHECK_ERROR(root);
+    CHECK_ERROR(rootDir);
+
     return Error(
-        "Failed to get the realpath of the store directory: " + root.error());
+        "Failed to get the realpath of the store root directory: " +
+        rootDir.error());
   }
 
-  return Owned<Store>(new Store(
-      Owned<StoreProcess>(new StoreProcess(root.get()))));
+  return Owned<slave::Store>(new Store(
+      Owned<StoreProcess>(new StoreProcess(rootDir.get()))));
 }
 
 
@@ -175,92 +207,34 @@ Future<Nothing> Store::recover()
 }
 
 
-Future<vector<string>> Store::get(const Image::Appc& image)
+Future<vector<string>> Store::get(const Image& image)
 {
   return dispatch(process.get(), &StoreProcess::get, image);
 }
 
 
-StoreProcess::StoreProcess(const string& _root) : root(_root) {}
-
-
-// Implemented as a helper function because it's going to be used for a
-// newly downloaded image too.
-static Try<CachedImage> createImage(const string& imagePath)
-{
-  Option<Error> error = spec::validateLayout(imagePath);
-  if (error.isSome()) {
-    return Error("Invalid image layout: " + error.get().message);
-  }
-
-  string imageId = Path(imagePath).basename();
-
-  error = spec::validateImageID(imageId);
-  if (error.isSome()) {
-    return Error("Invalid image ID: " + error.get().message);
-  }
-
-  Try<string> read = os::read(paths::getImageManifestPath(imagePath));
-  if (read.isError()) {
-    return Error("Failed to read manifest: " + read.error());
-  }
-
-  Try<AppcImageManifest> manifest = spec::parse(read.get());
-  if (manifest.isError()) {
-    return Error("Failed to parse manifest: " + manifest.error());
-  }
-
-  return CachedImage(manifest.get(), imageId, imagePath);
-}
-
-
-Future<vector<string>> StoreProcess::get(const Image::Appc& image)
-{
-  if (!images.contains(image.name())) {
-    return Failure("No image named '" + image.name() + "' can be found");
-  }
-
-  // Get local candidates.
-  vector<CachedImage> candidates;
-  foreach (const CachedImage& candidate, images[image.name()].values()) {
-    // The first match is returned.
-    // TODO(xujyan): Some tie-breaking rules are necessary.
-    if (matches(image, candidate)) {
-      LOG(INFO) << "Found match for image '" << image.name()
-                << "' in the store";
-
-      // The Appc store current doesn't support dependencies and this is
-      // enforced by manifest validation: if the image's manifest contains
-      // dependencies it would fail the validation and wouldn't be stored
-      // in the store.
-      return vector<string>({candidate.rootfs()});
-    }
-  }
-
-  return Failure("No image named '" + image.name() +
-                 "' can match the requirements");
-}
+StoreProcess::StoreProcess(const string& _rootDir) : rootDir(_rootDir) {}
 
 
 Future<Nothing> StoreProcess::recover()
 {
   // Recover everything in the store.
-  Try<list<string>> imageIds = os::ls(paths::getImagesDir(root));
+  Try<list<string>> imageIds = os::ls(paths::getImagesDir(rootDir));
   if (imageIds.isError()) {
     return Failure(
         "Failed to list images under '" +
-        paths::getImagesDir(root) + "': " +
+        paths::getImagesDir(rootDir) + "': " +
         imageIds.error());
   }
 
   foreach (const string& imageId, imageIds.get()) {
-    string path = paths::getImagePath(root, imageId);
+    string path = paths::getImagePath(rootDir, imageId);
     if (!os::stat::isdir(path)) {
       LOG(WARNING) << "Unexpected entry in storage: " << imageId;
       continue;
     }
 
-    Try<CachedImage> image = createImage(path);
+    Try<CachedImage> image = CachedImage::create(path);
     if (image.isError()) {
       LOG(WARNING) << "Unexpected entry in storage: " << image.error();
       continue;
@@ -274,6 +248,40 @@ Future<Nothing> StoreProcess::recover()
   return Nothing();
 }
 
+
+Future<vector<string>> StoreProcess::get(const Image& image)
+{
+  if (image.type() != Image::APPC) {
+    return Failure("Not an Appc image: " + stringify(image.type()));
+  }
+
+  const Image::Appc& appc = image.appc();
+
+  if (!images.contains(appc.name())) {
+    return Failure("No Appc image named '" + appc.name() + "' can be found");
+  }
+
+  // Get local candidates.
+  vector<CachedImage> candidates;
+  foreach (const CachedImage& candidate, images[appc.name()].values()) {
+    // The first match is returned.
+    // TODO(xujyan): Some tie-breaking rules are necessary.
+    if (matches(appc, candidate)) {
+      LOG(INFO) << "Found match for Appc image '" << appc.name()
+                << "' in the store";
+
+      // The Appc store current doesn't support dependencies and this
+      // is enforced by manifest validation: if the image's manifest
+      // contains dependencies it would fail the validation and
+      // wouldn't be stored in the store.
+      return vector<string>({candidate.rootfs()});
+    }
+  }
+
+  return Failure("No Appc image named '" + appc.name() +
+                 "' can match the requirements");
+}
+
 } // namespace appc {
 } // namespace slave {
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/appc/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/appc/store.hpp b/src/slave/containerizer/provisioner/appc/store.hpp
index 07218d1..e845519 100644
--- a/src/slave/containerizer/provisioner/appc/store.hpp
+++ b/src/slave/containerizer/provisioner/appc/store.hpp
@@ -19,17 +19,7 @@
 #ifndef __PROVISIONER_APPC_STORE_HPP__
 #define __PROVISIONER_APPC_STORE_HPP__
 
-#include <string>
-#include <vector>
-
-#include <mesos/mesos.hpp>
-
-#include <process/future.hpp>
-#include <process/owned.hpp>
-
-#include <stout/try.hpp>
-
-#include "slave/flags.hpp"
+#include "slave/containerizer/provisioner/store.hpp"
 
 namespace mesos {
 namespace internal {
@@ -40,44 +30,27 @@ namespace appc {
 class StoreProcess;
 
 
-// An image store abstraction that "stores" images. It serves as a read-through
-// cache (cache misses are fetched remotely and transparently) for images.
-// TODO(xujyan): The store currently keeps cached images indefinitely and we
-// should introduce cache eviction policies.
-class Store
+class Store : public slave::Store
 {
 public:
-  static Try<process::Owned<Store>> create(const Flags& flags);
+  static Try<process::Owned<slave::Store>> create(const Flags& flags);
 
   ~Store();
 
-  process::Future<Nothing> recover();
+  virtual process::Future<Nothing> recover();
 
-  // Get the specified image (and all its recursive dependencies) as a list
-  // of rootfs layers in the topological order (dependencies go before
-  // dependents in the list). The images required to build this list are
-  // either retrieved from the local cache or fetched remotely.
-  // NOTE: The returned list should not have duplicates. e.g., in the
-  // following scenario the result should be [C, B, D, A] (B before D in this
-  // example is decided by the order in which A specifies its dependencies).
-  //
-  // A --> B --> C
-  // |           ^
-  // |---> D ----|
-  //
-  // The returned future fails if the requested image or any of its
-  // dependencies cannot be found or failed to be fetched.
-  // TODO(xujyan): Fetching remotely is not implemented for now and until
-  // then the future fails directly if the image is not in the local cache.
-  // TODO(xujyan): The store currently doesn't support images that have
-  // dependencies and we should add it later.
-  process::Future<std::vector<std::string>> get(const Image::Appc& image);
+  // TODO(xujyan): Fetching remotely is not implemented for now and
+  // until then the future fails directly if the image is not in the
+  // local cache.
+  // TODO(xujyan): The store currently doesn't support images that
+  // have dependencies and we should add it later.
+  virtual process::Future<std::vector<std::string>> get(const Image& image);
 
 private:
   Store(process::Owned<StoreProcess> process);
 
-  Store(const Store&); // Not copyable.
-  Store& operator=(const Store&); // Not assignable.
+  Store(const Store&) = delete; // Not copyable.
+  Store& operator=(const Store&) = delete; // Not assignable.
 
   process::Owned<StoreProcess> process;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/paths.cpp b/src/slave/containerizer/provisioner/paths.cpp
index 601c51f..8d0f7d3 100644
--- a/src/slave/containerizer/provisioner/paths.cpp
+++ b/src/slave/containerizer/provisioner/paths.cpp
@@ -96,10 +96,10 @@ string getContainerRootfsDir(
 }
 
 
-Try<hashmap<ContainerID, string>> listContainers(
+Try<hashset<ContainerID>> listContainers(
     const string& provisionerDir)
 {
-  hashmap<ContainerID, string> results;
+  hashset<ContainerID> results;
 
   string containersDir = getContainersDir(provisionerDir);
   if (!os::exists(containersDir)) {
@@ -124,18 +124,18 @@ Try<hashmap<ContainerID, string>> listContainers(
 
     ContainerID containerId;
     containerId.set_value(entry);
-    results.put(containerId, containerPath);
+    results.insert(containerId);
   }
 
   return results;
 }
 
 
-Try<hashmap<string, hashmap<string, string>>> listContainerRootfses(
+Try<hashmap<string, hashset<string>>> listContainerRootfses(
     const string& provisionerDir,
     const ContainerID& containerId)
 {
-  hashmap<string, hashmap<string, string>> results;
+  hashmap<string, hashset<string>> results;
 
   string backendsDir = getBackendsDir(
       getContainerDir(
@@ -159,7 +159,7 @@ Try<hashmap<string, hashmap<string, string>>> listContainerRootfses(
       return Error("Unable to list the backend directory: " + rootfses.error());
     }
 
-    hashmap<string, string> backendResults;
+    hashset<string> backendResults;
 
     foreach (const string& rootfsId, rootfses.get()) {
       string rootfs = getRootfsDir(getRootfsesDir(backendDir), rootfsId);
@@ -169,7 +169,7 @@ Try<hashmap<string, hashmap<string, string>>> listContainerRootfses(
         continue;
       }
 
-      backendResults.put(rootfsId, rootfs);
+      backendResults.insert(rootfsId);
     }
 
     if (backendResults.empty()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/paths.hpp b/src/slave/containerizer/provisioner/paths.hpp
index 7ec8fee..7ebd36d 100644
--- a/src/slave/containerizer/provisioner/paths.hpp
+++ b/src/slave/containerizer/provisioner/paths.hpp
@@ -24,6 +24,7 @@
 #include <mesos/mesos.hpp>
 
 #include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
 #include <stout/try.hpp>
 
 namespace mesos {
@@ -35,17 +36,13 @@ namespace paths {
 // The provisioner rootfs directory is as follows:
 // <work_dir> ('--work_dir' flag)
 // |-- provisioner
-//     |-- <image_type> (APPC, DOCKER, etc.)
-//         |-- containers
-//             |-- <container_id>
-//                 |-- backends
-//                     |-- <backend> (copy, bind, etc.)
-//                         |-- rootfses
-//                             |-- <rootfs_id> (the rootfs)
+//     |-- containers
+//         |-- <container_id>
+//             |-- backends
+//                 |-- <backend> (copy, bind, etc.)
+//                     |-- rootfses
+//                         |-- <rootfs_id> (the rootfs)
 //
-// NOTE: Each container could have multiple image types, therefore the
-// same <container_id> directory can be under other directories (e.g.,
-// <work_dir>/provisioner/DOCKER, <work_dir>/provisioner/APPC, etc.).
 // There can be multiple backends due to the change of backend flags.
 // Under each backend a rootfs is identified by the 'rootfs_id' which
 // is a UUID.
@@ -63,15 +60,15 @@ std::string getContainerRootfsDir(
 
 
 // Recursively "ls" the container directory and return a map of
-// backend -> rootfsId -> rootfsPath.
-Try<hashmap<std::string, hashmap<std::string, std::string>>>
+// backend -> {rootfsId, ...}
+Try<hashmap<std::string, hashset<std::string>>>
 listContainerRootfses(
     const std::string& provisionerDir,
     const ContainerID& containerId);
 
 
-// Return a map of containerId -> containerPath;
-Try<hashmap<ContainerID, std::string>> listContainers(
+// Return a set of container IDs.
+Try<hashset<ContainerID>> listContainers(
     const std::string& provisionerDir);
 
 } // namespace paths {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/provisioner.cpp b/src/slave/containerizer/provisioner/provisioner.cpp
index cb751dc..213f8a6 100644
--- a/src/slave/containerizer/provisioner/provisioner.cpp
+++ b/src/slave/containerizer/provisioner/provisioner.cpp
@@ -16,62 +16,396 @@
  * limitations under the License.
  */
 
+#include <mesos/type_utils.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/process.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
+#include <stout/os.hpp>
 #include <stout/stringify.hpp>
-#include <stout/strings.hpp>
+#include <stout/uuid.hpp>
 
-#include "slave/containerizer/provisioner/provisioner.hpp"
+#include "slave/paths.hpp"
 
-#include "slave/containerizer/provisioner/appc/provisioner.hpp"
+#include "slave/containerizer/provisioner/backend.hpp"
+#include "slave/containerizer/provisioner/paths.hpp"
+#include "slave/containerizer/provisioner/provisioner.hpp"
+#include "slave/containerizer/provisioner/store.hpp"
 
 using namespace process;
 
+using std::list;
 using std::string;
+using std::vector;
+
+using mesos::slave::ContainerState;
 
 namespace mesos {
 namespace internal {
 namespace slave {
 
-Try<hashmap<Image::Type, Owned<Provisioner>>> Provisioner::create(
+class ProvisionerProcess : public Process<ProvisionerProcess>
+{
+public:
+  ProvisionerProcess(
+      const Flags& flags,
+      const string& rootDir,
+      const hashmap<Image::Type, Owned<Store>>& stores,
+      const hashmap<string, Owned<Backend>>& backends);
+
+  Future<Nothing> recover(
+      const list<ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  Future<string> provision(
+      const ContainerID& containerId,
+      const Image& image);
+
+  Future<bool> destroy(const ContainerID& containerId);
+
+private:
+  Future<string> _provision(
+      const ContainerID& containerId,
+      const vector<string>& layers);
+
+  const Flags flags;
+
+  // Absolute path to the provisioner root directory. It can be
+  // derived from '--work_dir' but we keep a separate copy here
+  // because we converted it into an absolute path so managed rootfs
+  // paths match the ones in 'mountinfo' (important if mount-based
+  // backends are used).
+  const string rootDir;
+
+  const hashmap<Image::Type, Owned<Store>> stores;
+  const hashmap<string, Owned<Backend>> backends;
+
+  struct Info
+  {
+    // Mappings: backend -> {rootfsId, ...}
+    hashmap<string, hashset<string>> rootfses;
+  };
+
+  hashmap<ContainerID, Owned<Info>> infos;
+};
+
+
+Try<Owned<Provisioner>> Provisioner::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  if (flags.provisioners.isNone()) {
-    return hashmap<Image::Type, Owned<Provisioner>>();
+  string _rootDir = slave::paths::getProvisionerDir(flags.work_dir);
+
+  Try<Nothing> mkdir = os::mkdir(_rootDir);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create provisioner root directory '" +
+        _rootDir + "': " + mkdir.error());
+  }
+
+  Result<string> rootDir = os::realpath(_rootDir);
+  if (rootDir.isError()) {
+    return Error(
+        "Failed to resolve the realpath of provisioner root directory '" +
+        _rootDir + "': " + rootDir.error());
+  }
+
+  CHECK_SOME(rootDir); // Can't be None since we just created it.
+
+  Try<hashmap<Image::Type, Owned<Store>>> stores = Store::create(flags);
+  if (stores.isError()) {
+    return Error("Failed to create image stores: " + stores.error());
+  }
+
+  hashmap<string, Owned<Backend>> backends = Backend::create(flags);
+  if (backends.empty()) {
+    return Error("No usable provisioner backend created");
+  }
+
+  if (!backends.contains(flags.image_provisioner_backend)) {
+    return Error(
+        "The specified provisioner backend '" +
+        flags.image_provisioner_backend + "' is unsupported");
+  }
+
+  return Owned<Provisioner>(new Provisioner(
+      Owned<ProvisionerProcess>(new ProvisionerProcess(
+          flags,
+          rootDir.get(),
+          stores.get(),
+          backends))));
+}
+
+
+Provisioner::Provisioner(Owned<ProvisionerProcess> _process)
+  : process(_process)
+{
+  spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+Provisioner::~Provisioner()
+{
+  if (process.get() != NULL) {
+    terminate(process.get());
+    wait(process.get());
+  }
+}
+
+
+Future<Nothing> Provisioner::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  return dispatch(
+      CHECK_NOTNULL(process.get()),
+      &ProvisionerProcess::recover,
+      states,
+      orphans);
+}
+
+
+Future<string> Provisioner::provision(
+    const ContainerID& containerId,
+    const Image& image)
+{
+  return dispatch(
+      CHECK_NOTNULL(process.get()),
+      &ProvisionerProcess::provision,
+      containerId,
+      image);
+}
+
+
+Future<bool> Provisioner::destroy(const ContainerID& containerId)
+{
+  return dispatch(
+      CHECK_NOTNULL(process.get()),
+      &ProvisionerProcess::destroy,
+      containerId);
+}
+
+
+ProvisionerProcess::ProvisionerProcess(
+    const Flags& _flags,
+    const string& _rootDir,
+    const hashmap<Image::Type, Owned<Store>>& _stores,
+    const hashmap<string, Owned<Backend>>& _backends)
+  : flags(_flags),
+    rootDir(_rootDir),
+    stores(_stores),
+    backends(_backends) {}
+
+
+Future<Nothing> ProvisionerProcess::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  // Register living containers, including the ones that do not
+  // provision images.
+  hashset<ContainerID> alive;
+  foreach (const ContainerState& state, states) {
+    alive.insert(state.container_id());
   }
 
-  hashmap<Image::Type,
-          Try<Owned<Provisioner>>(*)(const Flags&, Fetcher*)> creators;
+  // 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<hashset<ContainerID>> containers =
+    provisioner::paths::listContainers(rootDir);
 
-  // Register all supported creators.
-  creators.put(Image::APPC, &appc::AppcProvisioner::create);
+  if (containers.isError()) {
+    return Failure(
+        "Failed to list the containers managed by the provisioner: " +
+        containers.error());
+  }
+
+  // Scan the list of containers, register all of them with 'infos'
+  // but mark unknown orphans for immediate cleanup.
+  hashset<ContainerID> unknownOrphans;
+
+  foreach (const ContainerID& containerId, containers.get()) {
+    Owned<Info> info = Owned<Info>(new Info());
 
-  hashmap<Image::Type, Owned<Provisioner>> provisioners;
+    Try<hashmap<string, hashset<string>>> rootfses =
+      provisioner::paths::listContainerRootfses(rootDir, containerId);
 
-  // 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 (rootfses.isError()) {
+      return Failure(
+          "Unable to list rootfses belonged to container " +
+          stringify(containerId) + ": " + rootfses.error());
+    }
 
-     if (!creators.contains(imageType)) {
-       return Error("Unsupported provisioner '" + type + "'");
-     }
+    foreachkey (const string& backend, rootfses.get()) {
+      if (!backends.contains(backend)) {
+        return Failure(
+            "Found rootfses managed by an unrecognized backend: " + backend);
+      }
 
-     Try<Owned<Provisioner>> provisioner = creators[imageType](flags, fetcher);
-     if (provisioner.isError()) {
-       return Error("Failed to create '" + stringify(imageType) +
-                    "' provisioner: " + provisioner.error());
-     }
+      info->rootfses.put(backend, rootfses.get()[backend]);
+    }
+
+    infos.put(containerId, info);
+
+    if (alive.contains(containerId) || orphans.contains(containerId)) {
+      LOG(INFO) << "Recovered container " << containerId;
+      continue;
+    } else {
+      // For immediate cleanup below.
+      unknownOrphans.insert(containerId);
+    }
+  }
 
-     provisioners[imageType] = provisioner.get();
+  // Cleanup unknown orphan containers' rootfses.
+  list<Future<bool>> cleanups;
+  foreach (const ContainerID& containerId, unknownOrphans) {
+    LOG(INFO) << "Cleaning up unknown orphan container " << containerId;
+    cleanups.push_back(destroy(containerId));
   }
 
-  return provisioners;
+  Future<Nothing> cleanup = collect(cleanups)
+    .then([]() -> Future<Nothing> { return Nothing(); });
+
+  // Recover stores.
+  list<Future<Nothing>> recovers;
+  foreachvalue (const Owned<Store>& store, stores) {
+    recovers.push_back(store->recover());
+  }
+
+  Future<Nothing> recover = collect(recovers)
+    .then([]() -> Future<Nothing> { return Nothing(); });
+
+  // A successful provisioner recovery depends on:
+  // 1) Recovery of living containers and known orphans (done above).
+  // 2) Successful cleanup of unknown orphans.
+  // 3) Successful store recovery.
+  //
+  // TODO(jieyu): Do not recover 'store' before unknown orphans are
+  // cleaned up. In the future, we may want to cleanup unused rootfses
+  // in 'store', which might fail if there still exist unknown orphans
+  // holding references to them.
+  return collect(cleanup, recover)
+    .then([=]() -> Future<Nothing> {
+      LOG(INFO) << "Provisioner recovery complete";
+      return Nothing();
+    });
+}
+
+
+Future<string> ProvisionerProcess::provision(
+    const ContainerID& containerId,
+    const Image& image)
+{
+  if (!stores.contains(image.type())) {
+    return Failure(
+        "Unsupported container image type: " +
+        stringify(image.type()));
+  }
+
+  // Get and then provision image layers from the store.
+  return stores.get(image.type()).get()->get(image)
+    .then(defer(self(), &Self::_provision, containerId, lambda::_1));
+}
+
+
+Future<string> ProvisionerProcess::_provision(
+    const ContainerID& containerId,
+    const vector<string>& layers)
+{
+  // TODO(jieyu): Choose a backend smartly. For instance, if there is
+  // only one layer returned from the store. prefer to use bind
+  // backend because it's the simplest.
+  const string& backend = flags.image_provisioner_backend;
+  CHECK(backends.contains(backend));
+
+  string rootfsId = UUID::random().toString();
+
+  string rootfs = provisioner::paths::getContainerRootfsDir(
+      rootDir,
+      containerId,
+      backend,
+      rootfsId);
+
+  LOG(INFO) << "Provisioning image rootfs '" << rootfs
+            << "' for container " << containerId;
+
+  // NOTE: It's likely that the container ID already exists in 'infos'
+  // because one container might provision multiple images.
+  if (!infos.contains(containerId)) {
+    infos.put(containerId, Owned<Info>(new Info()));
+  }
+
+  infos[containerId]->rootfses[backend].insert(rootfsId);
+
+  return backends.get(backend).get()->provision(layers, rootfs)
+    .then([rootfs]() -> Future<string> { return rootfs; });
+}
+
+
+Future<bool> ProvisionerProcess::destroy(const ContainerID& containerId)
+{
+  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) {
+    if (!backends.contains(backend)) {
+      return Failure("Unknown backend '" + backend + "'");
+    }
+
+    foreach (const string& rootfsId, info->rootfses[backend]) {
+      string rootfs = provisioner::paths::getContainerRootfsDir(
+          rootDir,
+          containerId,
+          backend,
+          rootfsId);
+
+      LOG(INFO) << "Destroying container rootfs at '" << rootfs
+                << "' for container " << containerId;
+
+      futures.push_back(backends.get(backend).get()->destroy(rootfs));
+    }
+  }
+
+  // NOTE: We calculate 'containerDir' here so that the following
+  // lambda does not need to bind 'this'.
+  string containerDir =
+    provisioner::paths::getContainerDir(rootDir, containerId);
+
+  // TODO(xujyan): Revisit the usefulness of this return value.
+  return collect(futures)
+    .then([containerDir]() -> Future<bool> {
+      // This should be fairly cheap as the directory should only
+      // contain a few empty sub-directories at this point.
+      //
+      // TODO(jieyu): Currently, it's possible that some directories
+      // cannot be removed due to EBUSY. EBUSY is caused by the race
+      // between cleaning up this container and new containers copying
+      // the host mount table. It's OK to ignore them. The cleanup
+      // will be retried during slave recovery.
+      Try<Nothing> rmdir = os::rmdir(containerDir);
+      if (rmdir.isError()) {
+        LOG(ERROR) << "Failed to remove the provisioned container directory "
+                   << "at '" << containerDir << "': " << rmdir.error();
+      }
+
+      return true;
+    });
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/provisioner.hpp b/src/slave/containerizer/provisioner/provisioner.hpp
index c55225e..912fc5a 100644
--- a/src/slave/containerizer/provisioner/provisioner.hpp
+++ b/src/slave/containerizer/provisioner/provisioner.hpp
@@ -25,7 +25,6 @@
 
 #include <mesos/slave/isolator.hpp> // For ContainerState.
 
-#include <stout/hashmap.hpp>
 #include <stout/nothing.hpp>
 #include <stout/try.hpp>
 
@@ -40,15 +39,20 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
+// Forward declaration.
+class ProvisionerProcess;
+
+
 class Provisioner
 {
 public:
-  virtual ~Provisioner() {}
+  // Create the provisioner based on the specified flags.
+  static Try<process::Owned<Provisioner>> create(
+      const Flags& flags,
+      Fetcher* fetcher);
 
-  // Create provisioners based on specified flags. An error is returned if
-  // any of the provisioners specified in --provisioner failed to be created.
-  static Try<hashmap<Image::Type, process::Owned<Provisioner>>>
-    create(const Flags& flags, Fetcher* fetcher);
+  // NOTE: Made 'virtual' for mocking and testing.
+  virtual ~Provisioner();
 
   // Recover root filesystems for containers from the run states and
   // the orphan containers (known to the launcher but not known to the
@@ -57,19 +61,30 @@ public:
   // directories) to not leak anything.
   virtual process::Future<Nothing> recover(
       const std::list<mesos::slave::ContainerState>& states,
-      const hashset<ContainerID>& orphans) = 0;
+      const hashset<ContainerID>& orphans);
 
   // Provision a root filesystem for the container using the specified
   // 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
   // filesystem have been removed. Return false if there is no
   // provisioned root filesystem for the given container.
-  virtual process::Future<bool> destroy(const ContainerID& containerId) = 0;
+  virtual process::Future<bool> destroy(const ContainerID& containerId);
+
+protected:
+  Provisioner() {} // For creating mock object.
+
+private:
+  explicit Provisioner(process::Owned<ProvisionerProcess> process);
+
+  Provisioner(const Provisioner&) = delete; // Not copyable.
+  Provisioner& operator=(const Provisioner&) = delete; // Not assignable.
+
+  process::Owned<ProvisionerProcess> process;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/store.cpp b/src/slave/containerizer/provisioner/store.cpp
new file mode 100644
index 0000000..35d1199
--- /dev/null
+++ b/src/slave/containerizer/provisioner/store.cpp
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+
+#include <mesos/type_utils.hpp>
+
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/strings.hpp>
+
+#include "slave/containerizer/provisioner/store.hpp"
+
+#include "slave/containerizer/provisioner/appc/store.hpp"
+
+using namespace process;
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<hashmap<Image::Type, Owned<Store>>> Store::create(const Flags& flags)
+{
+  if (flags.image_providers.isNone()) {
+    return hashmap<Image::Type, Owned<Store>>();
+  }
+
+  hashmap<Image::Type, Try<Owned<Store>>(*)(const Flags&)> creators;
+  creators.put(Image::APPC, &appc::Store::create);
+
+  hashmap<Image::Type, Owned<Store>> stores;
+
+  foreach (const string& type,
+           strings::tokenize(flags.image_providers.get(), ",")) {
+    Image::Type imageType;
+    if (!Image::Type_Parse(strings::upper(type), &imageType)) {
+      return Error("Unknown image type '" + type + "'");
+    }
+
+    if (!creators.contains(imageType)) {
+      return Error("Unsupported image type '" + type + "'");
+    }
+
+    Try<Owned<Store>> store = creators[imageType](flags);
+    if (store.isError()) {
+      return Error(
+          "Failed to create store for image type '" +
+          type + "': " + store.error());
+    }
+
+    stores.put(imageType, store.get());
+  }
+
+  return stores;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/containerizer/provisioner/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/store.hpp b/src/slave/containerizer/provisioner/store.hpp
new file mode 100644
index 0000000..cf3e7d7
--- /dev/null
+++ b/src/slave/containerizer/provisioner/store.hpp
@@ -0,0 +1,75 @@
+/**
+ * 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_STORE_HPP__
+#define __PROVISIONER_STORE_HPP__
+
+#include <string>
+#include <vector>
+
+#include <mesos/mesos.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// An image store abstraction that "stores" images. It serves as a
+// read-through cache (cache misses are fetched remotely and
+// transparently) for images.
+class Store
+{
+public:
+  static Try<hashmap<Image::Type, process::Owned<Store>>> create(
+      const Flags& flags);
+
+  virtual ~Store() {}
+
+  virtual process::Future<Nothing> recover() = 0;
+
+  // Get the specified image (and all its recursive dependencies) as a
+  // list of rootfs layers in the topological order (dependencies go
+  // before dependents in the list). The images required to build this
+  // list are either retrieved from the local cache or fetched
+  // remotely.
+  //
+  // NOTE: The returned list should not have duplicates. e.g., in the
+  // following scenario the result should be [C, B, D, A] (B before D
+  // in this example is decided by the order in which A specifies its
+  // dependencies).
+  //
+  // A --> B --> C
+  // |           ^
+  // |---> D ----|
+  //
+  // The returned future fails if the requested image or any of its
+  // dependencies cannot be found or failed to be fetched.
+  virtual process::Future<std::vector<std::string>> get(const Image& image) = 0;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __PROVISIONER_STORE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index ff167ec..add4196 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -59,21 +59,22 @@ mesos::internal::slave::Flags::Flags()
       "for the Mesos Containerizer.",
       "posix/cpu,posix/mem");
 
-  add(&Flags::provisioners,
-      "provisioners",
-      "Comma separated list of image rootfs provisioners,\n"
-      "e.g., appc,docker");
+  add(&Flags::image_providers,
+      "image_providers",
+      "Comma separated list of supported image providers,\n"
+      "e.g., 'APPC,DOCKER'.");
+
+  add(&Flags::image_provisioner_backend,
+      "image_provisioner_backend",
+      "Strategy for provisioning container rootfs from images,\n"
+      "e.g., 'bind', 'copy'.",
+      "copy");
 
   add(&Flags::appc_store_dir,
       "appc_store_dir",
-      "Directory the appc provisioner will store images in",
+      "Directory the appc provisioner will store images in.",
       "/tmp/mesos/store/appc");
 
-  add(&Flags::appc_provisioner_backend,
-      "appc_provisioner_backend",
-      "Strategy for provisioning container rootfs from appc images",
-      "copy");
-
   add(&Flags::default_role,
       "default_role",
       "Any resources in the --resources flag that\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 799c963..e31a418 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -47,10 +47,9 @@ public:
   Option<std::string> resources;
   std::string isolation;
 
-  Option<std::string> provisioners;
-
+  Option<std::string> image_providers;
+  std::string image_provisioner_backend;
   std::string appc_store_dir;
-  std::string appc_provisioner_backend;
 
   std::string default_role;
   Option<std::string> attributes;

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/paths.cpp b/src/slave/paths.cpp
index 26b7f88..fb77e64 100644
--- a/src/slave/paths.cpp
+++ b/src/slave/paths.cpp
@@ -68,6 +68,12 @@ string getSandboxRootDir(const string& rootDir)
 }
 
 
+string getProvisionerDir(const string& rootDir)
+{
+  return path::join(rootDir, "provisioner");
+}
+
+
 string getArchiveDir(const string& rootDir)
 {
   return path::join(rootDir, "archive");
@@ -369,17 +375,6 @@ string getPersistentVolumePath(
 }
 
 
-string getProvisionerDir(
-    const string& rootDir,
-    const Image::Type& imageType)
-{
-  return path::join(
-      rootDir,
-      "provisioner",
-      stringify(imageType));
-}
-
-
 string createExecutorDirectory(
     const string& rootDir,
     const SlaveID& slaveId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/slave/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/paths.hpp b/src/slave/paths.hpp
index 8873e78..f743fb4 100644
--- a/src/slave/paths.hpp
+++ b/src/slave/paths.hpp
@@ -90,8 +90,7 @@ namespace paths {
 //   |   |-- roles
 //   |       |-- <role>
 //   |           |-- <persistence_id> (persistent volume)
-//   |-- provisioners
-//       |-- <image_type> (as defined in Image::Type)
+//   |-- provisioner
 
 const char LATEST_SYMLINK[] = "latest";
 
@@ -103,6 +102,9 @@ std::string getMetaRootDir(const std::string& rootDir);
 std::string getSandboxRootDir(const std::string& rootDir);
 
 
+std::string getProvisionerDir(const std::string& rootDir);
+
+
 std::string getArchiveDir(const std::string& rootDir);
 
 
@@ -264,12 +266,6 @@ std::string getPersistentVolumePath(
     const std::string& persistenceId);
 
 
-// Contents of the provisioner directory are managed by the provisioner itself.
-std::string getProvisionerDir(
-    const std::string& rootDir,
-    const Image::Type& imageType);
-
-
 std::string createExecutorDirectory(
     const std::string& rootDir,
     const SlaveID& slaveId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index ffa371f..4dbfeac 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -92,14 +92,10 @@ public:
       rootfses.put(imageName, rootfs.get().share());
     }
 
-    // Create the TestAppcProvisioner for the above root filesystems.
-    hashmap<Image::Type, Owned<Provisioner>> provisioners;
-    provisioners.put(
-        Image::APPC,
-        Owned<Provisioner>(new TestAppcProvisioner(rootfses)));
+    Owned<Provisioner> provisioner(new TestProvisioner(rootfses));
 
     Try<Isolator*> _isolator =
-      LinuxFilesystemIsolatorProcess::create(flags, provisioners);
+      LinuxFilesystemIsolatorProcess::create(flags, provisioner);
 
     if (_isolator.isError()) {
       return Error(

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/tests/containerizer/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner.hpp b/src/tests/containerizer/provisioner.hpp
index 3ae2400..54aab5f 100644
--- a/src/tests/containerizer/provisioner.hpp
+++ b/src/tests/containerizer/provisioner.hpp
@@ -34,10 +34,10 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-class TestAppcProvisioner : public slave::Provisioner
+class TestProvisioner : public slave::Provisioner
 {
 public:
-  TestAppcProvisioner(
+  TestProvisioner(
       const hashmap<std::string, process::Shared<Rootfs>>& _rootfses)
     : rootfses(_rootfses)
   {
@@ -46,17 +46,17 @@ public:
     using testing::Invoke;
 
     ON_CALL(*this, recover(_, _))
-      .WillByDefault(Invoke(this, &TestAppcProvisioner::unmocked_recover));
+      .WillByDefault(Invoke(this, &TestProvisioner::unmocked_recover));
     EXPECT_CALL(*this, recover(_, _))
       .WillRepeatedly(DoDefault());
 
     ON_CALL(*this, provision(_, _))
-      .WillByDefault(Invoke(this, &TestAppcProvisioner::unmocked_provision));
+      .WillByDefault(Invoke(this, &TestProvisioner::unmocked_provision));
     EXPECT_CALL(*this, provision(_, _))
       .WillRepeatedly(DoDefault());
 
     ON_CALL(*this, destroy(_))
-      .WillByDefault(Invoke(this, &TestAppcProvisioner::unmocked_destroy));
+      .WillByDefault(Invoke(this, &TestProvisioner::unmocked_destroy));
     EXPECT_CALL(*this, destroy(_))
       .WillRepeatedly(DoDefault());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/tests/containerizer/provisioner_appc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_appc_tests.cpp b/src/tests/containerizer/provisioner_appc_tests.cpp
index 3318557..bd9b2e8 100644
--- a/src/tests/containerizer/provisioner_appc_tests.cpp
+++ b/src/tests/containerizer/provisioner_appc_tests.cpp
@@ -29,6 +29,9 @@
 #include <stout/stringify.hpp>
 #include <stout/uuid.hpp>
 
+#include "slave/paths.hpp"
+
+#include "slave/containerizer/provisioner/paths.hpp"
 #include "slave/containerizer/provisioner/provisioner.hpp"
 
 #include "slave/containerizer/provisioner/appc/spec.hpp"
@@ -51,10 +54,10 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-class AppcProvisionerTest : public TemporaryDirectoryTest {};
+class AppcSpecTest : public TemporaryDirectoryTest {};
 
 
-TEST_F(AppcProvisionerTest, ValidateImageManifest)
+TEST_F(AppcSpecTest, ValidateImageManifest)
 {
   JSON::Value manifest = JSON::parse(
       "{"
@@ -97,7 +100,7 @@ TEST_F(AppcProvisionerTest, ValidateImageManifest)
 }
 
 
-TEST_F(AppcProvisionerTest, ValidateLayout)
+TEST_F(AppcSpecTest, ValidateLayout)
 {
   string image = os::getcwd();
 
@@ -120,12 +123,15 @@ TEST_F(AppcProvisionerTest, ValidateLayout)
 }
 
 
-TEST_F(AppcProvisionerTest, StoreRecover)
+class AppcStoreTest : public TemporaryDirectoryTest {};
+
+
+TEST_F(AppcStoreTest, Recover)
 {
   // Create store.
   slave::Flags flags;
   flags.appc_store_dir = path::join(os::getcwd(), "store");
-  Try<Owned<Store>> store = Store::create(flags);
+  Try<Owned<slave::Store>> store = Store::create(flags);
   ASSERT_SOME(store);
 
   // Create a simple image in the store:
@@ -161,8 +167,8 @@ TEST_F(AppcProvisionerTest, StoreRecover)
       "  ]"
       "}").get();
 
-  // The 'imageId' below has the correct format but it's not computed by
-  // hashing the tarball of the image. It's OK here as we assume
+  // The 'imageId' below has the correct format but it's not computed
+  // by hashing the tarball of the image. It's OK here as we assume
   // the images under 'images' have passed such check when they are
   // downloaded and validated.
   string imageId =
@@ -182,7 +188,7 @@ TEST_F(AppcProvisionerTest, StoreRecover)
 
   Image image;
   image.mutable_appc()->set_name("foo.com/bar");
-  Future<vector<string>> layers = store.get()->get(image.appc());
+  Future<vector<string>> layers = store.get()->get(image);
   AWAIT_READY(layers);
 
   EXPECT_EQ(1u, layers.get().size());
@@ -193,23 +199,25 @@ TEST_F(AppcProvisionerTest, StoreRecover)
 }
 
 
+class ProvisionerAppcTest : public TemporaryDirectoryTest {};
+
+
 #ifdef __linux__
-// This test verifies that the provisioner can provision an rootfs from an
-// image that is already put into the store directory.
-TEST_F(AppcProvisionerTest, ROOT_Provision)
+// This test verifies that the provisioner can provision an rootfs
+// from an image that is already put into the store directory.
+TEST_F(ProvisionerAppcTest, ROOT_Provision)
 {
   // Create provisioner.
   slave::Flags flags;
+  flags.image_providers = "APPC";
   flags.appc_store_dir = path::join(os::getcwd(), "store");
-  flags.appc_provisioner_backend = "bind";
-  flags.provisioners = "appc";
+  flags.image_provisioner_backend = "bind";
   flags.work_dir = "work_dir";
 
   Fetcher fetcher;
-  Try<hashmap<Image::Type, Owned<Provisioner>>> provisioners =
-    Provisioner::create(flags, &fetcher);
-  ASSERT_SOME(provisioners);
-  ASSERT_TRUE(provisioners.get().contains(Image::APPC));
+
+  Try<Owned<Provisioner>> provisioner = Provisioner::create(flags, &fetcher);
+  ASSERT_SOME(provisioner);
 
   // Create a simple image in the store:
   // <store>
@@ -244,8 +252,8 @@ TEST_F(AppcProvisionerTest, ROOT_Provision)
       "  ]"
       "}").get();
 
-  // The 'imageId' below has the correct format but it's not computed by
-  // hashing the tarball of the image. It's OK here as we assume
+  // The 'imageId' below has the correct format but it's not computed
+  // by hashing the tarball of the image. It's OK here as we assume
   // the images under 'images' have passed such check when they are
   // downloaded and validated.
   string imageId =
@@ -261,7 +269,7 @@ TEST_F(AppcProvisionerTest, ROOT_Provision)
       os::write(path::join(imagePath, "manifest"), stringify(manifest)));
 
   // Recover. This is when the image in the store is loaded.
-  AWAIT_READY(provisioners.get()[Image::APPC]->recover({}, {}));
+  AWAIT_READY(provisioner.get()->recover({}, {}));
 
   // Simulate a task that requires an image.
   Image image;
@@ -270,30 +278,30 @@ TEST_F(AppcProvisionerTest, ROOT_Provision)
   ContainerID containerId;
   containerId.set_value("12345");
 
-  Future<string> rootfs =
-    provisioners.get()[Image::APPC]->provision(containerId, image);
+  Future<string> rootfs = provisioner.get()->provision(containerId, image);
   AWAIT_READY(rootfs);
 
-  string containerDir = path::join(
-      flags.work_dir,
-      "provisioners",
-      stringify(Image::APPC),
-      "containers",
-      containerId.value());
+  string provisionerDir = slave::paths::getProvisionerDir(flags.work_dir);
 
-  Try<list<string>> rootfses = os::ls(path::join(
-      containerDir,
-      "backends",
-      flags.appc_provisioner_backend,
-      "rootfses"));
+  string containerDir =
+    slave::provisioner::paths::getContainerDir(
+        provisionerDir,
+        containerId);
+
+  Try<hashmap<string, hashset<string>>> rootfses =
+    slave::provisioner::paths::listContainerRootfses(
+        provisionerDir,
+        containerId);
 
   ASSERT_SOME(rootfses);
 
   // Verify that the rootfs is successfully provisioned.
-  EXPECT_EQ(1u, rootfses.get().size());
-  EXPECT_EQ(rootfses.get().front(), Path(rootfs.get()).basename());
+  ASSERT_TRUE(rootfses->contains(flags.image_provisioner_backend));
+  ASSERT_EQ(1u, rootfses->get(flags.image_provisioner_backend)->size());
+  EXPECT_EQ(*rootfses->get(flags.image_provisioner_backend)->begin(),
+            Path(rootfs.get()).basename());
 
-  Future<bool> destroy = provisioners.get()[Image::APPC]->destroy(containerId);
+  Future<bool> destroy = provisioner.get()->destroy(containerId);
   AWAIT_READY(destroy);
 
   // One rootfs is destroyed.
@@ -305,23 +313,21 @@ TEST_F(AppcProvisionerTest, ROOT_Provision)
 #endif // __linux__
 
 
-// This test verifies that a provisioner can recover the rootfs provisioned
-// by a previous provisioner and then destroy it. Note that we use the copy
-// backend in this test so Linux is not required.
-TEST_F(AppcProvisionerTest, Recover)
+// This test verifies that a provisioner can recover the rootfs
+// provisioned by a previous provisioner and then destroy it. Note
+// that we use the copy backend in this test so Linux is not required.
+TEST_F(ProvisionerAppcTest, Recover)
 {
   // Create provisioner.
   slave::Flags flags;
+  flags.image_providers = "APPC";
   flags.appc_store_dir = path::join(os::getcwd(), "store");
-  flags.appc_provisioner_backend = "copy";
-  flags.provisioners = "appc";
+  flags.image_provisioner_backend = "copy";
   flags.work_dir = "work_dir";
 
   Fetcher fetcher;
-  Try<hashmap<Image::Type, Owned<Provisioner>>> provisioners1 =
-    Provisioner::create(flags, &fetcher);
+  Try<Owned<Provisioner>> provisioners1 = Provisioner::create(flags, &fetcher);
   ASSERT_SOME(provisioners1);
-  ASSERT_TRUE(provisioners1.get().contains(Image::APPC));
 
   // Create a simple image in the store:
   // <store>
@@ -336,8 +342,8 @@ TEST_F(AppcProvisionerTest, Recover)
       "  \"name\": \"foo.com/bar\""
       "}").get();
 
-  // The 'imageId' below has the correct format but it's not computed by
-  // hashing the tarball of the image. It's OK here as we assume
+  // The 'imageId' below has the correct format but it's not computed
+  // by hashing the tarball of the image. It's OK here as we assume
   // the images under 'images' have passed such check when they are
   // downloaded and validated.
   string imageId =
@@ -353,7 +359,7 @@ TEST_F(AppcProvisionerTest, Recover)
       os::write(path::join(imagePath, "manifest"), stringify(manifest)));
 
   // Recover. This is when the image in the store is loaded.
-  AWAIT_READY(provisioners1.get()[Image::APPC]->recover({}, {}));
+  AWAIT_READY(provisioners1.get()->recover({}, {}));
 
   Image image;
   image.mutable_appc()->set_name("foo.com/bar");
@@ -361,48 +367,46 @@ TEST_F(AppcProvisionerTest, Recover)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  Future<string> rootfs =
-    provisioners1.get()[Image::APPC]->provision(containerId, image);
+  Future<string> rootfs = provisioners1.get()->provision(containerId, image);
   AWAIT_READY(rootfs);
 
   // Create a new provisioner to recover the state from the container.
-  Try<hashmap<Image::Type, Owned<Provisioner>>> provisioners2 =
-    Provisioner::create(flags, &fetcher);
+  Try<Owned<Provisioner>> provisioners2 = Provisioner::create(flags, &fetcher);
   ASSERT_SOME(provisioners2);
-  ASSERT_TRUE(provisioners2.get().contains(Image::APPC));
 
   mesos::slave::ContainerState state;
 
   // Here we are using an ExecutorInfo in the ContainerState without a
-  // ContainerInfo. This is the situation where the Image is specified via
-  // --default_container_info so it's not part of the recovered ExecutorInfo.
+  // ContainerInfo. This is the situation where the Image is specified
+  // via --default_container_info so it's not part of the recovered
+  // ExecutorInfo.
   state.mutable_container_id()->CopyFrom(containerId);
 
-  AWAIT_READY(provisioners2.get()[Image::APPC]->recover({state}, {}));
+  AWAIT_READY(provisioners2.get()->recover({state}, {}));
 
   // It's possible for the user to provision two different rootfses
   // from the same image.
-  AWAIT_READY(provisioners2.get()[Image::APPC]->provision(containerId, image));
+  AWAIT_READY(provisioners2.get()->provision(containerId, image));
+
+  string provisionerDir = slave::paths::getProvisionerDir(flags.work_dir);
 
-  string containerDir = path::join(
-      flags.work_dir,
-      "provisioners",
-      stringify(Image::APPC),
-      "containers",
-      containerId.value());
+  string containerDir =
+    slave::provisioner::paths::getContainerDir(
+        provisionerDir,
+        containerId);
 
-  Try<list<string>> rootfses = os::ls(path::join(
-      containerDir,
-      "backends",
-      flags.appc_provisioner_backend,
-      "rootfses"));
+  Try<hashmap<string, hashset<string>>> rootfses =
+    slave::provisioner::paths::listContainerRootfses(
+        provisionerDir,
+        containerId);
 
   ASSERT_SOME(rootfses);
 
   // Verify that the rootfs is successfully provisioned.
-  EXPECT_EQ(2u, rootfses.get().size());
+  ASSERT_TRUE(rootfses->contains(flags.image_provisioner_backend));
+  EXPECT_EQ(2u, rootfses->get(flags.image_provisioner_backend)->size());
 
-  Future<bool> destroy = provisioners2.get()[Image::APPC]->destroy(containerId);
+  Future<bool> destroy = provisioners2.get()->destroy(containerId);
   AWAIT_READY(destroy);
   EXPECT_TRUE(destroy.get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/76861c52/src/tests/paths_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/paths_tests.cpp b/src/tests/paths_tests.cpp
index efdc2c0..da01dae 100644
--- a/src/tests/paths_tests.cpp
+++ b/src/tests/paths_tests.cpp
@@ -104,6 +104,13 @@ TEST_F(PathsTest, Meta)
 }
 
 
+TEST_F(PathsTest, ProvisionerDir)
+{
+  EXPECT_EQ(path::join(rootDir, "provisioner"),
+            paths::getProvisionerDir(rootDir));
+}
+
+
 TEST_F(PathsTest, Archive)
 {
   EXPECT_EQ(path::join(rootDir, "archive"), paths::getArchiveDir(rootDir));
@@ -219,13 +226,6 @@ TEST_F(PathsTest, PersistentVolume)
 }
 
 
-TEST_F(PathsTest, ProvisionerDir)
-{
-  string dir = path::join(rootDir, "provisioner", "APPC");
-
-  EXPECT_EQ(dir, paths::getProvisionerDir(rootDir, imageType));
-}
-
 } // namespace paths {
 } // namespace slave {
 } // namespace internal {