You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by nn...@apache.org on 2015/06/05 02:08:34 UTC

[1/4] mesos git commit: Added QoS Controller in slave.

Repository: mesos
Updated Branches:
  refs/heads/master 9e23a3e6f -> 3e139c61d


Added QoS Controller in slave.

Wired up QoS Controller in slave and necessary test code for the new
slave constructor argument.

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


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

Branch: refs/heads/master
Commit: a2d151554a1028b83fc1afe14204ad1409e79663
Parents: f6cad2e
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Thu Jun 4 14:30:12 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Thu Jun 4 16:44:24 2015 -0700

----------------------------------------------------------------------
 src/local/local.cpp   | 23 ++++++++++++++++++++++-
 src/slave/flags.hpp   |  1 +
 src/slave/main.cpp    | 16 +++++++++++++++-
 src/slave/slave.cpp   | 35 +++++++++++++++++++++++++++++++++--
 src/slave/slave.hpp   | 11 ++++++++++-
 src/tests/cluster.hpp | 17 +++++++++++++++--
 src/tests/mesos.cpp   |  6 ++++--
 src/tests/mesos.hpp   | 35 ++++++++++++++++++++++++++++++++++-
 8 files changed, 134 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 6bf4b77..1953d84 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -93,6 +93,7 @@ using mesos::internal::slave::StatusUpdateManager;
 using mesos::modules::Anonymous;
 using mesos::modules::ModuleManager;
 
+using mesos::slave::QoSController;
 using mesos::slave::ResourceEstimator;
 
 using process::Owned;
@@ -128,6 +129,7 @@ static vector<GarbageCollector*>* garbageCollectors = NULL;
 static vector<StatusUpdateManager*>* statusUpdateManagers = NULL;
 static vector<Fetcher*>* fetchers = NULL;
 static vector<ResourceEstimator*>* resourceEstimators = NULL;
+static vector<QoSController*>* qosControllers = NULL;
 
 
 PID<Master> launch(const Flags& flags, Allocator* _allocator)
@@ -293,6 +295,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
   statusUpdateManagers = new vector<StatusUpdateManager*>();
   fetchers = new vector<Fetcher*>();
   resourceEstimators = new vector<ResourceEstimator*>();
+  qosControllers = new vector<QoSController*>();
 
   vector<UPID> pids;
 
@@ -322,6 +325,16 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
 
     resourceEstimators->push_back(resourceEstimator.get());
 
+    Try<QoSController*> qosController =
+      QoSController::create(flags.qos_controller);
+
+    if (qosController.isError()) {
+      EXIT(1) << "Failed to create QoS Controller: "
+              << qosController.error();
+    }
+
+    qosControllers->push_back(qosController.get());
+
     Try<Containerizer*> containerizer =
       Containerizer::create(flags, true, fetchers->back());
 
@@ -338,7 +351,8 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
         files,
         garbageCollectors->back(),
         statusUpdateManagers->back(),
-        resourceEstimators->back());
+        resourceEstimators->back(),
+        qosControllers->back());
 
     slaves[containerizer.get()] = slave;
 
@@ -415,6 +429,13 @@ void shutdown()
     delete resourceEstimators;
     resourceEstimators = NULL;
 
+    foreach (QoSController* controller, *qosControllers) {
+      delete controller;
+    }
+
+    delete qosControllers;
+    qosControllers = NULL;
+
     delete registrar;
     registrar = NULL;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 15dd838..32d36ac 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -105,6 +105,7 @@ public:
   std::string authenticatee;
   Option<std::string> hooks;
   Option<std::string> resource_estimator;
+  Option<std::string> qos_controller;
   Duration oversubscribed_resources_interval;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index c4d8940..af090ae 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -53,6 +53,7 @@ using namespace mesos::internal::slave;
 using mesos::modules::Anonymous;
 using mesos::modules::ModuleManager;
 
