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 2016/02/06 04:20:59 UTC

mesos git commit: Modified the cgroup net_cls isolator to use NetClsHandleManager.

Repository: mesos
Updated Branches:
  refs/heads/master 4dbebcfaf -> bf1309ce9


Modified the cgroup net_cls isolator to use NetClsHandleManager.

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


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

Branch: refs/heads/master
Commit: bf1309ce9a7d8228bb4a75bb8312edd00ba1b588
Parents: 4dbebcf
Author: Avinash sridharan <av...@mesosphere.io>
Authored: Fri Feb 5 15:41:33 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Feb 5 19:20:12 2016 -0800

----------------------------------------------------------------------
 .../mesos/isolators/cgroups/net_cls.cpp         | 193 +++++++++++++++----
 .../mesos/isolators/cgroups/net_cls.hpp         |  37 +++-
 2 files changed, 191 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bf1309ce/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp b/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
index f96c327..f918c66 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/net_cls.cpp
@@ -22,6 +22,7 @@
 #include <process/future.hpp>
 #include <process/pid.hpp>
 
+#include <stout/numify.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/try.hpp>
@@ -74,8 +75,22 @@ static string hexify(uint32_t handle)
 // handle more efficient. One idea for making it more efficient would
 // be to store the last allocated handle and start the search at this
 // position and performing a circular search on the bitmap.
-Try<NetClsHandle> NetClsHandleManager::alloc(uint16_t primary)
+Try<NetClsHandle> NetClsHandleManager::alloc(
+    const Option<uint16_t>& _primary)
 {
+  uint16_t primary;
+  if (_primary.isNone()) {
+    // Currently, the interval set `primaries` is assumed to be a
+    // singleton. The singleton is used as the primary handle for all
+    // net_cls operations in this isolator. In the future, the
+    // `cgroups/net_cls` isolator will take in ranges instead of a
+    // singleton value. At that point we might not need a default
+    // primary handle.
+    primary = (*primaries.begin()).lower();
+  } else {
+    primary = _primary.get();
+  }
+
   if (!primaries.contains(primary)) {
     return Error(
         "Primary handle " + hexify(primary) +
@@ -185,16 +200,6 @@ Try<bool> NetClsHandleManager::isUsed(const NetClsHandle& handle)
 }
 
 
-CgroupsNetClsIsolatorProcess::CgroupsNetClsIsolatorProcess(
-    const Flags& _flags,
-    const string& _hierarchy)
-  : flags(_flags),
-    hierarchy(_hierarchy) {}
-
-
-CgroupsNetClsIsolatorProcess::~CgroupsNetClsIsolatorProcess() {}
-
-
 Try<Isolator*> CgroupsNetClsIsolatorProcess::create(const Flags& flags)
 {
   Try<string> hierarchy = cgroups::prepare(
@@ -228,13 +233,32 @@ Try<Isolator*> CgroupsNetClsIsolatorProcess::create(const Flags& flags)
     }
   }
 
-  process::Owned<MesosIsolatorProcess> process(
-      new CgroupsNetClsIsolatorProcess(flags, hierarchy.get()));
+  Owned<MesosIsolatorProcess> process(
+      new CgroupsNetClsIsolatorProcess(
+          flags,
+          hierarchy.get(),
+          IntervalSet<uint16_t>()));
 
   return new MesosIsolator(process);
 }
 
 
+CgroupsNetClsIsolatorProcess::CgroupsNetClsIsolatorProcess(
+    const Flags& _flags,
+    const string& _hierarchy,
+    const IntervalSet<uint16_t>& primaries)
+  : flags(_flags),
+    hierarchy(_hierarchy)
+{
+  if (!primaries.empty()) {
+    handleManager = NetClsHandleManager(primaries);
+  }
+}
+
+
+CgroupsNetClsIsolatorProcess::~CgroupsNetClsIsolatorProcess() {}
+
+
 Future<Nothing> CgroupsNetClsIsolatorProcess::recover(
     const list<ContainerState>& states,
     const hashset<ContainerID>& orphans)
@@ -246,8 +270,8 @@ Future<Nothing> CgroupsNetClsIsolatorProcess::recover(
     Try<bool> exists = cgroups::exists(hierarchy, cgroup);
     if (exists.isError()) {
       infos.clear();
-      return Failure("Failed to check cgroup for container '" +
-                     stringify(containerId) + "'");
+      return Failure(
+          "Failed to check cgroup for container " + stringify(containerId));
     }
 
     if (!exists.get()) {
@@ -259,7 +283,20 @@ Future<Nothing> CgroupsNetClsIsolatorProcess::recover(
       continue;
     }
 
-    infos.emplace(containerId, Info(cgroup));
+    // Read the net_cls handle.
+    Result<NetClsHandle> handle = recoverHandle(hierarchy, cgroup);
+    if (handle.isError()) {
+      infos.clear();
+      return Failure(
+          "Failed to recover the net_cls handle for " +
+          stringify(containerId) + ": " + handle.error());
+    }
+
+    if (handle.isSome()) {
+      infos.emplace(containerId, Info(cgroup, handle.get()));
+    } else {
+      infos.emplace(containerId, Info(cgroup));
+    }
   }
 
   // Remove orphan cgroups.
@@ -284,16 +321,30 @@ Future<Nothing> CgroupsNetClsIsolatorProcess::recover(
       continue;
     }
 
+    Result<NetClsHandle> handle = recoverHandle(hierarchy, cgroup);
+    if (handle.isError()) {
+      infos.clear();
+      return Failure(
+          "Failed to recover the net_cls handle for orphan container " +
+          stringify(containerId) + ": " + handle.error());
+    }
+
     // Known orphan cgroups will be destroyed by the containerizer
     // using the normal cleanup path. See MESOS-2367 for details.
     if (orphans.contains(containerId)) {
-      infos.emplace(containerId, Info(cgroup));
+      if (handle.isSome()) {
+        infos.emplace(containerId, Info(cgroup, handle.get()));
+      } else {
+        infos.emplace(containerId, Info(cgroup));
+      }
+
       continue;
     }
 
     LOG(INFO) << "Removing unknown orphaned cgroup '" << cgroup << "'";
 
     // We don't wait on the destroy as we don't want to block recovery.
+    // TODO(jieyu): Release the handle after destroy has been done.
     cgroups::destroy(hierarchy, cgroup, cgroups::DESTROY_TIMEOUT);
   }
 
@@ -338,18 +389,18 @@ Future<Option<ContainerLaunchInfo>> CgroupsNetClsIsolatorProcess::prepare(
 
   // Use this info to create the cgroup, but do not insert it into
   // infos till the cgroup has been created successfully.
-  Info info(path::join(flags.cgroups_root, containerId.value()));
+  const string cgroup = path::join(flags.cgroups_root, containerId.value());
 
   // Create a cgroup for this container.
-  Try<bool> exists = cgroups::exists(hierarchy, info.cgroup);
+  Try<bool> exists = cgroups::exists(hierarchy, cgroup);
   if (exists.isError()) {
-    return Failure("Failed to check if the cgroup already exists: " +
-                   exists.error());
+    return Failure(
+        "Failed to check if the cgroup already exists: " + exists.error());
   } else if (exists.get()) {
     return Failure("The cgroup already exists");
   }
 
-  Try<Nothing> create = cgroups::create(hierarchy, info.cgroup);
+  Try<Nothing> create = cgroups::create(hierarchy, cgroup);
   if (create.isError()) {
     return Failure("Failed to create the cgroup: " + create.error());
   }
@@ -360,16 +411,27 @@ Future<Option<ContainerLaunchInfo>> CgroupsNetClsIsolatorProcess::prepare(
   if (containerConfig.has_user()) {
     Try<Nothing> chown = os::chown(
         containerConfig.user(),
-        path::join(hierarchy, info.cgroup),
+        path::join(hierarchy, cgroup),
         false);
 
     if (chown.isError()) {
-      return Failure("Failed to change ownership of cgroup hierarchy: " +
-                     chown.error());
+      return Failure(
+          "Failed to change ownership of cgroup hierarchy: " + chown.error());
     }
   }
 
-  infos.emplace(containerId, info);
+  if (handleManager.isSome()) {
+    Try<NetClsHandle> handle = handleManager->alloc();
+    if (handle.isError()) {
+      return Failure (
+          "Failed to allocate a net_cls handle: " +
+          handle.error());
+    }
+
+    infos.emplace(containerId, Info(cgroup, handle.get()));
+  } else {
+    infos.emplace(containerId, Info(cgroup));
+  }
 
   return update(containerId, containerConfig.executor_info().resources())
     .then([]() -> Future<Option<ContainerLaunchInfo>> {
@@ -390,10 +452,25 @@ Future<Nothing> CgroupsNetClsIsolatorProcess::isolate(
 
   Try<Nothing> assign = cgroups::assign(hierarchy, info.cgroup, pid);
   if (assign.isError()) {
-    return Failure("Failed to assign container '" +
-                   stringify(containerId) + "' to its own cgroup '" +
-                   path::join(hierarchy, info.cgroup) +
-                   "': " + assign.error());
+    return Failure(
+        "Failed to assign container " + stringify(containerId) +
+        " to its own cgroup '" + path::join(hierarchy, info.cgroup) +
+        "': " + assign.error());
+  }
+
+  // If info.handle is not specified, the assumption is that the
+  // operator is responsible for assigning the net_cls handles.
+  if (info.handle.isSome()) {
+    Try<Nothing> write = cgroups::net_cls::classid(
+        hierarchy,
+        info.cgroup,
+        info.handle->get());
+
+    if (write.isError()) {
+      return Failure(
+          "Failed to assign a net_cls handle to the cgroup: " +
+          write.error());
+    }
   }
 
   return Nothing();
@@ -427,10 +504,58 @@ Future<Nothing> CgroupsNetClsIsolatorProcess::cleanup(
   const Info& info = infos.at(containerId);
 
   return cgroups::destroy(hierarchy, info.cgroup, cgroups::DESTROY_TIMEOUT)
-    .then(defer(PID<CgroupsNetClsIsolatorProcess>(this), [=]() {
-      infos.erase(containerId);
-      return Nothing();
-    }));
+    .then(defer(PID<CgroupsNetClsIsolatorProcess>(this),
+                &CgroupsNetClsIsolatorProcess::_cleanup,
+                containerId));
+}
+
+
+Future<Nothing> CgroupsNetClsIsolatorProcess::_cleanup(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Nothing();
+  }
+
+  const Info& info = infos.at(containerId);
+
+  if (info.handle.isSome()) {
+    Try<Nothing> free = handleManager->free(info.handle.get());
+    if (free.isError()) {
+      return Failure("Could not free the net_cls handle: " + free.error());
+    }
+  }
+
+  infos.erase(containerId);
+
+  return Nothing();
+}
+
+
+Result<NetClsHandle> CgroupsNetClsIsolatorProcess::recoverHandle(
+    const std::string& hierarchy,
+    const std::string& cgroup)
+{
+  Try<uint32_t> classid = cgroups::net_cls::classid(hierarchy, cgroup);
+  if (classid.isError()) {
+    return Error("Failed to read 'net_cls.classid': " + classid.error());
+  }
+
+  if (classid.get() == 0) {
+    return None();
+  }
+
+  NetClsHandle handle(classid.get());
+
+  // Mark the handle as used in handle manager.
+  if (handleManager.isSome()) {
+    Try<Nothing> reserve = handleManager->reserve(handle);
+    if (reserve.isError()) {
+      return Error("Failed to reserve the handle: " + reserve.error());
+    }
+  }
+
+  return handle;
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/bf1309ce/src/slave/containerizer/mesos/isolators/cgroups/net_cls.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/net_cls.hpp b/src/slave/containerizer/mesos/isolators/cgroups/net_cls.hpp
index d8cebd7..3f8d07c 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/net_cls.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/net_cls.hpp
@@ -25,6 +25,7 @@
 
 #include <stout/hashmap.hpp>
 #include <stout/interval.hpp>
+#include <stout/none.hpp>
 #include <stout/nothing.hpp>
 #include <stout/result.hpp>
 #include <stout/try.hpp>
@@ -55,6 +56,19 @@ struct NetClsHandle
     secondary = handle & 0xffff;
   };
 
+  // Get the 32-bit representation of the handle in the form of
+  // 0xAAAABBBB. Where 0xAAAA is the primary handle and 0xBBBB is the
+  // secondary handle.
+  uint32_t get() const
+  {
+    uint32_t handle = primary;
+
+    handle <<= 16;
+    handle |= secondary;
+
+    return handle;
+  };
+
   uint16_t primary;
   uint16_t secondary;
 };
@@ -78,7 +92,7 @@ public:
 
   // Allocates a primary handle from the given interval set.
   Try<uint16_t> allocPrimary() { return Error("Not Implemented"); }
-  Try<NetClsHandle> alloc(uint16_t primary);
+  Try<NetClsHandle> alloc(const Option<uint16_t>& primary = None());
 
   Try<Nothing> reserve(const NetClsHandle& handle);
   Try<Nothing> free(const NetClsHandle& handle);
@@ -135,22 +149,35 @@ public:
       const ContainerID& containerId);
 
 private:
-  CgroupsNetClsIsolatorProcess(
-      const Flags& flags,
-      const std::string& hierarchy);
-
   struct Info
   {
     Info(const std::string& _cgroup)
       : cgroup(_cgroup) {}
 
+    Info(const std::string& _cgroup, const NetClsHandle &_handle)
+      : cgroup(_cgroup), handle(_handle) {}
+
     const std::string cgroup;
+    const Option<NetClsHandle> handle;
   };
 
+  CgroupsNetClsIsolatorProcess(
+      const Flags& _flags,
+      const std::string& _hierarchy,
+      const IntervalSet<uint16_t>& primaries);
+
+  process::Future<Nothing> _cleanup(
+      const ContainerID& containerId);
+
+  Result<NetClsHandle> recoverHandle(
+      const std::string& hierarchy,
+      const std::string& cgroup);
+
   const Flags flags;
   const std::string hierarchy;
 
   hashmap<ContainerID, Info> infos;
+  Option<NetClsHandleManager> handleManager;
 };
 
 } // namespace slave {