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/22 18:08:44 UTC

mesos git commit: Added a few filesystem isolation and provisioning metrics.

Repository: mesos
Updated Branches:
  refs/heads/master beaf2228a -> fd7b8ee08


Added a few filesystem isolation and provisioning metrics.

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


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

Branch: refs/heads/master
Commit: fd7b8ee083be83eae9d720b145d7619432700dd2
Parents: beaf222
Author: Jiang Yan Xu <ya...@jxu.me>
Authored: Mon Sep 21 17:48:29 2015 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Tue Sep 22 09:08:16 2015 -0700

----------------------------------------------------------------------
 .../isolators/filesystem/linux.cpp              | 35 +++++++++-
 .../isolators/filesystem/linux.hpp              | 14 ++++
 .../containerizer/provisioner/backends/bind.cpp | 26 +++++++
 .../containerizer/provisioner/provisioner.cpp   | 71 ++++++++++++++------
 .../containerizer/filesystem_isolator_tests.cpp | 62 +++++++++++++++++
 5 files changed, 187 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fd7b8ee0/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 e821b27..514b0e0 100644
--- a/src/slave/containerizer/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/isolators/filesystem/linux.cpp
@@ -24,6 +24,8 @@
 
 #include <process/collect.hpp>
 
+#include <process/metrics/metrics.hpp>
+
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
 #include <stout/os.hpp>
@@ -145,7 +147,8 @@ LinuxFilesystemIsolatorProcess::LinuxFilesystemIsolatorProcess(
     const Flags& _flags,
     const Owned<Provisioner>& _provisioner)
   : flags(_flags),
-    provisioner(_provisioner) {}
+    provisioner(_provisioner),
+    metrics(PID<LinuxFilesystemIsolatorProcess>(this)) {}
 
 
 LinuxFilesystemIsolatorProcess::~LinuxFilesystemIsolatorProcess() {}
@@ -829,6 +832,36 @@ Future<Nothing> LinuxFilesystemIsolatorProcess::cleanup(
     .then([]() -> Future<Nothing> { return Nothing(); });
 }
 
+
+LinuxFilesystemIsolatorProcess::Metrics::Metrics(
+    const PID<LinuxFilesystemIsolatorProcess>& isolator)
+  : containers_new_rootfs(
+      "containerizer/mesos/filesystem/containers_new_rootfs",
+      defer(isolator, &LinuxFilesystemIsolatorProcess::_containers_new_rootfs))
+{
+  process::metrics::add(containers_new_rootfs);
+}
+
+
+LinuxFilesystemIsolatorProcess::Metrics::~Metrics()
+{
+  process::metrics::remove(containers_new_rootfs);
+}
+
+
+double LinuxFilesystemIsolatorProcess::_containers_new_rootfs()
+{
+  double count = 0.0;
+
+  foreachvalue (const Owned<Info>& info, infos) {
+    if (info->sandbox.isSome()) {
+      ++count;
+    }
+  }
+
+  return count;
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd7b8ee0/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 041af7b..93e85f2 100644
--- a/src/slave/containerizer/isolators/filesystem/linux.hpp
+++ b/src/slave/containerizer/isolators/filesystem/linux.hpp
@@ -23,6 +23,9 @@
 #include <mesos/resources.hpp>
 
 #include <process/owned.hpp>
+#include <process/pid.hpp>
+
+#include <process/metrics/gauge.hpp>
 
 #include <stout/hashmap.hpp>
 
@@ -126,6 +129,17 @@ private:
   };
 
   hashmap<ContainerID, process::Owned<Info>> infos;
+
+  struct Metrics
+  {
+    explicit Metrics(
+        const process::PID<LinuxFilesystemIsolatorProcess>& isolator);
+    ~Metrics();
+
+    process::metrics::Gauge containers_new_rootfs;
+  } metrics;
+
+  double _containers_new_rootfs();
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd7b8ee0/src/slave/containerizer/provisioner/backends/bind.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/backends/bind.cpp b/src/slave/containerizer/provisioner/backends/bind.cpp
index d853b49..5cd83a6 100644
--- a/src/slave/containerizer/provisioner/backends/bind.cpp
+++ b/src/slave/containerizer/provisioner/backends/bind.cpp
@@ -23,6 +23,9 @@
 #include <process/dispatch.hpp>
 #include <process/process.hpp>
 
+#include <process/metrics/counter.hpp>
+#include <process/metrics/metrics.hpp>
+
 #include <stout/foreach.hpp>
 #include <stout/os.hpp>
 
@@ -45,6 +48,14 @@ public:
   Future<Nothing> provision(const vector<string>& layers, const string& rootfs);
 
   Future<bool> destroy(const string& rootfs);
+
+  struct Metrics
+  {
+    Metrics();
+    ~Metrics();
+
+    process::metrics::Counter remove_rootfs_errors;
+  } metrics;
 };
 
 
