You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ya...@apache.org on 2015/09/04 02:30:12 UTC

[1/3] mesos git commit: Implemented AppcProvisioner.

Repository: mesos
Updated Branches:
  refs/heads/master fb631516c -> 313591aa6


Implemented AppcProvisioner.

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


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

Branch: refs/heads/master
Commit: 313591aa677b8781b40ea7bfed0deed9c84deba7
Parents: d45ccc5
Author: Jiang Yan Xu <ya...@jxu.me>
Authored: Sat Aug 29 21:19:53 2015 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Thu Sep 3 17:29:16 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   2 +
 src/slave/containerizer/provisioner.cpp         |  46 ++-
 src/slave/containerizer/provisioner.hpp         |   2 +
 src/slave/containerizer/provisioners/appc.cpp   | 378 +++++++++++++++++++
 src/slave/containerizer/provisioners/appc.hpp   |  78 ++++
 .../containerizer/provisioners/appc/paths.cpp   | 161 +++++++-
 .../containerizer/provisioners/appc/paths.hpp   |  19 +-
 src/slave/flags.cpp                             |   6 +
 src/slave/flags.hpp                             |   2 +
 .../containerizer/appc_provisioner_tests.cpp    | 112 ++++++
 10 files changed, 790 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 4b643a3..5fdca0f 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -482,6 +482,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/containerizer/mesos/containerizer.cpp			\
 	slave/containerizer/mesos/launch.cpp				\
 	slave/containerizer/provisioner.cpp				\
+	slave/containerizer/provisioners/appc.cpp			\
 	slave/containerizer/provisioners/appc/paths.cpp			\
 	slave/containerizer/provisioners/appc/spec.cpp			\
 	slave/containerizer/provisioners/appc/store.cpp			\
@@ -760,6 +761,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/launcher.hpp				\
 	slave/containerizer/linux_launcher.hpp				\
 	slave/containerizer/provisioner.hpp				\
+	slave/containerizer/provisioners/appc.hpp			\
 	slave/containerizer/provisioners/appc/paths.hpp			\
 	slave/containerizer/provisioners/appc/spec.hpp			\
 	slave/containerizer/provisioners/appc/store.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/containerizer/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner.cpp b/src/slave/containerizer/provisioner.cpp
index efc7e69..95894c0 100644
--- a/src/slave/containerizer/provisioner.cpp
+++ b/src/slave/containerizer/provisioner.cpp
@@ -16,10 +16,18 @@
  * limitations under the License.
  */
 
+#include <stout/hashset.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+
 #include "slave/containerizer/provisioner.hpp"
 
+#include "slave/containerizer/provisioners/appc.hpp"
+
 using namespace process;
 