+using mesos::slave::QoSController;
 using mesos::slave::ResourceEstimator;
 
 using mesos::SlaveInfo;
@@ -203,6 +204,16 @@ int main(int argc, char** argv)
     return EXIT_FAILURE;
   }
 
+  Try<QoSController*> qosController =
+    QoSController::create(flags.qos_controller);
+
+  if (qosController.isError()) {
+    cerr << "Failed to create QoS Controller: "
+         << qosController.error() << endl;
+    return EXIT_FAILURE;
+  }
+
+
   LOG(INFO) << "Starting Mesos slave";
 
   Slave* slave = new Slave(
@@ -212,7 +223,8 @@ int main(int argc, char** argv)
       &files,
       &gc,
       &statusUpdateManager,
-      resourceEstimator.get());
+      resourceEstimator.get(),
+      qosController.get());
 
   process::spawn(slave);
   process::wait(slave->self());
@@ -221,6 +233,8 @@ int main(int argc, char** argv)
 
   delete resourceEstimator.get();
 
+  delete qosController.get();
+
   delete detector.get();
 
   delete containerizer.get();

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 1b7a24a..054929b 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -87,6 +87,7 @@
 #include "slave/slave.hpp"
 #include "slave/status_update_manager.hpp"
 
+using mesos::slave::QoSController;
 using mesos::slave::ResourceEstimator;
 
 using std::list;
@@ -118,7 +119,8 @@ Slave::Slave(const slave::Flags& _flags,
              Files* _files,
              GarbageCollector* _gc,
              StatusUpdateManager* _statusUpdateManager,
-             ResourceEstimator* _resourceEstimator)
+             ResourceEstimator* _resourceEstimator,
+             QoSController* _qosController)
   : ProcessBase(process::ID::generate("slave")),
     state(RECOVERING),
     flags(_flags),
@@ -138,7 +140,8 @@ Slave::Slave(const slave::Flags& _flags,
     authenticated(false),
     reauthenticate(false),
     executorDirectoryMaxAllowedAge(age(0)),
-    resourceEstimator(_resourceEstimator) {}
+    resourceEstimator(_resourceEstimator),
+    qosController(_qosController) {}
 
 
 Slave::~Slave()
@@ -331,6 +334,13 @@ void Slave::initialize()
             << initialize.error();
   }
 
+  // TODO(nnielsen): Pass ResourceMonitor* to 'initialize'.
+  initialize = qosController->initialize();
+  if (initialize.isError()) {
+    EXIT(1) << "Failed to initialize the QoS Controller: "
+            << initialize.error();
+  }
+
   // Ensure slave work directory exists.
   CHECK_SOME(os::mkdir(flags.work_dir))
     << "Failed to create slave work directory '" << flags.work_dir << "'";
@@ -3999,6 +4009,10 @@ void Slave::__recover(const Future<Nothing>& future)
 
     // Forward oversubscribed resources.
     forwardOversubscribed();
+
+    // Start acting on correction from QoS Controller.
+    qosController->corrections()
+      .onAny(defer(self(), &Self::qosCorrections, lambda::_1));
   } else {
     // Slave started in cleanup mode.
     CHECK_EQ("cleanup", flags.recover);
@@ -4160,6 +4174,23 @@ void Slave::_forwardOversubscribed(const Future<Resources>& oversubscribable)
 }
 
 
+void Slave::qosCorrections(
+    const Future<list<mesos::slave::QoSCorrection>>& future)
+{
+  if (!future.isReady()) {
+    LOG(WARNING) << "Failed to get corrections from QoS Controller: "
+                  << (future.isFailed() ? future.failure() : "discarded");
+  } else {
+    // TODO(nnielsen): Print correction, once the operator overload
+    // for QoSCorrection has been implemented.
+    LOG(INFO) << "Received new QoS corrections";
+  }
+
+  qosController->corrections()
+    .onAny(defer(self(), &Self::qosCorrections, lambda::_1));
+}
+
+
 // TODO(dhamon): Move these to their own metrics.hpp|cpp.
 double Slave::_tasks_staging()
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 291f086..4d2c316 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -33,6 +33,7 @@
 
 #include <mesos/module/authenticatee.hpp>
 
