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

[5/6] mesos git commit: Removed unused per-process perf sampling code for simplification.

Removed unused per-process perf sampling code for simplification.

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


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

Branch: refs/heads/master
Commit: 3aecee03f72c5eb2130229ad16934c8dba8baf79
Parents: ec58bdd
Author: Paul Brett <pa...@twopensource.com>
Authored: Mon Aug 31 18:02:00 2015 -0700
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Mon Aug 31 19:10:15 2015 -0700

----------------------------------------------------------------------
 src/linux/perf.cpp                        | 215 ++++++-------------------
 src/linux/perf.hpp                        |  21 ---
 src/tests/containerizer/cgroups_tests.cpp |  12 +-
 src/tests/containerizer/perf_tests.cpp    |  95 ++---------
 4 files changed, 74 insertions(+), 269 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3aecee03/src/linux/perf.cpp
----------------------------------------------------------------------
diff --git a/src/linux/perf.cpp b/src/linux/perf.cpp
index 56ef391..1dda79e 100644
--- a/src/linux/perf.cpp
+++ b/src/linux/perf.cpp
@@ -61,91 +61,8 @@ namespace perf {
 // Delimiter for fields in perf stat output.
 static const char PERF_DELIMITER[] = ",";
 
-// Use an empty string as the key for the parse output when sampling a
-// set of pids. No valid cgroup can be an empty string.
-static const char PIDS_KEY[] = "";
-
 namespace internal {
 
-vector<string> argv(
-    const set<string>& events,
-    const set<string>& cgroups,
-    const Duration& duration)
-{
-  vector<string> argv = {
-    "stat",
-
-    // System-wide collection from all CPUs.
-    "--all-cpus",
-
-    // Print counts using a CSV-style output to make it easy to import
-    // directly into spreadsheets. Columns are separated by the string
-    // specified in PERF_DELIMITER.
-    "--field-separator", PERF_DELIMITER,
-
-    // Ensure all output goes to stdout.
-    "--log-fd", "1"
-  };
-
-  // Nested loop to produce all pairings of event and cgroup.
-  foreach (const string& event, events) {
-    foreach (const string& cgroup, cgroups) {
-      argv.push_back("--event");
-      argv.push_back(event);
-      argv.push_back("--cgroup");
-      argv.push_back(cgroup);
-    }
-  }
-
-  argv.push_back("--");
-  argv.push_back("sleep");
-  argv.push_back(stringify(duration.secs()));
-
-  return argv;
-}
-
-
-vector<string> argv(
-    const set<string>& events,
-    const string& cgroup,
-    const Duration& duration)
-{
-  set<string> cgroups;
-  cgroups.insert(cgroup);
-
-  return argv(events, cgroups, duration);
-}
-
-
-vector<string> argv(
-    const set<string>& events,
-    const set<pid_t>& pids,
-    const Duration& duration)
-{
-  vector<string> argv = {
-    "stat",
-
-    // System-wide collection from all CPUs.
-    "--all-cpus",
-
-    // Print counts using a CSV-style output to make it easy to import
-    // directly into spreadsheets. Columns are separated by the string
-    // specified in PERF_DELIMITER.
-    "--field-separator", PERF_DELIMITER,
-
-    // Ensure all output goes to stdout.
-    "--log-fd", "1",
-
-    "--event", strings::join(",", events),
-    "--pid", strings::join(",", pids),
-    "--",
-    "sleep", stringify(duration.secs())
-  };
-
-  return argv;
-}
-
-
 // Normalize a perf event name. After normalization the event name
 // should match an event field in the PerfStatistics protobuf.
 inline string normalize(const string& s)
@@ -341,45 +258,6 @@ private:
   Option<Subprocess> perf;
 };
 
-
-// Helper to select a single key from the hashmap of perf statistics.
-Future<mesos::PerfStatistics> select(
-    const string& key,
-    const hashmap<string, mesos::PerfStatistics>& statistics)
-{
-  return statistics.get(key).get();
-}
-
-
-Future<hashmap<string, mesos::PerfStatistics>> sample(
-    const vector<string>& argv,
-    const Duration& duration)
-{
-  Time start = Clock::now();
-
-  Perf* perf = new Perf(argv);
-  Future<string> future = perf->future();
-  spawn(perf, true);
-
-  auto parse = [start, duration](const string& output) ->
-      Future<hashmap<string, mesos::PerfStatistics>> {
-    Try<hashmap<string, mesos::PerfStatistics>> parse = perf::parse(output);
-
-    if (parse.isError()) {
-      return Failure("Failed to parse perf sample: " + parse.error());
-    }
-
-    foreachvalue (mesos::PerfStatistics& statistics, parse.get()) {
-      statistics.set_timestamp(start.secs());
-      statistics.set_duration(duration.secs());
-    }
-
-    return parse.get();
-  };
-
-  return future.then(parse);
-}
-
 } // namespace internal {
 
 
