You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2017/07/31 22:46:12 UTC

[1/5] mesos git commit: Changed Device::path to optional and introduced Device::Number.

Repository: mesos
Updated Branches:
  refs/heads/master dbe1fbcba -> c65fb009e


Changed Device::path to optional and introduced Device::Number.

A device in cgroup can be represented as either a path or a major:minor
number. Need to change the `required` Device::path to `optional` to
add Number in the device protobuf.

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


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

Branch: refs/heads/master
Commit: 03b77fb4643c5e7e2957ae09a5a6ca38b6407437
Parents: dbe1fbc
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Jul 31 15:45:29 2017 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jul 31 15:45:29 2017 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto                                    | 8 +++++++-
 include/mesos/v1/mesos.proto                                 | 8 +++++++-
 .../mesos/isolators/cgroups/subsystems/devices.cpp           | 6 ++++++
 3 files changed, 20 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/03b77fb4/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 41e42b4..45253a3 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -3084,7 +3084,13 @@ message FileInfo {
  * Describes information abount a device.
  */
 message Device {
-  required string path = 1;
+  message Number {
+    required uint64 major = 1;
+    required uint64 minor = 2;
+  }
+
+  optional string path = 1;
+  optional Number number = 2;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/03b77fb4/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 9de282f..4dfc8b0 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -3067,7 +3067,13 @@ message FileInfo {
  * Describes information abount a device.
  */
 message Device {
-  required string path = 1;
+  message Number {
+    required uint64 major = 1;
+    required uint64 minor = 2;
+  }
+
+  optional string path = 1;
+  optional Number number = 2;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/03b77fb4/src/slave/containerizer/mesos/isolators/cgroups/subsystems/devices.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/devices.cpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/devices.cpp
index ddf2a4d..4c43191 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/devices.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/devices.cpp
@@ -86,6 +86,12 @@ Try<Owned<Subsystem>> DevicesSubsystem::create(
   if (flags.allowed_devices.isSome()) {
     foreach (const DeviceAccess& device_access,
              flags.allowed_devices->allowed_devices()) {
+      if (!device_access.device().has_path()) {
+        VLOG(1) << "Skipping a whitelisted device since no device "
+                << "path is provided";
+        continue;
+      }
+
       string path = device_access.device().path();
       const DeviceAccess_Access access = device_access.access();
       bool readAccess = (access.has_read() && access.read());


[4/5] mesos git commit: Implemented blkio subsystem usage() for resource statistics.

Posted by gi...@apache.org.
Implemented blkio subsystem usage() for resource statistics.

Implemented blkio subsystem usage() for resource statistics.

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


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

Branch: refs/heads/master
Commit: d5325188689c903513f0d9fa121dfdda06fb0de3
Parents: 1b89fb0
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Jul 31 15:45:40 2017 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jul 31 15:45:40 2017 -0700

----------------------------------------------------------------------
 .../isolators/cgroups/subsystems/blkio.cpp      | 335 +++++++++++++++++++
 .../isolators/cgroups/subsystems/blkio.hpp      |   4 +
 2 files changed, 339 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d5325188/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.cpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.cpp
index 6be0f9e..96014b5 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.cpp
@@ -16,11 +16,25 @@
 
 #include <process/id.hpp>
 
+#include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
+
+#include "linux/cgroups.hpp"
+
 #include "slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.hpp"
 
+namespace cfq = cgroups::blkio::cfq;
+namespace throttle = cgroups::blkio::throttle;
+
+using process::Failure;
+using process::Future;
 using process::Owned;
 
 using std::string;
+using std::vector;
+
+using cgroups::blkio::Device;
+using cgroups::blkio::Operation;
 
 namespace mesos {
 namespace internal {
@@ -40,6 +54,327 @@ BlkioSubsystem::BlkioSubsystem(
   : ProcessBase(process::ID::generate("cgroups-blkio-subsystem")),
     Subsystem(_flags, _hierarchy) {}
 
+
+static void setValue(
+    const cgroups::blkio::Value& statValue,
+    CgroupInfo::Blkio::Value* value)
+{
+  if (statValue.op.isNone()) {
+    value->set_op(CgroupInfo::Blkio::UNKNOWN);
+  } else {
+    switch(statValue.op.get()) {
+      case Operation::TOTAL:
+        value->set_op(CgroupInfo::Blkio::TOTAL);
+        break;
+      case Operation::READ:
+        value->set_op(CgroupInfo::Blkio::READ);
+        break;
+      case Operation::WRITE:
+        value->set_op(CgroupInfo::Blkio::WRITE);
+        break;
+      case Operation::SYNC:
+        value->set_op(CgroupInfo::Blkio::SYNC);
+        break;
+      case Operation::ASYNC:
+        value->set_op(CgroupInfo::Blkio::ASYNC);
+        break;
+    }
+  }
+
+  value->set_value(statValue.value);
+}
+
+
+Future<ResourceStatistics> BlkioSubsystem::usage(
+    const ContainerID& containerId,
+    const string& cgroup)
+{
+  hashmap<dev_t, CgroupInfo::Blkio::CFQ::Statistics> cfq;
+  hashmap<dev_t, CgroupInfo::Blkio::CFQ::Statistics> cfqRecursive;
+  hashmap<dev_t, CgroupInfo::Blkio::Throttling::Statistics> throttling;
+
+  CgroupInfo::Blkio::CFQ::Statistics totalCfq;
+  CgroupInfo::Blkio::CFQ::Statistics totalCfqRecursive;
+  CgroupInfo::Blkio::Throttling::Statistics totalThrottling;
+
+  // Get CFQ statistics.
+  Try<vector<cgroups::blkio::Value>> time = cfq::time(hierarchy, cgroup);
+  if (time.isError()) {
+    return Failure(time.error());
+  }
+
+  foreach (const cgroups::blkio::Value& value, time.get()) {
+    if (value.device.isNone()) {
+      totalCfq.set_time(value.value);
+    } else {
+      cfq[value.device.get()].set_time(value.value);
+    }
+  }
+
+  Try<vector<cgroups::blkio::Value>> sectors = cfq::sectors(hierarchy, cgroup);
+  if (sectors.isError()) {
+    return Failure(sectors.error());
+  }
+
+  foreach (const cgroups::blkio::Value& value, sectors.get()) {
+    if (value.device.isNone()) {
+      totalCfq.set_sectors(value.value);
+    } else {
+      cfq[value.device.get()].set_sectors(value.value);
+    }
+  }
+
+  Try<vector<cgroups::blkio::Value>> io_service_bytes =
+    cfq::io_service_bytes(hierarchy, cgroup);
+
+  if (io_service_bytes.isError()) {
+    return Failure(io_service_bytes.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_service_bytes.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfq[statValue.device.get()].add_io_service_bytes()
+      : totalCfq.add_io_service_bytes();
+
+    setValue(statValue, value);
+  }
+
+  Try<vector<cgroups::blkio::Value>> io_serviced =
+    cfq::io_serviced(hierarchy, cgroup);
+
+  if (io_serviced.isError()) {
+    return Failure(io_serviced.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_serviced.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfq[statValue.device.get()].add_io_serviced()
+      : totalCfq.add_io_serviced();
+
+    setValue(statValue, value);
+  }
+
+  Try<vector<cgroups::blkio::Value>> io_service_time =
+    cfq::io_service_time(hierarchy, cgroup);
+
+  if (io_service_time.isError()) {
+    return Failure(io_service_time.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_service_time.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfq[statValue.device.get()].add_io_service_time()
+      : totalCfq.add_io_service_time();
+
+    setValue(statValue, value);
+  }
+
+  Try<vector<cgroups::blkio::Value>> io_wait_time =
+    cfq::io_wait_time(hierarchy, cgroup);
+
+  if (io_wait_time.isError()) {
+    return Failure(io_wait_time.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_wait_time.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfq[statValue.device.get()].add_io_wait_time()
+      : totalCfq.add_io_wait_time();
+
+    setValue(statValue, value);
+  }
+
+  Try<vector<cgroups::blkio::Value>> io_merged =
+    cfq::io_merged(hierarchy, cgroup);
+
+  if (io_merged.isError()) {
+    return Failure(io_merged.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_merged.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfq[statValue.device.get()].add_io_merged()
+      : totalCfq.add_io_merged();
+
+    setValue(statValue, value);
+  }
+
+  Try<vector<cgroups::blkio::Value>> io_queued =
+    cfq::io_queued(hierarchy, cgroup);
+
+  if (io_queued.isError()) {
+    return Failure(io_queued.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_queued.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfq[statValue.device.get()].add_io_queued()
+      : totalCfq.add_io_queued();
+
+    setValue(statValue, value);
+  }
+
+  // Get CFQ recursive statistics (blkio.*_recursive).
+  time = cfq::time_recursive(hierarchy, cgroup);
+  if (time.isError()) {
+    return Failure(time.error());
+  }
+
+  foreach (const cgroups::blkio::Value& value, time.get()) {
+    if (value.device.isNone()) {
+      totalCfqRecursive.set_time(value.value);
+    } else {
+      cfqRecursive[value.device.get()].set_time(value.value);
+    }
+  }
+
+  sectors = cfq::sectors_recursive(hierarchy, cgroup);
+  if (sectors.isError()) {
+    return Failure(sectors.error());
+  }
+
+  foreach (const cgroups::blkio::Value& value, sectors.get()) {
+    if (value.device.isNone()) {
+      totalCfqRecursive.set_sectors(value.value);
+    } else {
+      cfqRecursive[value.device.get()].set_sectors(value.value);
+    }
+  }
+
+  io_service_bytes = cfq::io_service_bytes_recursive(hierarchy, cgroup);
+  if (io_service_bytes.isError()) {
+    return Failure(io_service_bytes.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_service_bytes.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfqRecursive[statValue.device.get()].add_io_service_bytes()
+      : totalCfqRecursive.add_io_service_bytes();
+
+    setValue(statValue, value);
+  }
+
+  io_serviced = cfq::io_serviced_recursive(hierarchy, cgroup);
+  if (io_serviced.isError()) {
+    return Failure(io_serviced.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_serviced.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfqRecursive[statValue.device.get()].add_io_serviced()
+      : totalCfqRecursive.add_io_serviced();
+
+    setValue(statValue, value);
+  }
+
+  io_service_time = cfq::io_service_time_recursive(hierarchy, cgroup);
+  if (io_service_time.isError()) {
+    return Failure(io_service_time.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_service_time.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfqRecursive[statValue.device.get()].add_io_service_time()
+      : totalCfqRecursive.add_io_service_time();
+
+    setValue(statValue, value);
+  }
+
+  io_wait_time = cfq::io_wait_time_recursive(hierarchy, cgroup);
+  if (io_wait_time.isError()) {
+    return Failure(io_wait_time.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_wait_time.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfqRecursive[statValue.device.get()].add_io_wait_time()
+      : totalCfqRecursive.add_io_wait_time();
+
+    setValue(statValue, value);
+  }
+
+  io_merged = cfq::io_merged_recursive(hierarchy, cgroup);
+  if (io_merged.isError()) {
+    return Failure(io_merged.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_merged.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfqRecursive[statValue.device.get()].add_io_merged()
+      : totalCfqRecursive.add_io_merged();
+
+    setValue(statValue, value);
+  }
+
+  io_queued = cfq::io_queued_recursive(hierarchy, cgroup);
+  if (io_queued.isError()) {
+    return Failure(io_queued.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_queued.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? cfqRecursive[statValue.device.get()].add_io_queued()
+      : totalCfqRecursive.add_io_queued();
+
+    setValue(statValue, value);
+  }
+
+  // Get throttling statistics.
+  io_serviced = throttle::io_serviced(hierarchy, cgroup);
+  if (io_serviced.isError()) {
+    return Failure(io_serviced.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_serviced.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? throttling[statValue.device.get()].add_io_serviced()
+      : totalThrottling.add_io_serviced();
+
+    setValue(statValue, value);
+  }
+
+  io_service_bytes = throttle::io_service_bytes(hierarchy, cgroup);
+  if (io_service_bytes.isError()) {
+    return Failure(io_service_bytes.error());
+  }
+
+  foreach (const cgroups::blkio::Value& statValue, io_service_bytes.get()) {
+    CgroupInfo::Blkio::Value* value = statValue.device.isSome()
+      ? throttling[statValue.device.get()].add_io_service_bytes()
+      : totalThrottling.add_io_service_bytes();
+
+    setValue(statValue, value);
+  }
+
+  // Add up resource statistics.
+  ResourceStatistics result;
+  CgroupInfo::Blkio::Statistics* stat = result.mutable_blkio_statistics();
+
+  foreachkey (dev_t dev, cfq) {
+    cfq[dev].mutable_device()->set_major(major(dev));
+    cfq[dev].mutable_device()->set_minor(minor(dev));
+    stat->add_cfq()->CopyFrom(cfq[dev]);
+  }
+
+  foreachkey (dev_t dev, cfqRecursive) {
+    cfqRecursive[dev].mutable_device()->set_major(major(dev));
+    cfqRecursive[dev].mutable_device()->set_minor(minor(dev));
+    stat->add_cfq_recursive()->CopyFrom(cfqRecursive[dev]);
+  }
+
+  foreachkey (dev_t dev, throttling) {
+    throttling[dev].mutable_device()->set_major(major(dev));
+    throttling[dev].mutable_device()->set_minor(minor(dev));
+    stat->add_throttling()->CopyFrom(throttling[dev]);
+  }
+
+  stat->add_cfq()->CopyFrom(totalCfq);
+  stat->add_cfq_recursive()->CopyFrom(totalCfqRecursive);
+  stat->add_throttling()->CopyFrom(totalThrottling);
+
+  return result;
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d5325188/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.hpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.hpp
index a2c575c..ebddf4e 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/blkio.hpp
@@ -49,6 +49,10 @@ public:
     return CGROUP_SUBSYSTEM_BLKIO_NAME;
   };
 
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId,
+      const std::string& cgroup);
+
 private:
   BlkioSubsystem(const Flags& flags, const std::string& hierarchy);
 };


[2/5] mesos git commit: Added protobuf scheme for blkio subsystem in CgroupInfo.

Posted by gi...@apache.org.
Added protobuf scheme for blkio subsystem in CgroupInfo.

Only statistics information for blkio in protobuf.

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


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

Branch: refs/heads/master
Commit: 9ed68402988502d29eb6a510a3686def2bfd451b
Parents: 03b77fb
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Jul 31 15:45:33 2017 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jul 31 15:45:33 2017 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto    | 68 +++++++++++++++++++++++++++++++++++++++
 include/mesos/v1/mesos.proto | 68 +++++++++++++++++++++++++++++++++++++++
 2 files changed, 136 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9ed68402/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 45253a3..f31f5bd 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -1584,6 +1584,9 @@ message ResourceStatistics {
   // Per disk (resource) statistics.
   repeated DiskStatistics disk_statistics = 43;
 
+  // Cgroups blkio statistics.
+  optional CgroupInfo.Blkio.Statistics blkio_statistics = 44;
+
   // Perf statistics.
   optional PerfStatistics perf = 13;
 
@@ -2901,6 +2904,71 @@ message ContainerStatus {
  * Linux control group (cgroup) information.
  */
 message CgroupInfo {
+  // Configuration of a blkio cgroup subsystem.
+  message Blkio {
+    enum Operation {
+      UNKNOWN = 0;
+      TOTAL = 1;
+      READ = 2;
+      WRITE = 3;
+      SYNC = 4;
+      ASYNC = 5;
+    }
+
+    // Describes a stat value without the device descriptor part.
+    message Value {
+      optional Operation op = 1; // Required.
+      optional uint64 value = 2; // Required.
+    }
+
+    message CFQ {
+      message Statistics {
+        // Stats are grouped by block devices. If `device` is not
+        // set, it represents `Total`.
+        optional Device.Number device = 1;
+        // blkio.sectors
+        optional uint64 sectors = 2;
+        // blkio.time
+        optional uint64 time = 3;
+        // blkio.io_serviced
+        repeated Value io_serviced = 4;
+        // blkio.io_service_bytes
+        repeated Value io_service_bytes = 5;
+        // blkio.io_service_time
+        repeated Value io_service_time = 6;
+        // blkio.io_wait_time
+        repeated Value io_wait_time = 7;
+        // blkio.io_merged
+        repeated Value io_merged = 8;
+        // blkio.io_queued
+        repeated Value io_queued = 9;
+      }
+
+      // TODO(jasonlai): Add fields for blkio weight and weight
+      // device.
+    }
+
+    message Throttling {
+      message Statistics {
+        // Stats are grouped by block devices. If `device` is not
+        // set, it represents `Total`.
+        optional Device.Number device = 1;
+        // blkio.throttle.io_serviced
+        repeated Value io_serviced = 2;
+        // blkio.throttle.io_service_bytes
+        repeated Value io_service_bytes = 3;
+      }
+
+      // TODO(jasonlai): Add fields for blkio.throttle.*_device.
+    }
+
+    message Statistics {
+      repeated CFQ.Statistics cfq = 1;
+      repeated CFQ.Statistics cfq_recursive = 2;
+      repeated Throttling.Statistics throttling = 3;
+    }
+  }
+
   // Configuration of a net_cls cgroup subsystem.
   message NetCls {
     // The 32-bit classid consists of two parts, a 16 bit major handle

http://git-wip-us.apache.org/repos/asf/mesos/blob/9ed68402/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 4dfc8b0..66386a8 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -1567,6 +1567,9 @@ message ResourceStatistics {
   // Per disk (resource) statistics.
   repeated DiskStatistics disk_statistics = 43;
 
+  // Cgroups blkio statistics.
+  optional CgroupInfo.Blkio.Statistics blkio_statistics = 44;
+
   // Perf statistics.
   optional PerfStatistics perf = 13;
 
@@ -2884,6 +2887,71 @@ message ContainerStatus {
  * Linux control group (cgroup) information.
  */
 message CgroupInfo {
+  // Configuration of a blkio cgroup subsystem.
+  message Blkio {
+    enum Operation {
+      UNKNOWN = 0;
+      TOTAL = 1;
+      READ = 2;
+      WRITE = 3;
+      SYNC = 4;
+      ASYNC = 5;
+    }
+
+    // Describes a stat value without the device descriptor part.
+    message Value {
+      optional Operation op = 1; // Required.
+      optional uint64 value = 2; // Required.
+    }
+
+    message CFQ {
+      message Statistics {
+        // Stats are grouped by block devices. If `device` is not
+        // set, it represents `Total`.
+        optional Device.Number device = 1;
+        // blkio.sectors
+        optional uint64 sectors = 2;
+        // blkio.time
+        optional uint64 time = 3;
+        // blkio.io_serviced
+        repeated Value io_serviced = 4;
+        // blkio.io_service_bytes
+        repeated Value io_service_bytes = 5;
+        // blkio.io_service_time
+        repeated Value io_service_time = 6;
+        // blkio.io_wait_time
+        repeated Value io_wait_time = 7;
+        // blkio.io_merged
+        repeated Value io_merged = 8;
+        // blkio.io_queued
+        repeated Value io_queued = 9;
+      }
+
+      // TODO(jasonlai): Add fields for blkio weight and weight
+      // device.
+    }
+
+    message Throttling {
+      message Statistics {
+        // Stats are grouped by block devices. If `device` is not
+        // set, it represents `Total`.
+        optional Device.Number device = 1;
+        // blkio.throttle.io_serviced
+        repeated Value io_serviced = 2;
+        // blkio.throttle.io_service_bytes
+        repeated Value io_service_bytes = 3;
+      }
+
+      // TODO(jasonlai): Add fields for blkio.throttle.*_device.
+    }
+
+    message Statistics {
+      repeated CFQ.Statistics cfq = 1;
+      repeated CFQ.Statistics cfq_recursive = 2;
+      repeated Throttling.Statistics throttling = 3;
+    }
+  }
+
   // Configuration of a net_cls cgroup subsystem.
   message NetCls {
     // The 32-bit classid consists of two parts, a 16 bit major handle


[3/5] mesos git commit: Added stats/control helpers for the Blkio cgroup subsystem.

Posted by gi...@apache.org.
Added stats/control helpers for the Blkio cgroup subsystem.

- Data structure for Blkio entities
- Stats helpers for blkio.throttle.io* (generic blkio stats)
- Stats helpers for blkio.io* (CFQ related stats)
- Comments from the kernel blkio doc for helper functions

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


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

Branch: refs/heads/master
Commit: 1b89fb07419aa53cae661a1715b3258a8e75c140
Parents: 9ed6840
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Jul 31 15:45:36 2017 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jul 31 15:45:36 2017 -0700

----------------------------------------------------------------------
 src/linux/cgroups.cpp | 345 +++++++++++++++++++++++++++++++++++++++++++++
 src/linux/cgroups.hpp | 237 +++++++++++++++++++++++++++++++
 2 files changed, 582 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1b89fb07/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 334005a..28c31f6 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -1925,6 +1925,351 @@ Result<string> cgroup(pid_t pid, const string& subsystem)
 } // namespace internal {
 
 
+namespace blkio {
+
+Result<string> cgroup(pid_t pid)
+{
+  return internal::cgroup(pid, "blkio");
+}
+
+
+Try<Device> Device::parse(const string& s)
+{
+  vector<string> device = strings::tokenize(s, ":");
+  if (device.size() != 2) {
+    return Error("Invalid major:minor device number: '" + s + "'");
+  }
+
+  Try<unsigned int> major = numify<unsigned int>(device[0]);
+  if (major.isError()) {
+    return Error("Invalid device major number: '" + device[0] + "'");
+  }
+
+  Try<unsigned int> minor = numify<unsigned int>(device[1]);
+  if (minor.isError()) {
+    return Error("Invalid device minor number: '" + device[1] + "'");
+  }
+
+  return Device(makedev(major.get(), minor.get()));
+}
+
+
+static bool isOperation(const string& s)
+{
+  return (s == "Total" ||
+          s == "Read" ||
+          s == "Write" ||
+          s == "Sync" ||
+          s == "Async");
+}
+
+
+static Try<Operation> parseOperation(const string& s)
+{
+  if (s == "Total") {
+    return Operation::TOTAL;
+  } else if (s == "Read") {
+    return Operation::READ;
+  } else if (s == "Write") {
+    return Operation::WRITE;
+  } else if (s == "Sync") {
+    return Operation::SYNC;
+  } else if (s == "Async") {
+    return Operation::ASYNC;
+  }
+
+  return Error("Invalid Operation value: '" + s + "'");
+}
+
+
+Try<Value> Value::parse(const string& s)
+{
+  vector<string> tokens = strings::tokenize(s, " ");
+  if (tokens.size() == 1) {
+    Try<uint64_t> value = numify<uint64_t>(tokens[0]);
+    if (value.isError()) {
+      return Error("Value is not a number: '" + tokens[0] + "'");
+    }
+
+    return Value{None(), None(), value.get()};
+  }
+
+  Option<Device> device;
+  int offset = 0;
+
+  if (tokens.size() == 3) {
+    Try<Device> dev = Device::parse(tokens[0]);
+    if (dev.isError()) {
+      return Error(dev.error());
+    }
+
+    device = dev.get();
+    offset++;
+  } else if (tokens.size() != 2) {
+    return Error("Invalid blkio value: '" + s + "'");
+  }
+
+  if (!isOperation(tokens[offset])) {
+    Try<Device> dev = Device::parse(tokens[offset]);
+    if (dev.isError()) {
+      return Error(dev.error());
+    }
+
+    Try<uint64_t> value = numify<uint64_t>(tokens[offset + 1]);
+    if (value.isError()) {
+      return Error("Value is not a number: '" + tokens[offset + 1] + "'");
+    }
+
+    return Value{dev.get(), None(), value.get()};
+  }
+
+  Try<Operation> operation = parseOperation(tokens[offset]);
+  if (operation.isError()) {
+    return Error(operation.error());
+  }
+
+  Try<uint64_t> value = numify<uint64_t>(tokens[offset + 1]);
+  if (value.isError()) {
+    return Error("Value is not a number: " + value.error());
+  }
+
+  return Value{device, operation.get(), value.get()};
+}
+
+
+static Try<vector<Value>> readEntries(
+    const string& hierarchy,
+    const string& cgroup,
+    const string& control)
+{
+  Try<string> read = cgroups::read(hierarchy, cgroup, control);
+  if (read.isError()) {
+    return Error("Failed to read from '" + control + "': " + read.error());
+  }
+
+  vector<Value> entries;
+
+  foreach (const string& s, strings::tokenize(read.get(), "\n")) {
+    Try<Value> value = Value::parse(s);
+    if (value.isError()) {
+      return Error("Failed to parse blkio value '" + s + "' from '" +
+                   control + "': " + value.error());
+    }
+
+    entries.push_back(value.get());
+  }
+
+  return entries;
+}
+
+
+namespace cfq {
+
+Try<vector<Value>> time(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.time");
+}
+
+
+Try<vector<Value>> time_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.time_recursive");
+}
+
+
+Try<vector<Value>> sectors(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.sectors");
+}
+
+
+Try<vector<Value>> sectors_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.sectors_recursive");
+}
+
+
+Try<vector<Value>> io_merged(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_merged");
+}
+
+
+Try<vector<Value>> io_merged_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_merged_recursive");
+}
+
+
+Try<vector<Value>> io_queued(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_queued");
+}
+
+
+Try<vector<Value>> io_queued_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_queued_recursive");
+}
+
+
+Try<vector<Value>> io_service_bytes(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_service_bytes");
+}
+
+
+Try<vector<Value>> io_service_bytes_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_service_bytes_recursive");
+}
+
+
+Try<vector<Value>> io_service_time(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_service_time");
+}
+
+
+Try<vector<Value>> io_service_time_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_service_time_recursive");
+}
+
+
+Try<vector<Value>> io_serviced(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_serviced");
+}
+
+
+Try<vector<Value>> io_serviced_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_serviced_recursive");
+}
+
+
+Try<vector<Value>> io_wait_time(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_wait_time");
+}
+
+
+Try<vector<Value>> io_wait_time_recursive(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.io_wait_time_recursive");
+}
+
+} // namespace cfq {
+
+
+namespace throttle {
+
+Try<vector<Value>> io_service_bytes(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.throttle.io_service_bytes");
+}
+
+
+Try<vector<Value>> io_serviced(
+    const string& hierarchy,
+    const string& cgroup)
+{
+  return readEntries(
+      hierarchy,
+      cgroup,
+      "blkio.throttle.io_serviced");
+}
+
+} // namespace throttle {
+} // namespace blkio {
+
+
 namespace cpu {
 
 Result<string> cgroup(pid_t pid)

http://git-wip-us.apache.org/repos/asf/mesos/blob/1b89fb07/src/linux/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.hpp b/src/linux/cgroups.hpp
index eaf0dca..964543c 100644
--- a/src/linux/cgroups.hpp
+++ b/src/linux/cgroups.hpp
@@ -401,6 +401,243 @@ Try<hashmap<std::string, uint64_t>> stat(
     const std::string& file);
 
 
+// Blkio subsystem.
+namespace blkio {
+
+// Returns the cgroup that the specified pid is a member of within the
+// hierarchy that the 'blkio' subsystem is mounted, or None if the subsystem
+// is not mounted or the pid is not a member of a cgroup.
+Result<std::string> cgroup(pid_t pid);
+
+
+// Wrapper class for dev_t.
+class Device
+{
+public:
+  constexpr Device(dev_t device) : value(device) {}
+  inline unsigned int getMajor() const { return major(value); }
+  inline unsigned int getMinor() const { return minor(value); }
+
+  inline bool operator==(const Device& that) const
+  {
+    return value == that.value;
+  }
+
+  inline bool operator!=(const Device& that) const
+  {
+    return value != that.value;
+  }
+
+  inline operator dev_t() const { return value; }
+
+public:
+  static Try<Device> parse(const std::string& s);
+
+private:
+  dev_t value;
+};
+
+
+enum class Operation {
+  TOTAL,
+  READ,
+  WRITE,
+  SYNC,
+  ASYNC,
+};
+
+
+// Entry for a blkio file. The format of each entry can either be:
+// 1. <value>
+// 2. <dev> <value>
+// 3. <dev> <op> <value>
+// 4. <op> <value>
+//
+// For details:
+// https://www.kernel.org/doc/Documentation/cgroup-v1/blkio-controller.txt
+struct Value
+{
+  Option<Device> device;
+  Option<Operation> op;
+  uint64_t value;
+
+  static Try<Value> parse(const std::string& s);
+};
+
+
+namespace cfq {
+
+Try<std::vector<Value>> time(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+Try<std::vector<Value>> time_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+Try<std::vector<Value>> sectors(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+Try<std::vector<Value>> sectors_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total number of bios/requests merged into requests
+// belonging to the given cgroup from blkio.io_merged.
+Try<std::vector<Value>> io_merged(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total number of bios/requests merged into requests
+// belonging to the given cgroup and all its descendants from
+// blkio.io_merged_recursive.
+Try<std::vector<Value>> io_merged_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total number of requests queued up in the given
+// cgroup from blkio.io_queued.
+Try<std::vector<Value>> io_queued(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total number of requests queued up in the given
+// cgroup and all its descendants from blkio.io_queued_recursive.
+Try<std::vector<Value>> io_queued_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the number of bytes transferred to/from the disk by
+// the given cgroup from blkio.io_service_bytes.
+Try<std::vector<Value>> io_service_bytes(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the number of bytes transferred to/from the disk by
+// the given cgroup and all its descendants from
+// blkio.io_service_bytes_recursive.
+Try<std::vector<Value>> io_service_bytes_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total amount of time between request dispatch and
+// completion by the IOs done by the given cgroup from
+// blkio.io_service_time.
+Try<std::vector<Value>> io_service_time(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total amount of time between request dispatch and
+// completion by the IOs done by the given cgroup and all its
+// descendants from blkio.io_service_time_recursive.
+Try<std::vector<Value>> io_service_time_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the number of IOs (bio) issued to the disk by the given
+// cgroup from blkio.io_serviced.
+Try<std::vector<Value>> io_serviced(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the number of IOs (bio) issued to the disk by the given
+// cgroup and all its descendants from blkio.io_serviced_recursive.
+Try<std::vector<Value>> io_serviced_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total amount of time the IOs for the given cgroup
+// spent waiting in the schedule queues for service from
+// blkio.io_wait_time.
+Try<std::vector<Value>> io_wait_time(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the total amount of time the IOs for the given cgroup
+// and all its descendants spent waiting in the scheduler queues
+// for service from blkio.io_wait_time_recursive.
+Try<std::vector<Value>> io_wait_time_recursive(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+} // namespace cfq {
+
+
+namespace throttle {
+
+// Returns the numbers of bytes transferred to/from the disk for
+// the given cgroup from blkio.throttle.io_service_bytes.
+Try<std::vector<Value>> io_service_bytes(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+
+// Returns the numbers of IOs (bio) issued to the disk for the
+// given cgroup from blkio.throttle.io_serviced.
+Try<std::vector<Value>> io_serviced(
+    const std::string& hierarchy,
+    const std::string& cgroup);
+
+} // namespace throttle {
+
+
+inline std::ostream& operator<<(std::ostream& stream, const Device& device)
+{
+  return stream << device.getMajor() << ':' << device.getMinor();
+}
+
+
+inline std::ostream& operator<<(std::ostream& stream, const Operation op)
+{
+  switch (op) {
+    case Operation::TOTAL:
+      return stream << "Total";
+    case Operation::READ:
+      return stream << "Read";
+    case Operation::WRITE:
+      return stream << "Write";
+    case Operation::SYNC:
+      return stream << "Sync";
+    case Operation::ASYNC:
+      return stream << "Async";
+  }
+
+  UNREACHABLE();
+}
+
+
+inline std::ostream& operator<<(std::ostream& stream, const Value& value)
+{
+  if (value.device.isSome()) {
+    stream << value.device.get() << ' ';
+  }
+
+  if (value.op.isSome()) {
+    stream << value.op.get() << ' ';
+  }
+
+  return stream << value.value;
+}
+
+} // namespace blkio {
+
+
 // Cpu controls.
 namespace cpu {
 


[5/5] mesos git commit: Fixed cgroup device Entry.Selector major/minor number type.

Posted by gi...@apache.org.
Fixed cgroup device Entry.Selector major/minor number type.

A cgroup device is identified by a major and a minor number. They
are 'unsigned int' type. A pair of <major:minor> represents a
'dev_t'. For details:
http://man7.org/linux/man-pages/man3/makedev.3.html

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


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

Branch: refs/heads/master
Commit: c65fb009e452e9a6cb0c9e2f8541423b74dd64cc
Parents: d532518
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Jul 31 15:45:43 2017 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jul 31 15:45:43 2017 -0700

----------------------------------------------------------------------
 src/linux/cgroups.cpp | 6 ++----
 src/linux/cgroups.hpp | 4 ++--
 2 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c65fb009/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 28c31f6..21f9983 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -2914,8 +2914,7 @@ Try<Entry> Entry::parse(const string& s)
   entry.selector.minor = None();
 
   if (deviceNumbers[0] != "*") {
-    Try<dev_t> major = numify<dev_t>(deviceNumbers[0]);
-
+    Try<unsigned int> major = numify<unsigned int>(deviceNumbers[0]);
     if (major.isError()) {
       return Error("Invalid format");
     }
@@ -2924,8 +2923,7 @@ Try<Entry> Entry::parse(const string& s)
   }
 
   if (deviceNumbers[1] != "*") {
-    Try<dev_t> minor = numify<dev_t>(deviceNumbers[1]);
-
+    Try<unsigned int> minor = numify<unsigned int>(deviceNumbers[1]);
     if (minor.isError()) {
       return Error("Invalid format");
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/c65fb009/src/linux/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.hpp b/src/linux/cgroups.hpp
index 964543c..e708747 100644
--- a/src/linux/cgroups.hpp
+++ b/src/linux/cgroups.hpp
@@ -891,8 +891,8 @@ struct Entry
     };
 
     Type type;
-    Option<dev_t> major; // Matches all `major` numbers if None.
-    Option<dev_t> minor; // Matches all `minor` numbers if None.
+    Option<unsigned int> major; // Matches all `major` numbers if None.
+    Option<unsigned int> minor; // Matches all `minor` numbers if None.
   };
 
   struct Access