+#include <mesos/slave/qos_controller.hpp>
 #include <mesos/slave/resource_estimator.hpp>
 
 #include <process/http.hpp>
@@ -91,7 +92,8 @@ public:
         Files* files,
         GarbageCollector* gc,
         StatusUpdateManager* statusUpdateManager,
-        mesos::slave::ResourceEstimator* resourceEstimator);
+        mesos::slave::ResourceEstimator* resourceEstimator,
+        mesos::slave::QoSController* qosController);
 
   virtual ~Slave();
 
@@ -351,6 +353,11 @@ public:
   // Called when the slave was signaled from the specified user.
   void signaled(int signal, int uid);
 
+  // Made 'virtual' for Slave mocking.
+  virtual void qosCorrections(
+      const process::Future<std::list<
+          mesos::slave::QoSCorrection>>& correction);
+
 private:
   void _authenticate();
   void authenticationTimeout(process::Future<bool> future);
@@ -511,6 +518,8 @@ private:
 
   mesos::slave::ResourceEstimator* resourceEstimator;
 
+  mesos::slave::QoSController* qosController;
+
   // The most recent estimate of the total amount of oversubscribed
   // (allocated and oversubscribable) resources.
   Resources oversubscribedResources;

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/tests/cluster.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.hpp b/src/tests/cluster.hpp
index f5a7c54..cfe7ef0 100644
--- a/src/tests/cluster.hpp
+++ b/src/tests/cluster.hpp
@@ -173,6 +173,8 @@ public:
         const Option<slave::GarbageCollector*>& gc = None(),
         const Option<slave::StatusUpdateManager*>& statusUpdateManager = None(),
         const Option<mesos::slave::ResourceEstimator*>& resourceEstimator =
+          None(),
+        const Option<mesos::slave::QoSController*>& qosController =
           None());
 
     // Stops and cleans up a slave at the specified PID. If 'shutdown'
@@ -202,6 +204,7 @@ public:
       bool createdContainerizer; // Whether we own the containerizer.
 
       process::Owned<mesos::slave::ResourceEstimator> resourceEstimator;
+      process::Owned<mesos::slave::QoSController> qosController;
       process::Owned<slave::Fetcher> fetcher;
       process::Owned<slave::StatusUpdateManager> statusUpdateManager;
       process::Owned<slave::GarbageCollector> gc;