@@ -398,53 +276,67 @@ Future<Version> version()
 };
 
 
-Future<mesos::PerfStatistics> sample(
-    const set<string>& events,
-    pid_t pid,
-    const Duration& duration)
-{
-  set<pid_t> pids;
-  pids.insert(pid);
-  return sample(events, pids, duration);
-}
-
-
-Future<mesos::PerfStatistics> sample(
+Future<hashmap<string, mesos::PerfStatistics>> sample(
     const set<string>& events,
-    const set<pid_t>& pids,
+    const set<string>& cgroups,
     const Duration& duration)
 {
   if (!supported()) {
     return Failure("Perf is not supported");
   }
 
-  return internal::sample(internal::argv(events, pids, duration), duration)
-    .then(lambda::bind(&internal::select, PIDS_KEY, lambda::_1));
-}
+  vector<string> argv = {
+    "stat",
 
+    // System-wide collection from all CPUs.
+    "--all-cpus",
 
-Future<mesos::PerfStatistics> sample(
-    const set<string>& events,
-    const string& cgroup,
-    const Duration& duration)
-{
-  set<string> cgroups;
-  cgroups.insert(cgroup);
-  return sample(events, cgroups, duration)
-    .then(lambda::bind(&internal::select, cgroup, lambda::_1));
-}
+    // Print counts using a CSV-style output to make it easy to import
+    // directly into spreadsheets. Columns are separated by the string
+    // specified in PERF_DELIMITER.
+    "--field-separator", PERF_DELIMITER,
 
+    // Ensure all output goes to stdout.
+    "--log-fd", "1"
+  };
 
-Future<hashmap<string, mesos::PerfStatistics>> sample(
-    const set<string>& events,
-    const set<string>& cgroups,
-    const Duration& duration)
-{
-  if (!supported()) {
-    return Failure("Perf is not supported");
+  // Add all pairwise combinations of event and cgroup.
+  foreach (const string& event, events) {
+    foreach (const string& cgroup, cgroups) {
+      argv.push_back("--event");
+      argv.push_back(event);
+      argv.push_back("--cgroup");
+      argv.push_back(cgroup);
+    }
   }
 
-  return internal::sample(internal::argv(events, cgroups, duration), duration);
+  argv.push_back("--");
+  argv.push_back("sleep");
+  argv.push_back(stringify(duration.secs()));
+
+  Time start = Clock::now();
+
+  internal::Perf* perf = new internal::Perf(argv);
+  Future<string> future = perf->future();
+  spawn(perf, true);
+
+  auto parse = [start, duration](const string& output) ->
+      Future<hashmap<string, mesos::PerfStatistics>> {
+    Try<hashmap<string, mesos::PerfStatistics>> parse = perf::parse(output);
+
+    if (parse.isError()) {
+      return Failure("Failed to parse perf sample: " + parse.error());
+    }
+
+    foreachvalue (mesos::PerfStatistics& statistics, parse.get()) {
+      statistics.set_timestamp(start.secs());
+      statistics.set_duration(duration.secs());
+    }
+
+    return parse.get();
+  };
+
+  return future.then(parse);
 }
 
 
@@ -492,18 +384,15 @@ Try<hashmap<string, mesos::PerfStatistics>> parse(const string& output)
 
   foreach (const string& line, strings::tokenize(output, "\n")) {
     vector<string> tokens = strings::tokenize(line, PERF_DELIMITER);
-    // Expected format for an output line is either:
-    // value,event          (when sampling pids)
-    // value,event,cgroup   (when sampling a cgroup)
-    // assuming PERF_DELIMITER = ",".
-    if (tokens.size() < 2 || tokens.size() > 3) {
+    // Expected format for an output line is: value,event,cgroup
+    // (assuming PERF_DELIMITER = ",").
+    if (tokens.size() != 3) {
       return Error("Unexpected perf output at line: " + line);
     }
 
     const string value = tokens[0];
     const string event = internal::normalize(tokens[1]);
-    // Use the special PIDS_KEY when sampling pids.
-    const string cgroup = (tokens.size() == 3 ? tokens[2] : PIDS_KEY);
+    const string cgroup = tokens[2];
 
     if (!statistics.contains(cgroup)) {
       statistics.put(cgroup, mesos::PerfStatistics());

http://git-wip-us.apache.org/repos/asf/mesos/blob/3aecee03/src/linux/perf.hpp
----------------------------------------------------------------------
diff --git a/src/linux/perf.hpp b/src/linux/perf.hpp
index fbb7649..c444456 100644
--- a/src/linux/perf.hpp
+++ b/src/linux/perf.hpp
@@ -34,20 +34,6 @@
 
 namespace perf {
 
-// Sample the perf events for process pid for duration.
-process::Future<mesos::PerfStatistics> sample(
-    const std::set<std::string>& events,
-    pid_t pid,
-    const Duration& duration);
-
-
-// Sample the perf events for processes in pids for duration.
-process::Future<mesos::PerfStatistics> sample(
-    const std::set<std::string>& events,
-    const std::set<pid_t>& pids,
-    const Duration& duration);
-
-
 // Sample the perf events for process(es) in the perf_event cgroups
 // for duration. The returned hashmap is keyed by cgroup.
 // NOTE: cgroups should be relative to the perf_event subsystem mount,
@@ -58,13 +44,6 @@ process::Future<hashmap<std::string, mesos::PerfStatistics>> sample(
     const Duration& duration);
 
 
-// Sample the perf events for process(es) in the perf_event cgroup.
-process::Future<mesos::PerfStatistics> sample(
-    const std::set<std::string>& events,
-    const std::string& cgroup,
-    const Duration& duration);
-
-
 // Validate a set of events are accepted by `perf stat`.
 bool valid(const std::set<std::string>& events);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3aecee03/src/tests/containerizer/cgroups_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cgroups_tests.cpp b/src/tests/containerizer/cgroups_tests.cpp
index ed1306b..75a3bc0 100644
--- a/src/tests/containerizer/cgroups_tests.cpp
+++ b/src/tests/containerizer/cgroups_tests.cpp
@@ -1001,19 +1001,21 @@ TEST_F(CgroupsAnyHierarchyWithPerfEventTest, ROOT_CGROUPS_Perf)
   // NOTE: Wait at least 2 seconds as we've seen some variance in how
   // well 'perf' does across Linux distributions (e.g., Ubuntu 14.04)
   // and we want to make sure that we collect some non-zero values.
-  Future<mesos::PerfStatistics> statistics =
-    perf::sample(events, TEST_CGROUPS_ROOT, Seconds(2));
+  Future<hashmap<string, mesos::PerfStatistics>> statistics =
+    perf::sample(events, {TEST_CGROUPS_ROOT}, Seconds(2));
+
   AWAIT_READY(statistics);
 
-  ASSERT_TRUE(statistics->has_cycles());
+  ASSERT_TRUE(statistics->contains(TEST_CGROUPS_ROOT));
+  ASSERT_TRUE(statistics->at(TEST_CGROUPS_ROOT).has_cycles());
 
   // TODO(benh): Some Linux distributions (Ubuntu 14.04) fail to
   // properly sample 'cycles' with 'perf', so we don't explicitly
   // check the value here. See MESOS-3082.
   // EXPECT_LT(0u, statistics->cycles());
 
-  ASSERT_TRUE(statistics->has_task_clock());
-  EXPECT_LT(0.0, statistics->task_clock());
+  ASSERT_TRUE(statistics->at(TEST_CGROUPS_ROOT).has_task_clock());
+  EXPECT_LT(0.0, statistics->at(TEST_CGROUPS_ROOT).task_clock());
 
   // Kill the child process.
   ASSERT_NE(-1, ::kill(pid, SIGKILL));

http://git-wip-us.apache.org/repos/asf/mesos/blob/3aecee03/src/tests/containerizer/perf_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/perf_tests.cpp b/src/tests/containerizer/perf_tests.cpp
index d2e3b10..8c29ca3 100644
--- a/src/tests/containerizer/perf_tests.cpp
+++ b/src/tests/containerizer/perf_tests.cpp
@@ -24,7 +24,6 @@
 
 #include <process/clock.hpp>
 #include <process/gtest.hpp>
-#include <process/reap.hpp>
 
 #include <stout/gtest.hpp>
 #include <stout/stringify.hpp>
@@ -60,29 +59,18 @@ TEST_F(PerfTest, ROOT_Events)
 
 TEST_F(PerfTest, Parse)
 {
-  // uint64 and floats should be parsed.
-  Try<hashmap<string, mesos::PerfStatistics> > parse =
-    perf::parse("123,cycles\n0.123,task-clock");
-  ASSERT_SOME(parse);
-
-  ASSERT_TRUE(parse->contains(""));
-  mesos::PerfStatistics statistics = parse->get("").get();
-
-  ASSERT_TRUE(statistics.has_cycles());
-  EXPECT_EQ(123u, statistics.cycles());
-  ASSERT_TRUE(statistics.has_task_clock());
-  EXPECT_EQ(0.123, statistics.task_clock());
+  // Parse multiple cgroups with uint64 and floats.
+  Try<hashmap<string, mesos::PerfStatistics>> parse =
+    perf::parse("123,cycles,cgroup1\n"
+                "456,cycles,cgroup2\n"
+                "0.456,task-clock,cgroup2\n"
+                "0.123,task-clock,cgroup1");
 
-  // Parse multiple cgroups.
-  parse = perf::parse("123,cycles,cgroup1\n"
-                      "456,cycles,cgroup2\n"
-                      "0.456,task-clock,cgroup2\n"
-                      "0.123,task-clock,cgroup1");
   ASSERT_SOME(parse);
-  EXPECT_FALSE(parse->contains(""));
+  EXPECT_EQ(2u, parse->size());
 
   ASSERT_TRUE(parse->contains("cgroup1"));
-  statistics = parse->get("cgroup1").get();
+  mesos::PerfStatistics statistics = parse->get("cgroup1").get();
 
   ASSERT_TRUE(statistics.has_cycles());
   EXPECT_EQ(123u, statistics.cycles());
@@ -98,19 +86,20 @@ TEST_F(PerfTest, Parse)
   EXPECT_EQ(0.456, statistics.task_clock());
 
   // Statistics reporting <not supported> should not appear.
-  parse = perf::parse("<not supported>,cycles");
+  parse = perf::parse("<not supported>,cycles,cgroup1");
   ASSERT_SOME(parse);
 
-  ASSERT_TRUE(parse->contains(""));
-  statistics = parse->get("").get();
+  ASSERT_TRUE(parse->contains("cgroup1"));
+  statistics = parse->get("cgroup1").get();
   EXPECT_FALSE(statistics.has_cycles());
 
   // Statistics reporting <not counted> should be zero.
-  parse = perf::parse("<not counted>,cycles\n<not counted>,task-clock");
+  parse = perf::parse("<not counted>,cycles,cgroup1\n"
+                      "<not counted>,task-clock,cgroup1");
   ASSERT_SOME(parse);
 
-  ASSERT_TRUE(parse->contains(""));
-  statistics = parse->get("").get();
+  ASSERT_TRUE(parse->contains("cgroup1"));
+  statistics = parse->get("cgroup1").get();
 
   EXPECT_TRUE(statistics.has_cycles());
   EXPECT_EQ(0u, statistics.cycles());
@@ -125,60 +114,6 @@ TEST_F(PerfTest, Parse)
   EXPECT_ERROR(parse);
 }
 
-
-TEST_F(PerfTest, ROOT_SamplePid)
-{
-  // TODO(idownes): Replace this with a Subprocess when it supports
-  // DEATHSIG.
-  // Fork a child which we'll run perf against.
-  pid_t pid = fork();
-  ASSERT_GE(pid, 0);
-
-  if (pid == 0) {
-    // Kill ourself if the parent dies to prevent leaking the child.
-    prctl(PR_SET_PDEATHSIG, SIGKILL);
-
-    // Spin child to consume cpu cycles.
-    while (true);
-  }
-
-  // Continue in parent.
-  set<string> events;
-  // Hardware event.
-  events.insert("cycles");
-  // Software event.
-  events.insert("task-clock");
-
-  // Sample the child.
-  Duration duration = Milliseconds(100);
-  Future<mesos::PerfStatistics> statistics =
-    perf::sample(events, pid, duration);
-  AWAIT_READY(statistics);
-
-  // Kill the child and reap it.
-  Future<Option<int>> status = reap(pid);
-  kill(pid, SIGKILL);
-  AWAIT_READY(status);
-
-  // Check the sample timestamp is within the last 5 seconds. This is generous
-  // because there's the process reap delay in addition to the sampling
-  // duration.
-  ASSERT_TRUE(statistics->has_timestamp());
-  EXPECT_GT(
-      Seconds(5).secs(), Clock::now().secs() - statistics->timestamp());
-  EXPECT_EQ(duration.secs(), statistics->duration());
-
-  ASSERT_TRUE(statistics->has_cycles());
-
-  // TODO(benh): Some Linux distributions (Ubuntu 14.04) fail to
-  // properly sample 'cycles' with 'perf', so we don't explicitly
-  // check the value here. See MESOS-3082.
-  // EXPECT_LT(0u, statistics->cycles());
-
-  ASSERT_TRUE(statistics->has_task_clock());
-  EXPECT_LT(0.0, statistics->task_clock());
-}
-
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {