You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2014/05/27 23:54:05 UTC

git commit: Exposed event queue size from SchedulerProcess.

Repository: mesos
Updated Branches:
  refs/heads/master bd4dad4dc -> c5aa1dd22


Exposed event queue size from SchedulerProcess.

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


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

Branch: refs/heads/master
Commit: c5aa1dd22155d79c5a7c33076319299a40fd63b3
Parents: bd4dad4
Author: Dominic Hamon <dh...@twopensource.com>
Authored: Tue May 27 14:53:49 2014 -0700
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue May 27 14:53:50 2014 -0700

----------------------------------------------------------------------
 src/Makefile.am               |  1 +
 src/sched/sched.cpp           | 41 ++++++++++++++++
 src/tests/scheduler_tests.cpp | 96 ++++++++++++++++++++++++++++++++++++++
 3 files changed, 138 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c5aa1dd2/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index ae576c5..86fd0a4 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -948,6 +948,7 @@ mesos_tests_SOURCES =				\
   tests/resource_offers_tests.cpp		\
   tests/resources_tests.cpp			\
   tests/sasl_tests.cpp				\
+  tests/scheduler_tests.cpp			\
   tests/script.cpp				\
   tests/slave_recovery_tests.cpp		\
   tests/slave_tests.cpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/c5aa1dd2/src/sched/sched.cpp
----------------------------------------------------------------------
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index 9e2de7e..be23e01 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -40,9 +40,13 @@
 #include <process/dispatch.hpp>
 #include <process/id.hpp>
 #include <process/owned.hpp>
+#include <process/pid.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 
+#include <process/metrics/gauge.hpp>
+#include <process/metrics/metrics.hpp>
+
 #include <stout/check.hpp>
 #include <stout/duration.hpp>
 #include <stout/error.hpp>
@@ -103,6 +107,7 @@ public:
                    pthread_mutex_t* _mutex,
                    pthread_cond_t* _cond)
     : ProcessBase(ID::generate("scheduler")),
+      metrics(*this),
       driver(_driver),
       scheduler(_scheduler),
       framework(_framework),
@@ -993,6 +998,42 @@ protected:
 private:
   friend class mesos::MesosSchedulerDriver;
 
+  struct Metrics
+  {
+    Metrics(const SchedulerProcess& schedulerProcess)
+      : event_queue_size(
+          "scheduler/event_queue_size",
+          defer(schedulerProcess, &SchedulerProcess::_event_queue_size))
+    {
+      // TODO(dhamon): When we start checking the return value of 'add' we may
+      // get failures in situations where multiple SchedulerProcesses are active
+      // (ie, the fault tolerance tests). At that point we'll need MESOS-1285 to
+      // be fixed and to use self().id in the metric name.
+      process::metrics::add(event_queue_size);
+    }
+
+    ~Metrics()
+    {
+      process::metrics::remove(event_queue_size);
+    }
+
+    // Process metrics.
+    process::metrics::Gauge event_queue_size;
+  } metrics;
+
+  double _event_queue_size()
+  {
+    size_t size;
+
+    lock();
+    {
+      size = events.size();
+    }
+    unlock();
+
+    return static_cast<double>(size);
+  }
+
   MesosSchedulerDriver* driver;
   Scheduler* scheduler;
   FrameworkInfo framework;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c5aa1dd2/src/tests/scheduler_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_tests.cpp b/src/tests/scheduler_tests.cpp
new file mode 100644
index 0000000..721d1cf
--- /dev/null
+++ b/src/tests/scheduler_tests.cpp
@@ -0,0 +1,96 @@
+/**
+ * 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 <gmock/gmock.h>
+
+#include <vector>
+
+#include <mesos/scheduler.hpp>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+#include <process/http.hpp>
+#include <process/pid.hpp>
+
+#include <process/metrics/metrics.hpp>
+
+#include <stout/json.hpp>
+#include <stout/try.hpp>
+
+#include "master/master.hpp"
+
+#include "tests/mesos.hpp"
+
+using namespace mesos;
+using namespace mesos::internal;
+using namespace mesos::internal::tests;
+
+using mesos::internal::master::Master;
+
+using process::Future;
+using process::PID;
+
+using process::http::OK;
+
+using process::metrics::internal::MetricsProcess;
+
+using std::vector;
+
+using testing::_;
+using testing::Return;
+
+
+class SchedulerTest : public MesosTest {};
+
+
+TEST_F(SchedulerTest, MetricsEndpoint)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  ASSERT_EQ(DRIVER_RUNNING, driver.start());
+
+  Future<process::http::Response> response =
+    process::http::get(MetricsProcess::instance()->self(), "/snapshot");
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  EXPECT_SOME_EQ(
+      "application/json",
+      response.get().headers.get("Content-Type"));
+
+  Try<JSON::Object> parse = JSON::parse<JSON::Object>(response.get().body);
+
+  ASSERT_SOME(parse);
+
+  JSON::Object metrics = parse.get();
+
+  EXPECT_EQ(1u, metrics.values.count("scheduler/event_queue_size"));
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}