@@ -535,7 +538,8 @@ inline Try<process::PID<slave::Slave>> Cluster::Slaves::start(
     const Option<MasterDetector*>& detector,
     const Option<slave::GarbageCollector*>& gc,
     const Option<slave::StatusUpdateManager*>& statusUpdateManager,
-    const Option<mesos::slave::ResourceEstimator*>& resourceEstimator)
+    const Option<mesos::slave::ResourceEstimator*>& resourceEstimator,
+    const Option<mesos::slave::QoSController*>& qosController)
 {
   // TODO(benh): Create a work directory if using the default.
 
@@ -564,6 +568,14 @@ inline Try<process::PID<slave::Slave>> Cluster::Slaves::start(
     slave.resourceEstimator.reset(_resourceEstimator.get());
   }
 
+  if (qosController.isNone()) {
+    Try<mesos::slave::QoSController*> _qosController =
+      mesos::slave::QoSController::create(flags.qos_controller);
+
+    CHECK_SOME(_qosController);
+    slave.qosController.reset(_qosController.get());
+  }
+
   // Get a detector for the master(s) if one wasn't provided.
   if (detector.isNone()) {
     slave.detector = masters->detector();
@@ -588,7 +600,8 @@ inline Try<process::PID<slave::Slave>> Cluster::Slaves::start(
       &cluster->files,
       gc.get(slave.gc.get()),
       statusUpdateManager.get(slave.statusUpdateManager.get()),
-      resourceEstimator.get(slave.resourceEstimator.get()));
+      resourceEstimator.get(slave.resourceEstimator.get()),
+      qosController.get(slave.qosController.get()));
 
   process::PID<slave::Slave> pid = process::spawn(slave.slave);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index d3a8bb7..dff45b0 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -375,7 +375,8 @@ void MesosTest::ShutdownSlaves()
 
 MockSlave::MockSlave(const slave::Flags& flags,
                      MasterDetector* detector,
-                     slave::Containerizer* containerizer)
+                     slave::Containerizer* containerizer,
+                     const Option<mesos::slave::QoSController*>& _qosController)
   : slave::Slave(
       flags,
       detector,
@@ -383,7 +384,8 @@ MockSlave::MockSlave(const slave::Flags& flags,
       &files,
       &gc,
       statusUpdateManager = new slave::StatusUpdateManager(flags),
-      &resourceEstimator)
+      &resourceEstimator,
+      _qosController.isSome() ? _qosController.get() : &qosController)
 {
   // Set up default behaviors, calling the original methods.
   EXPECT_CALL(*this, runTask(_, _, _, _, _))

http://git-wip-us.apache.org/repos/asf/mesos/blob/a2d15155/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 7e134b7..86660ac 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -32,6 +32,7 @@
 
 #include <mesos/fetcher/fetcher.hpp>
 
+#include <mesos/slave/qos_controller.hpp>
 #include <mesos/slave/resource_estimator.hpp>
 
 #include <process/future.hpp>
@@ -740,6 +741,32 @@ public:
 };
 
 
+// The MockQoSController is a stub which lets tests fill the
+// correction queue for a slave.
+class MockQoSController : public mesos::slave::QoSController
+{
+public:
+  MockQoSController()
+  {
+    ON_CALL(*this, initialize())
+      .WillByDefault(Return(Nothing()));
+    EXPECT_CALL(*this, initialize())
+      .WillRepeatedly(DoDefault());
+
+    ON_CALL(*this, corrections())
+      .WillByDefault(
+          Return(process::Future<std::list<mesos::slave::QoSCorrection>>()));
+    EXPECT_CALL(*this, corrections())
+      .WillRepeatedly(DoDefault());
+  }
+
+  MOCK_METHOD0(initialize, Try<Nothing>());
+
+  MOCK_METHOD0(
+      corrections, process::Future<std::list<mesos::slave::QoSCorrection>>());
+};
+
+
 // Definition of a mock Slave to be used in tests with gmock, covering
 // potential races between runTask and killTask.
 class MockSlave : public slave::Slave
@@ -748,7 +775,8 @@ public:
   MockSlave(
       const slave::Flags& flags,
       MasterDetector* detector,
-      slave::Containerizer* containerizer);
+      slave::Containerizer* containerizer,
+      const Option<mesos::slave::QoSController*>& qosController = None());
 
   virtual ~MockSlave();
 
@@ -800,10 +828,15 @@ public:
   void unmocked___recover(
       const process::Future<Nothing>& future);
 
+  MOCK_METHOD1(qosCorrections, void(
+      const process::Future<std::list<
+          mesos::slave::QoSCorrection>>& correction));
+
 private:
   Files files;
   MockGarbageCollector gc;
   MockResourceEstimator resourceEstimator;
+  MockQoSController qosController;
   slave::StatusUpdateManager* statusUpdateManager;
 };
 


[2/4] mesos git commit: Added QoS Controller.

Posted by nn...@apache.org.
Added QoS Controller.

Added QoS Controller class and NOOP controller implementation.

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


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

Branch: refs/heads/master
Commit: f6cad2e2a63a3b76f4e152f8ecc151762174d418
Parents: 6cbd332
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Thu Jun 4 14:30:04 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Thu Jun 4 16:44:24 2015 -0700

----------------------------------------------------------------------
 include/mesos/slave/qos_controller.hpp | 68 +++++++++++++++++++++
 src/Makefile.am                        |  3 +
 src/slave/qos_controller.cpp           | 92 +++++++++++++++++++++++++++++
 src/slave/qos_controller.hpp           | 55 +++++++++++++++++
 4 files changed, 218 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f6cad2e2/include/mesos/slave/qos_controller.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/qos_controller.hpp b/include/mesos/slave/qos_controller.hpp
new file mode 100644
index 0000000..1d89acf
--- /dev/null
+++ b/include/mesos/slave/qos_controller.hpp
@@ -0,0 +1,68 @@
+/**
+ * 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.
+ */
+
+#ifndef __MESOS_SLAVE_QOS_CONTROLLER_HPP__
+#define __MESOS_SLAVE_QOS_CONTROLLER_HPP__
+
+#include <list>
+#include <string>
+
+#include <mesos/resources.hpp>
+
+#include <mesos/slave/oversubscription.hpp>
+
+#include <process/future.hpp>
+#include <process/queue.hpp>
+
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+namespace mesos {
+namespace slave {
+
+// A slave component used for oversubscription. When the revocable
+// tasks are running, it is important to constantly monitor the
+// original tasks running on those resources and guarantee performance
+// based on an SLA. In order to react to detected interference, the
+// QoS controller needs to be able to kill or throttle running
+// revocable tasks.
+class QoSController
+{
+public:
+  // Create a QoS Controller instance of the given type specified
+  // by the user. If the type is not specified, a default resource
+  // estimator instance will be created.
+  static Try<QoSController*> create(const Option<std::string>& type);
+
+  virtual ~QoSController() {}
+
+  // Initializes this QoS Controller.
+  // TODO(nnielsen): Pass ResourceMonitor* once it's exposed.
+  virtual Try<Nothing> initialize() = 0;
+
+  // A QoS Controller informs the slave about corrections to carry
+  // out, but returning futures to QoSCorrection objects. For more
+  // information, please refer to mesos.proto.
+  virtual process::Future<std::list<QoSCorrection>> corrections() = 0;
+};
+
+} // namespace slave {
+} // namespace mesos {
+
+#endif // __MESOS_SLAVE_QOS_CONTROLLER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f6cad2e2/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index d42d1b6..ec7f41f 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -388,6 +388,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/metrics.cpp						\
 	slave/monitor.cpp						\
 	slave/paths.cpp							\
+	slave/qos_controller.cpp					\
 	slave/resource_estimator.cpp					\
 	slave/slave.cpp							\
 	slave/state.cpp							\
@@ -500,6 +501,7 @@ slave_HEADERS =								\
   $(top_srcdir)/include/mesos/slave/isolator.hpp			\
   $(top_srcdir)/include/mesos/slave/oversubscription.hpp		\
   $(top_srcdir)/include/mesos/slave/oversubscription.proto		\
+  $(top_srcdir)/include/mesos/slave/qos_controller.hpp			\
   $(top_srcdir)/include/mesos/slave/resource_estimator.hpp
 
 nodist_slave_HEADERS = ../include/mesos/slave/oversubscription.pb.h
@@ -613,6 +615,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/metrics.hpp						\
 	slave/monitor.hpp						\
 	slave/paths.hpp							\
+	slave/qos_controller.hpp					\
 	slave/slave.hpp							\
 	slave/state.hpp							\
 	slave/status_update_manager.hpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/f6cad2e2/src/slave/qos_controller.cpp
----------------------------------------------------------------------
diff --git a/src/slave/qos_controller.cpp b/src/slave/qos_controller.cpp
new file mode 100644
index 0000000..81c4b3e
--- /dev/null
+++ b/src/slave/qos_controller.cpp
@@ -0,0 +1,92 @@
+/**
+ * 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 <process/delay.hpp>
+#include <process/dispatch.hpp>
+#include <process/process.hpp>
+
+#include <stout/error.hpp>
+
+#include "slave/qos_controller.hpp"
+
+using namespace process;
+
+using std::string;
+using std::list;
+
+namespace mesos {
+namespace slave {
+
+Try<QoSController*> QoSController::create(const Option<string>& type)
+{
+  // TODO(nnielsen): Support loading QoS Controller from module.
+  if (type.isNone()) {
+    return new internal::slave::NoopQoSController();
+  }
+
+  return Error("Unsupported QoS Controller '" + type.get() + "'");
+}
+
+} // namespace slave {
+} // namespace mesos {
+
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class NoopQoSControllerProcess : public Process<NoopQoSControllerProcess>
+{
+public:
+  virtual ~NoopQoSControllerProcess() {}
+
+  NoopQoSControllerProcess() {}
+};
+
+
+NoopQoSController::~NoopQoSController()
+{
+  if (process.get() != NULL) {
+    terminate(process.get());
+    wait(process.get());
+  }
+}
+
+
+Try<Nothing> NoopQoSController::initialize()
+{
+  if (process.get() != NULL) {
+    return Error("Noop QoS Controller has already been initialized");
+  }
+
+  process.reset(new NoopQoSControllerProcess());
+  spawn(process.get());
+
+  return Nothing();
+}
+
+
+process::Future<list<mesos::slave::QoSCorrection>>
+NoopQoSController::corrections()
+{
+  return Future<list<mesos::slave::QoSCorrection>>();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f6cad2e2/src/slave/qos_controller.hpp
----------------------------------------------------------------------
diff --git a/src/slave/qos_controller.hpp b/src/slave/qos_controller.hpp
new file mode 100644
index 0000000..b377983
--- /dev/null
+++ b/src/slave/qos_controller.hpp
@@ -0,0 +1,55 @@
+/**
+ * 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.
+ */
+
+#ifndef __SLAVE_QOS_CONTROLLER_HPP__
+#define __SLAVE_QOS_CONTROLLER_HPP__
+
+#include <mesos/slave/qos_controller.hpp>
+
+#include <process/owned.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Forward declaration.
+class NoopQoSControllerProcess;
+
+
+// The NOOP QoS Controller is an empty stub, which returns a future
+// which is never satisfied. Thus, the slave will never carry out any
+// corrections.
+class NoopQoSController : public mesos::slave::QoSController
+{
+public:
+  virtual ~NoopQoSController();
+
+  virtual Try<Nothing> initialize();
+
+  virtual process::Future<std::list<mesos::slave::QoSCorrection>> corrections();
+
+protected:
+  process::Owned<NoopQoSControllerProcess> process;
+};
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __SLAVE_QOS_CONTROLLER_HPP__


[4/4] mesos git commit: Added QoS Controller test.

Posted by nn...@apache.org.
Added QoS Controller test.

Added new QoS Controller test which use the TestQoSController to fill
the correction queue and verify the slave receiving corrections.

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


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

Branch: refs/heads/master
Commit: 3e139c61d2ea7084c2e24e226e0bd8a037f60446
Parents: a2d1515
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Thu Jun 4 14:30:31 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Thu Jun 4 16:45:49 2015 -0700

----------------------------------------------------------------------
 src/tests/oversubscription_tests.cpp | 44 +++++++++++++++++++++++++++++++
 1 file changed, 44 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3e139c61/src/tests/oversubscription_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/oversubscription_tests.cpp b/src/tests/oversubscription_tests.cpp
index a49cb39..afd7ff4 100644
--- a/src/tests/oversubscription_tests.cpp
+++ b/src/tests/oversubscription_tests.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include <list>
 #include <string>
 #include <vector>
 
@@ -23,6 +24,8 @@
 
 #include <mesos/resources.hpp>
 