@@ -180,6 +191,7 @@ Future<bool> BindBackendProcess::destroy(const string& rootfs)
 
         if (errno == EBUSY) {
           LOG(ERROR) << message;
+          ++metrics.remove_rootfs_errors;
         } else {
           return Failure(message);
         }
@@ -192,6 +204,20 @@ Future<bool> BindBackendProcess::destroy(const string& rootfs)
   return false;
 }
 
+
+BindBackendProcess::Metrics::Metrics()
+  : remove_rootfs_errors(
+      "containerizer/mesos/provisioner/bind/remove_rootfs_errors")
+{
+  process::metrics::add(remove_rootfs_errors);
+}
+
+
+BindBackendProcess::Metrics::~Metrics()
+{
+  process::metrics::remove(remove_rootfs_errors);
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd7b8ee0/src/slave/containerizer/provisioner/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/provisioner/provisioner.cpp b/src/slave/containerizer/provisioner/provisioner.cpp
index 213f8a6..30a85a5 100644
--- a/src/slave/containerizer/provisioner/provisioner.cpp
+++ b/src/slave/containerizer/provisioner/provisioner.cpp
@@ -23,6 +23,9 @@
 #include <process/dispatch.hpp>
 #include <process/process.hpp>
 
+#include <process/metrics/counter.hpp>
+#include <process/metrics/metrics.hpp>
+
 #include <stout/foreach.hpp>
 #include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
@@ -73,6 +76,8 @@ private:
       const ContainerID& containerId,
       const vector<string>& layers);
 
+  Future<bool> _destroy(const ContainerID& containerId);
+
   const Flags flags;
 
   // Absolute path to the provisioner root directory. It can be
@@ -92,6 +97,14 @@ private:
   };
 
   hashmap<ContainerID, Owned<Info>> infos;
+
+  struct Metrics
+  {
+    Metrics();
+    ~Metrics();
+
+    process::metrics::Counter remove_container_errors;
+  } metrics;
 };
 
 
@@ -382,30 +395,48 @@ Future<bool> ProvisionerProcess::destroy(const ContainerID& containerId)
     }
   }
 
-  // NOTE: We calculate 'containerDir' here so that the following
-  // lambda does not need to bind 'this'.
+  // TODO(xujyan): Revisit the usefulness of this return value.
+  return collect(futures)
+    .then(defer(self(), &ProvisionerProcess::_destroy, containerId));
+}
+
+
+Future<bool> ProvisionerProcess::_destroy(const ContainerID& containerId)
+{
+  // 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.
   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();
-      }
+  Try<Nothing> rmdir = os::rmdir(containerDir);
+  if (rmdir.isError()) {
+    LOG(ERROR) << "Failed to remove the provisioned container directory "
+               << "at '" << containerDir << "': " << rmdir.error();
 
-      return true;
-    });
+    ++metrics.remove_container_errors;
+  }
+
+  return true;
+}
+
+
+ProvisionerProcess::Metrics::Metrics()
+  : remove_container_errors(
+      "containerizer/mesos/provisioner/remove_container_errors")
+{
+  process::metrics::add(remove_container_errors);
+}
+
+
+ProvisionerProcess::Metrics::~Metrics()
+{
+  process::metrics::remove(remove_container_errors);
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd7b8ee0/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 e2eb049..1478531 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -23,6 +23,8 @@
 #include <process/owned.hpp>
 #include <process/gtest.hpp>
 
+#include <process/metrics/metrics.hpp>
+
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
 #include <stout/gtest.hpp>
@@ -232,6 +234,66 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
 }
 
 
+TEST_F(LinuxFilesystemIsolatorTest, ROOT_Metrics)
+{
+  slave::Flags flags = CreateSlaveFlags();
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
+  // Use a long running task so we can reliably capture the moment it's alive.
+  ExecutorInfo executor = CREATE_EXECUTOR_INFO(
+      "test_executor",
+      "sleep 1000");
+
+  executor.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+
+  string directory = path::join(os::getcwd(), "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<bool> launch = containerizer.get()->launch(
+      containerId,
+      executor,
+      directory,
+      None(),
+      SlaveID(),
+      PID<Slave>(),
+      false);
+
+  // Wait for the launch to complete.
+  AWAIT_READY(launch);
+
+  // Check metrics.
+  JSON::Object stats = Metrics();
+  EXPECT_EQ(1u, stats.values.count(
+      "containerizer/mesos/filesystem/containers_new_rootfs"));
+  EXPECT_EQ(
+      1, stats.values["containerizer/mesos/filesystem/containers_new_rootfs"]);
+
+  containerizer.get()->destroy(containerId);
+
+  // Wait on the container.
+  Future<containerizer::Termination> wait =
+    containerizer.get()->wait(containerId);
+
+  AWAIT_READY(wait);
+
+  // Executor was killed.
+  EXPECT_TRUE(wait.get().has_status());
+  EXPECT_EQ(9, wait.get().status());
+}
+
+
 // This test verifies that a volume with a relative host path is
 // properly created in the container's sandbox and is properly mounted
 // in the container's mount namespace.