+using std::string;
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -28,8 +36,42 @@ Try<hashmap<Image::Type, Owned<Provisioner>>> Provisioner::create(
     const Flags& flags,
     Fetcher* fetcher)
 {
-  // TODO(tnachen): Load provisioners when one of them is available.
-  return hashmap<Image::Type, Owned<Provisioner>>();
+  if (flags.provisioners.isNone()) {
+    return hashmap<Image::Type, Owned<Provisioner>>();
+  }
+
+  hashmap<Image::Type,
+          Try<Owned<Provisioner>>(*)(const Flags&, Fetcher*)> creators;
+
+  // Register all supported creators.
+  creators.put(Image::APPC, &appc::AppcProvisioner::create);
+
+  hashmap<Image::Type, Owned<Provisioner>> provisioners;
+
+  // NOTE: Change in '--provisioners' flag may result in leaked rootfs
+  // files on the disk but it's at least safe because files managed by
+  // different provisioners are totally separated.
+  foreach (const string& type,
+           strings::tokenize(flags.provisioners.get(), ",")) {
+     Image::Type imageType;
+     if (!Image::Type_Parse(strings::upper(type), &imageType)) {
+       return Error("Unknown provisioner '" + type + "'");
+     }
+
+     if (!creators.contains(imageType)) {
+       return Error("Unsupported provisioner '" + type + "'");
+     }
+
+     Try<Owned<Provisioner>> provisioner = creators[imageType](flags, fetcher);
+     if (provisioner.isError()) {
+       return Error("Failed to create '" + stringify(imageType) +
+                    "' provisioner: " + provisioner.error());
+     }
+
+     provisioners[imageType] = provisioner.get();
+  }
+
+  return provisioners;
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/containerizer/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner.hpp b/src/slave/containerizer/provisioner.hpp
index 541dd4e..9e0e0b8 100644
--- a/src/slave/containerizer/provisioner.hpp
+++ b/src/slave/containerizer/provisioner.hpp
@@ -45,6 +45,8 @@ class Provisioner
 public:
   virtual ~Provisioner() {}
 
+  // 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);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/containerizer/provisioners/appc.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc.cpp b/src/slave/containerizer/provisioners/appc.cpp
new file mode 100644
index 0000000..fc5ee19
--- /dev/null
+++ b/src/slave/containerizer/provisioners/appc.cpp
@@ -0,0 +1,378 @@
+/**
+ * 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/provisioners/appc.hpp"
+
+#include "slave/containerizer/provisioners/backend.hpp"
+
+#include "slave/containerizer/provisioners/appc/paths.hpp"
+#include "slave/containerizer/provisioners/appc/spec.hpp"
+#include "slave/containerizer/provisioners/appc/store.hpp"
+
+#include "slave/paths.hpp"
+
+using namespace process;
+
+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_backend)) {
+    return Error("The specified provisioner backend '" + flags.appc_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) {
+    if (state.executor_info().has_container() &&
+        state.executor_info().container().type() == ContainerInfo::MESOS) {
+      alive.insert(state.container_id());
+    }
+  }
+
+  // List provisioned containers; recover living ones; destroy unknown orphans.
+  // Note that known orphan containers are recovered as well and they will
+  // be destroyed by the containerizer using the normal cleanup path. See
+  // MESOS-2367 for details.
+  Try<hashmap<ContainerID, string>> containers =
+    paths::listContainers(root);
+
+  if (containers.isError()) {
+    return Failure("Failed to list the containers managed by Appc "
+                   "provisioner: " + containers.error());
+  }
+
+  // If no container has been launched the 'containers' directory will be empty.
+  foreachkey (const ContainerID& containerId, containers.get()) {
+    if (alive.contains(containerId) || orphans.contains(containerId)) {
+      Owned<Info> info = Owned<Info>(new Info());
+
+      Try<hashmap<string, hashmap<string, string>>> rootfses =
+        paths::listContainerRootfses(root, containerId);
+
+      if (rootfses.isError()) {
+        return Failure("Unable to list rootfses belonged to container '" +
+                       containerId.value() + "': " + rootfses.error());
+      }
+
+      foreachkey (const string& backend, rootfses.get()) {
+        if (!backends.contains(backend)) {
+          return Failure("Found rootfses managed by an unrecognized backend: " +
+                         backend);
+        }
+
+        info->rootfses.put(backend, rootfses.get()[backend]);
+      }
+
+      VLOG(1) << "Recovered container " << containerId;
+      infos.put(containerId, info);
+
+      continue;
+    }
+
+    // Destroy (unknown) orphan container's rootfses.
+    Try<hashmap<string, hashmap<string, string>>> rootfses =
+      paths::listContainerRootfses(root, containerId);
+
+    if (rootfses.isError()) {
+      return Failure("Unable to find rootfses for container '" +
+                     containerId.value() + "': " + rootfses.error());
+    }
+
+    foreachkey (const string& backend, rootfses.get()) {
+      if (!backends.contains(backend)) {
+        return Failure("Found rootfses managed by an unrecognized backend: " +
+                       backend);
+      }
+
+      foreachvalue (const string& rootfs, rootfses.get()[backend]) {
+        VLOG(1) << "Destroying orphan rootfs " << rootfs;
+
+        // Not waiting for the destruction and we don't care about
+        // the return value.
+        backends.get(backend).get()->destroy(rootfs)
+          .onFailed([rootfs](const std::string& error) {
+            LOG(WARNING) << "Failed to destroy orphan rootfs '" << rootfs
+                         << "': "<< error;
+          });
+      }
+    }
+  }
+
+  LOG(INFO) << "Recovered Appc provisioner rootfses";
+
+  return store->recover()
+    .then([]() -> Future<Nothing> {
+      LOG(INFO) << "Recovered Appc image store";
+      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 = paths::getContainerRootfsDir(
+      root, containerId, flags.appc_backend, rootfsId);
+
+  if (!infos.contains(containerId)) {
+    infos.put(containerId, Owned<Info>(new Info()));
+  }
+
+  infos[containerId]->rootfses[flags.appc_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_backend));
+  return backends.get(flags.appc_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));
+    }
+  }
+
+  // TODO(xujyan): Revisit the usefulness of this return value.
+  return collect(futures)
+    .then([=](const list<bool>& results) -> Future<bool> {
+      return true;
+    });
+}
+
+} // namespace appc {
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/containerizer/provisioners/appc.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc.hpp b/src/slave/containerizer/provisioners/appc.hpp
new file mode 100644
index 0000000..68e82e3
--- /dev/null
+++ b/src/slave/containerizer/provisioners/appc.hpp
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __MESOS_APPC_HPP__
+#define __MESOS_APPC_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.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 // __MESOS_APPC_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/containerizer/provisioners/appc/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc/paths.cpp b/src/slave/containerizer/provisioners/appc/paths.cpp
index 271116d..e6be851 100644
--- a/src/slave/containerizer/provisioners/appc/paths.cpp
+++ b/src/slave/containerizer/provisioners/appc/paths.cpp
@@ -16,11 +16,21 @@
  * limitations under the License.
  */
 
+#include <list>
+
+#include <glog/logging.h>
+
+#include <mesos/type_utils.hpp>
+
+#include <stout/os.hpp>
 #include <stout/path.hpp>
 
+#include <stout/os/stat.hpp>
+
 #include "slave/containerizer/provisioners/appc/paths.hpp"
 #include "slave/paths.hpp"
 
+using std::list;
 using std::string;
 
 namespace mesos {
@@ -55,7 +65,7 @@ string getImageRootfsPath(
 }
 
 
-string getImageRootfsPath(const std::string& imagePath)
+string getImageRootfsPath(const string& imagePath)
 {
   return path::join(imagePath, "rootfs");
 }
@@ -69,29 +79,158 @@ string getImageManifestPath(
 }
 
 
-string getImageManifestPath(const std::string& imagePath)
+string getImageManifestPath(const string& imagePath)
 {
   return path::join(imagePath, "manifest");
 }
 
 
+// Internal helpers for traversing the directory hierarchy.
+static string getContainersDir(const string& provisionerDir)
+{
+  return path::join(provisionerDir, "containers");
+}
+
+
+static string getContainerDir(
+    const string& containersDir,
+    const ContainerID& containerId)
+{
+  return path::join(containersDir, containerId.value());
+}
+
+
+static string getBackendsDir(const string& containerDir)
+{
+  return path::join(containerDir, "backends");
+}
+
+
+static string getBackendDir(const string& backendsDir, const string& backend)
+{
+  return path::join(backendsDir, backend);
+}
+
+
+static string getRootfsesDir(const string& backendDir)
+{
+  return path::join(backendDir, "rootfses");
+}
+
+
+static string getRootfsDir(const string& rootfsesDir, const string& roofsId)
+{
+  return path::join(rootfsesDir, roofsId);
+}
+
+
 string getContainerRootfsDir(
-    const string& rootDir,
-    const Image::Type& imageType,
+    const string& provisionerDir,
     const ContainerID& containerId,
     const string& backend,
     const string& rootfsId)
 {
-  return path::join(
-      slave::paths::getProvisionerDir(rootDir, imageType),
-      "containers",
-      containerId.value(),
-      "backends",
-      backend,
-      "rootfses",
+  return getRootfsDir(
+      getRootfsesDir(
+          getBackendDir(
+              getBackendsDir(
+                  getContainerDir(
+                      getContainersDir(provisionerDir),
+                      containerId)),
+              backend)),
       rootfsId);
 }
 
+
+Try<hashmap<ContainerID, string>> listContainers(
+    const string& provisionerDir)
+{
+  hashmap<ContainerID, string> results;
+
+  string containersDir = getContainersDir(provisionerDir);
+  if (!os::exists(containersDir)) {
+    // No container has been created yet.
+    return results;
+  }
+
+  Try<list<string>> containerIds = os::ls(containersDir);
+  if (containerIds.isError()) {
+    return Error("Unable to list the containers directory: " +
+                 containerIds.error());
+  }
+
+  foreach (const string& entry, containerIds.get()) {
+    string containerPath = path::join(containersDir, entry);
+
+    if (!os::stat::isdir(containerPath)) {
+      LOG(WARNING) << "Ignoring unexpected container entry at: "
+                   << containerPath;
+      continue;
+    }
+
+    ContainerID containerId;
+    containerId.set_value(entry);
+    results.put(containerId, containerPath);
+  }
+
+  return results;
+}
+
+
+Try<hashmap<string, hashmap<string, string>>> listContainerRootfses(
+    const string& provisionerDir,
+    const ContainerID& containerId)
+{
+  hashmap<string, hashmap<string, string>> results;
+
+  string backendsDir = getBackendsDir(
+      getContainerDir(
+          getContainersDir(provisionerDir),
+          containerId));
+
+  Try<list<string>> backends = os::ls(backendsDir);
+  if (backends.isError()) {
+    return Error("Unable to list the container directory: " + backends.error());
+  }
+
+  foreach (const string& backend, backends.get()) {
+    string backendDir = getBackendDir(backendsDir, backend);
+    if (!os::stat::isdir(backendDir)) {
+      LOG(WARNING) << "Ignoring unexpected backend entry at: " << backendDir;
+      continue;
+    }
+
+    Try<list<string>> rootfses = os::ls(getRootfsesDir(backendDir));
+    if (rootfses.isError()) {
+      return Error("Unable to list the backend directory: " + rootfses.error());
+    }
+
+    hashmap<string, string> backendResults;
+
+    foreach (const string& rootfsId, rootfses.get()) {
+      string rootfs = getRootfsDir(getRootfsesDir(backendDir), rootfsId);
+
+      if (!os::stat::isdir(rootfs)) {
+        LOG(WARNING) << "Ignoring unexpected rootfs entry at: " << backendDir;
+        continue;
+      }
+
+      backendResults.put(rootfsId, rootfs);
+    }
+
+    if (backendResults.empty()) {
+      LOG(WARNING) << "Ignoring a backend directory with no rootfs in it: "
+                   << backendDir;
+      continue;
+    }
+
+    // The rootfs directory has passed validation.
+    results.put(backend, backendResults);
+  }
+
+  return results;
+}
+
 } // namespace paths {
 } // namespace appc {
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/containerizer/provisioners/appc/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc/paths.hpp b/src/slave/containerizer/provisioners/appc/paths.hpp
index 6364737..54bab1f 100644
--- a/src/slave/containerizer/provisioners/appc/paths.hpp
+++ b/src/slave/containerizer/provisioners/appc/paths.hpp
@@ -19,11 +19,13 @@
 #ifndef __MESOS_APPC_PATHS__
 #define __MESOS_APPC_PATHS__
 
-#include <list>
 #include <string>
 
 #include <mesos/mesos.hpp>
 
+#include <stout/hashmap.hpp>
+#include <stout/try.hpp>
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -92,12 +94,23 @@ std::string getImageManifestPath(const std::string& imagePath);
 
 
 std::string getContainerRootfsDir(
-    const std::string& rootDir,
-    const Image::Type& imageType,
+    const std::string& provisionerDir,
     const ContainerID& containerId,
     const std::string& backend,
     const std::string& rootfsId);
 
+
+// Recursively "ls" the container directory and return a map of
+// backend -> rootfsId -> rootfsPath.
+Try<hashmap<std::string, hashmap<std::string, std::string>>>
+listContainerRootfses(
+    const std::string& provisionerDir,
+    const ContainerID& containerId);
+
+// Return a map of containerId -> containerPath;
+Try<hashmap<ContainerID, std::string>> listContainers(
+    const std::string& provisionerDir);
+
 } // namespace paths {
 } // namespace appc {
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index b36710d..49fb000 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -69,6 +69,12 @@ mesos::internal::slave::Flags::Flags()
       "Directory the appc provisioner will store images in",
       "/tmp/mesos/store/appc");
 
+  // TODO(xujyan): Change the default to 'copy' once it's added.
+  add(&Flags::appc_backend,
+      "appc_backend",
+      "Strategy for provisioning container rootfs from appc images",
+      "bind");
+
   add(&Flags::default_role,
       "default_role",
       "Any resources in the --resources flag that\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index e56738e..b8335aa 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -48,7 +48,9 @@ public:
   std::string isolation;
 
   Option<std::string> provisioners;
+
   std::string appc_store_dir;
+  std::string appc_backend;
 
   std::string default_role;
   Option<std::string> attributes;

http://git-wip-us.apache.org/repos/asf/mesos/blob/313591aa/src/tests/containerizer/appc_provisioner_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/appc_provisioner_tests.cpp b/src/tests/containerizer/appc_provisioner_tests.cpp
index a4526e9..f30e1a1 100644
--- a/src/tests/containerizer/appc_provisioner_tests.cpp
+++ b/src/tests/containerizer/appc_provisioner_tests.cpp
@@ -26,11 +26,14 @@
 #include <stout/path.hpp>
 #include <stout/stringify.hpp>
 
+#include "slave/containerizer/provisioner.hpp"
+
 #include "slave/containerizer/provisioners/appc/spec.hpp"
 #include "slave/containerizer/provisioners/appc/store.hpp"
 
 #include "tests/utils.hpp"
 
+using std::list;
 using std::string;
 using std::vector;
 
@@ -38,6 +41,9 @@ using namespace process;
 
 using namespace mesos::internal::slave::appc;
 
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::Provisioner;
+
 namespace mesos {
 namespace internal {
 namespace tests {
@@ -181,6 +187,112 @@ TEST_F(AppcProvisionerTest, StoreRecover)
   EXPECT_EQ(os::realpath(imagePath).get(), layers.get().front());
 }
 
+
+#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)
+{
+  // Create provisioner.
+  slave::Flags flags;
+  flags.appc_store_dir = path::join(os::getcwd(), "store");
+  flags.appc_backend = "bind";
+  flags.provisioners = "appc";
+  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));
+
+  // Create a simple image in the store:
+  // <store>
+  // |--images
+  //    |--<id>
+  //       |--manifest
+  //       |--rootfs/tmp/test
+  JSON::Value manifest = JSON::parse(
+      "{"
+      "  \"acKind\": \"ImageManifest\","
+      "  \"acVersion\": \"0.6.1\","
+      "  \"name\": \"foo.com/bar\","
+      "  \"labels\": ["
+      "    {"
+      "      \"name\": \"version\","
+      "      \"value\": \"1.0.0\""
+      "    },"
+      "    {"
+      "      \"name\": \"arch\","
+      "      \"value\": \"amd64\""
+      "    },"
+      "    {"
+      "      \"name\": \"os\","
+      "      \"value\": \"linux\""
+      "    }"
+      "  ],"
+      "  \"annotations\": ["
+      "    {"
+      "      \"name\": \"created\","
+      "      \"value\": \"1438983392\""
+      "    }"
+      "  ]"
+      "}").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 images under 'images' have passed such check when they are
+  // downloaded and validated.
+  string imageId =
+    "sha512-e77d96aa0240eedf134b8c90baeaf76dca8e78691836301d7498c84020446042e"
+    "797b296d6ab296e0954c2626bfb264322ebeb8f447dac4fac6511ea06bc61f0";
+
+  string imagePath = path::join(flags.appc_store_dir, "images", imageId);
+
+  ASSERT_SOME(os::mkdir(path::join(imagePath, "rootfs", "tmp")));
+  ASSERT_SOME(
+      os::write(path::join(imagePath, "rootfs", "tmp", "test"), "test"));
+  ASSERT_SOME(
+      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({}, {}));
+
+  // Simulate a task that requires an image.
+  Image image;
+  image.mutable_appc()->set_name("foo.com/bar");
+
+  ContainerID containerId;
+  containerId.set_value("12345");
+
+  Future<string> rootfs =
+    provisioners.get()[Image::APPC]->provision(containerId, image);
+  AWAIT_READY(rootfs);
+
+  Try<list<string>> rootfses = os::ls(path::join(
+      flags.work_dir,
+      "provisioners",
+      stringify(Image::APPC),
+      "containers",
+      containerId.value(),
+      "backends",
+      flags.appc_backend,
+      "rootfses"));
+
+  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());
+
+  Future<bool> destroy = provisioners.get()[Image::APPC]->destroy(containerId);
+  AWAIT_READY(destroy);
+
+  // One rootfs is destroyed.
+  EXPECT_TRUE(destroy.get());
+}
+#endif // __linux__
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[3/3] mesos git commit: Changed provisioner Store API and implementation so it works as a read-through cache.

Posted by ya...@apache.org.
Changed provisioner Store API and implementation so it works as a read-through cache.

- i.e., It fetches images transparently when it's not in the local cache.
- This way, the store doesn't have the sense of "localness" anymore but rather it's an abstraction that provides access to all discoverable images, no matter where they can be found.

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


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

Branch: refs/heads/master
Commit: eeb4227adc24d1a1966339b5c0966139f1c9611e
Parents: fb63151
Author: Jiang Yan Xu <ya...@jxu.me>
Authored: Sat Aug 29 21:19:34 2015 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Thu Sep 3 17:29:16 2015 -0700

----------------------------------------------------------------------
 .../containerizer/provisioners/appc/store.cpp   | 106 ++++++++++++++++---
 .../containerizer/provisioners/appc/store.hpp   |  50 ++++-----
 .../containerizer/appc_provisioner_tests.cpp    |  12 +--
 3 files changed, 120 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/eeb4227a/src/slave/containerizer/provisioners/appc/store.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc/store.cpp b/src/slave/containerizer/provisioners/appc/store.cpp
index fbd1c53..04ab2ad 100644
--- a/src/slave/containerizer/provisioners/appc/store.cpp
+++ b/src/slave/containerizer/provisioners/appc/store.cpp
@@ -43,6 +43,69 @@ namespace internal {
 namespace slave {
 namespace appc {
 
+// Defines a locally cached image (which has passed validation).
+struct CachedImage
+{
+  CachedImage(
+      const AppcImageManifest& _manifest,
+      const string& _id,
+      const string& _path)
+    : manifest(_manifest), id(_id), path(_path) {}
+
+  const AppcImageManifest manifest;
+
+  // Image ID of the format "sha512-value" where "value" is the hex
+  // encoded string of the sha512 digest of the uncompressed tar file
+  // of the image.
+  const string id;
+
+  // Absolute path to the extracted image.
+  const string path;
+};
+
+
+// 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)
+{
+  // The name must match.
+  if (candidate.manifest.name() != requirements.name()) {
+    return false;
+  }
+
+  // If an id is specified the candidate must match.
+  if (requirements.has_id() && (candidate.id != requirements.id())) {
+    return false;
+  }
+
+  // Extract labels for easier comparison, this also weeds out duplicates.
+  // TODO(xujyan): Detect duplicate labels in image manifest validation
+  // and Image::Appc validation.
+  hashmap<string, string> requiredLabels;
+  foreach (const Label& label, requirements.labels().labels()) {
+    requiredLabels[label.key()] = label.value();
+  }
+
+  hashmap<string, string> candidateLabels;
+  foreach (const AppcImageManifest::Label& label,
+           candidate.manifest.labels()) {
+    candidateLabels[label.name()] = label.value();
+  }
+
+  // Any label specified must be present and match in the candidate.
+  foreachpair (const string& name,
+               const string& value,
+               requiredLabels) {
+    if (!candidateLabels.contains(name) ||
+        candidateLabels.get(name).get() != value) {
+      return false;
+    }
+  }
+
+  return true;
+}
+
+
 class StoreProcess : public Process<StoreProcess>
 {
 public:
@@ -52,7 +115,7 @@ public:
 
   Future<Nothing> recover();
 
-  Future<std::vector<Store::Image>> get(const string& name);
+  Future<vector<string>> get(const Image::Appc& image);
 
 private:
   // Absolute path to the root directory of the store as defined by
@@ -60,7 +123,7 @@ private:
   const string root;
 
   // Mappings: name -> id -> image.
-  hashmap<string, hashmap<string, Store::Image>> images;
+  hashmap<string, hashmap<string, CachedImage>> images;
 };
 
 
@@ -107,17 +170,18 @@ Future<Nothing> Store::recover()
 }
 
 
-Future<vector<Store::Image>> Store::get(const string& name)
+Future<vector<string>> Store::get(const Image::Appc& image)
 {
-  return dispatch(process.get(), &StoreProcess::get, name);
+  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 by 'put()'.
-static Try<Store::Image> createImage(const string& imagePath)
+// 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()) {
@@ -141,22 +205,34 @@ static Try<Store::Image> createImage(const string& imagePath)
     return Error("Failed to parse manifest: " + manifest.error());
   }
 
-  return Store::Image(manifest.get(), imageId, imagePath);
+  return CachedImage(manifest.get(), imageId, imagePath);
 }
 
 
-Future<vector<Store::Image>> StoreProcess::get(const string& name)
+Future<vector<string>> StoreProcess::get(const Image::Appc& image)
 {
-  if (!images.contains(name)) {
-    return vector<Store::Image>();
+  if (!images.contains(image.name())) {
+    return Failure("No image named '" + image.name() + "' can be found");
   }
 
-  vector<Store::Image> result;
-  foreach (const Store::Image& image, images[name].values()) {
-    result.push_back(image);
+  // 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.path});
+    }
   }
 
-  return result;
+  return Failure("No image named '" + image.name() +
+                 "' can match the requirements");
 }
 
 
@@ -178,7 +254,7 @@ Future<Nothing> StoreProcess::recover()
       continue;
     }
 