+#include <mesos/slave/qos_controller.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gtest.hpp>
@@ -43,6 +46,7 @@
 #include "slave/slave.hpp"
 
 #include "tests/flags.hpp"
+#include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 
@@ -52,6 +56,9 @@ using mesos::internal::master::Master;
 
 using mesos::internal::slave::Slave;
 
+using mesos::slave::QoSCorrection;
+
+using std::list;
 using std::string;
 using std::vector;
 
@@ -388,6 +395,43 @@ TEST_F(OversubscriptionTest, FixedResourceEstimator)
   Shutdown();
 }
 
+
+// Tests interactions between QoS Controller and slave. The
+// TestQoSController's correction queue is filled and a mocked slave
+// is checked for receiving the given correction.
+TEST_F(OversubscriptionTest, ReceiveQoSCorrection)
+{
+  StandaloneMasterDetector detector;
+  TestContainerizer containerizer;
+
+  MockQoSController controller;
+
+  Queue<list<QoSCorrection>> corrections;
+
+  EXPECT_CALL(controller, corrections())
+    .WillRepeatedly(Invoke(&corrections, &Queue<list<QoSCorrection>>::get));
+
+  MockSlave slave(CreateSlaveFlags(), &detector, &containerizer, &controller);
+
+  Future<list<QoSCorrection>> qosCorrections;
+  EXPECT_CALL(slave, qosCorrections(_))
+    .WillOnce(FutureArg<0>(&qosCorrections));
+
+  spawn(slave);
+
+  list<QoSCorrection> expected = { QoSCorrection() };
+  corrections.put(expected);
+
+  AWAIT_READY(qosCorrections);
+
+  ASSERT_EQ(qosCorrections.get().size(), 1u);
+
+  // TODO(nnielsen): Test for equality of QoSCorrections.
+
+  terminate(slave);
+  wait(slave);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[3/4] mesos git commit: Added wrapper header for mesos/slave/oversubscription.pb.h.

Posted by nn...@apache.org.
Added wrapper header for mesos/slave/oversubscription.pb.h.

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


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

Branch: refs/heads/master
Commit: 6cbd33229f0bd148827bf1f667f4ce03d9884957
Parents: 9e23a3e
Author: Niklas Nielsen <ni...@qni.dk>
Authored: Thu Jun 4 14:29:46 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Thu Jun 4 16:44:24 2015 -0700

----------------------------------------------------------------------
 include/mesos/slave/oversubscription.hpp | 25 +++++++++++++++++++++++++
 src/Makefile.am                          |  5 +++--
 2 files changed, 28 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6cbd3322/include/mesos/slave/oversubscription.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/oversubscription.hpp b/include/mesos/slave/oversubscription.hpp
new file mode 100644
index 0000000..ffefaa0
--- /dev/null
+++ b/include/mesos/slave/oversubscription.hpp
@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+#ifndef __SLAVE_OVERSUBSCRIPTION_PROTO_HPP__
+#define __SLAVE_OVERSUBSCRIPTION_PROTO_HPP__
+
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include <mesos/slave/oversubscription.pb.h>
+
+#endif // __SLAVE_OVERSUBSCRIPTION_PROTO_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/6cbd3322/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 3cf8bd2..d42d1b6 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -498,8 +498,9 @@ slavedir = $(pkgincludedir)/slave
 
 slave_HEADERS =								\
   $(top_srcdir)/include/mesos/slave/isolator.hpp			\
-  $(top_srcdir)/include/mesos/slave/resource_estimator.hpp		\
-  $(top_srcdir)/include/mesos/slave/oversubscription.proto
+  $(top_srcdir)/include/mesos/slave/oversubscription.hpp		\
+  $(top_srcdir)/include/mesos/slave/oversubscription.proto		\
+  $(top_srcdir)/include/mesos/slave/resource_estimator.hpp
 
 nodist_slave_HEADERS = ../include/mesos/slave/oversubscription.pb.h