-    Try<Store::Image> image = createImage(path);
+    Try<CachedImage> image = createImage(path);
     if (image.isError()) {
       LOG(WARNING) << "Unexpected entry in storage: " << image.error();
       continue;

http://git-wip-us.apache.org/repos/asf/mesos/blob/eeb4227a/src/slave/containerizer/provisioners/appc/store.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc/store.hpp b/src/slave/containerizer/provisioners/appc/store.hpp
index e48d91b..95067f8 100644
--- a/src/slave/containerizer/provisioners/appc/store.hpp
+++ b/src/slave/containerizer/provisioners/appc/store.hpp
@@ -40,42 +40,38 @@ namespace appc {
 class StoreProcess;
 
 
-// Provides the provisioner with access to locally stored / cached Appc images.
+// 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
 {
 public:
-  // Defines an image in the store (which has passed validation).
-  struct Image
-  {
-    Image(
-        const AppcImageManifest& _manifest,
-        const std::string& _id,
-        const std::string& _path)
-      : manifest(_manifest), id(_id), path(_path) {}
-
-    const AppcImageManifest manifest;
-
-    // Image ID of the format "sha512-value" where "value" is the hex
-    // encoded string of the sha512 digest of the uncompressed tar file
-    // of the image.
-    const std::string id;
-
-    // Absolute path of the extracted image.
-    const std::string path;
-  };
-
   static Try<process::Owned<Store>> create(const Flags& flags);
 
   ~Store();
 
   process::Future<Nothing> recover();
 
-  // Get all images matched by name.
-  process::Future<std::vector<Image>> get(const std::string& name);
-
-  // TODO(xujyan): Implement a put() method that fetches an image into
-  // the store. i.e.,
-  // process::Future<StoredImage> put(const std::string& uri);
+  // 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);
 
 private:
   Store(process::Owned<StoreProcess> process);

http://git-wip-us.apache.org/repos/asf/mesos/blob/eeb4227a/src/tests/containerizer/appc_provisioner_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/appc_provisioner_tests.cpp b/src/tests/containerizer/appc_provisioner_tests.cpp
index 47b66b9..a4526e9 100644
--- a/src/tests/containerizer/appc_provisioner_tests.cpp
+++ b/src/tests/containerizer/appc_provisioner_tests.cpp
@@ -171,14 +171,14 @@ TEST_F(AppcProvisionerTest, StoreRecover)
   // Recover the image from disk.
   AWAIT_READY(store.get()->recover());
 
-  Future<vector<Store::Image>> _images = store.get()->get("foo.com/bar");
-  AWAIT_READY(_images);
+  Image image;
+  image.mutable_appc()->set_name("foo.com/bar");
+  Future<vector<string>> layers = store.get()->get(image.appc());
+  AWAIT_READY(layers);
 
-  vector<Store::Image> images = _images.get();
-
-  EXPECT_EQ(1u, images.size());
+  EXPECT_EQ(1u, layers.get().size());
   ASSERT_SOME(os::realpath(imagePath));
-  EXPECT_EQ(os::realpath(imagePath).get(), images.front().path);
+  EXPECT_EQ(os::realpath(imagePath).get(), layers.get().front());
 }
 
 } // namespace tests {


[2/3] mesos git commit: Changed the Appc provisioner directory to identify a rootfs by a rootfs_id (UUID).

Posted by ya...@apache.org.
Changed the Appc provisioner directory to identify a rootfs by a rootfs_id (UUID).

- So the user can provision multiple identical rootfses in the volumes of one container, a case we don't need to disallow.
- Previously a rootfs is identified by the 'image_id' of its topmost file system layer.

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


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

Branch: refs/heads/master
Commit: d45ccc5d14da54b6768161378146723c9603147f
Parents: eeb4227
Author: Jiang Yan Xu <ya...@jxu.me>
Authored: Wed Aug 26 15:24:04 2015 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Thu Sep 3 17:29:16 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/provisioners/appc/paths.cpp | 4 ++--
 src/slave/containerizer/provisioners/appc/paths.hpp | 8 ++++----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d45ccc5d/src/slave/containerizer/provisioners/appc/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc/paths.cpp b/src/slave/containerizer/provisioners/appc/paths.cpp
index 3113c84..271116d 100644
--- a/src/slave/containerizer/provisioners/appc/paths.cpp
+++ b/src/slave/containerizer/provisioners/appc/paths.cpp
@@ -80,7 +80,7 @@ string getContainerRootfsDir(
     const Image::Type& imageType,
     const ContainerID& containerId,
     const string& backend,
-    const string& imageId)
+    const string& rootfsId)
 {
   return path::join(
       slave::paths::getProvisionerDir(rootDir, imageType),
@@ -89,7 +89,7 @@ string getContainerRootfsDir(
       "backends",
       backend,
       "rootfses",
-      imageId);
+      rootfsId);
 }
 
 } // namespace paths {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d45ccc5d/src/slave/containerizer/provisioners/appc/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioners/appc/paths.hpp b/src/slave/containerizer/provisioners/appc/paths.hpp
index 41e3bf7..6364737 100644
--- a/src/slave/containerizer/provisioners/appc/paths.hpp
+++ b/src/slave/containerizer/provisioners/appc/paths.hpp
@@ -55,14 +55,14 @@ namespace paths {
 //                 |-- backends
 //                     |-- <backend> (copy, bind, etc.)
 //                         |-- rootfses
-//                             |-- <image_id> (the rootfs)
+//                             |-- <rootfs_id> (the rootfs)
 //
 // NOTE: Each container could have multiple image types, therefore there
 // can be the same <container_id> directory under other provisioners e.g.,
 // <work_dir>/provisioners/DOCKER, etc. Under each provisioner + container
 // there can be multiple backends due to the change of backend flags. For
-// appc, under each backend a rootfs is identified by the 'image_id' of
-// the topmost filesystem layer.
+// appc, under each backend a rootfs is identified by the 'rootfs_id' which
+// is a UUID.
 
 std::string getStagingDir(const std::string& storeDir);
 
@@ -96,7 +96,7 @@ std::string getContainerRootfsDir(
     const Image::Type& imageType,
     const ContainerID& containerId,
     const std::string& backend,
-    const std::string& imageId);
+    const std::string& rootfsId);
 
 } // namespace paths {
 } // namespace appc {