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/02/20 21:11:33 UTC

git commit: Introduced a Repairer class to master.

Repository: mesos
Updated Branches:
  refs/heads/master a5d333779 -> 51056f4e5


Introduced a Repairer class to master.

Add a Repairer class to master.  Currently it is just a stub, but
it will soon get wired up to the new observe end-point in master and
recieve slave state updates.  From there, it will be responsible for
persisting the state and ultimately performing repairs on unhealthy
slaves.

The Repairer has virtual methods and destructor so that it can
easily be mocked for testing.

Updated the existing master constructor to take a Repairer* and
local/local.cpp, master/main.cpp, & tests/cluster.hpp to new up a
Repairer, pass it to master's constructor and delete the Repairer when
it's finished.

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


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

Branch: refs/heads/master
Commit: 51056f4e576a964b458b2aaa3bb54f3ef39bed2f
Parents: a5d3337
Author: Charlie Carson <ch...@gmail.com>
Authored: Thu Feb 20 11:53:09 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Thu Feb 20 11:53:09 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am         |  1 +
 src/local/local.cpp     | 17 ++++++++++++++--
 src/master/main.cpp     | 14 ++++++++++++--
 src/master/master.cpp   |  2 ++
 src/master/master.hpp   |  3 +++
 src/master/repairer.cpp | 46 ++++++++++++++++++++++++++++++++++++++++++++
 src/master/repairer.hpp | 44 ++++++++++++++++++++++++++++++++++++++++++
 src/tests/cluster.hpp   |  7 +++++++
 8 files changed, 130 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index e02d416..f1ceab3 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -172,6 +172,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	master/registry.hpp						\
 	master/registry.proto                                           \
 	master/registrar.cpp						\
+	master/repairer.cpp						\
 	slave/constants.cpp						\
 	slave/gc.cpp							\
 	slave/monitor.cpp						\

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 5112391..e31c19c 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -39,6 +39,7 @@
 #include "master/hierarchical_allocator_process.hpp"
 #include "master/master.hpp"
 #include "master/registrar.hpp"
+#include "master/repairer.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/slave.hpp"
@@ -56,6 +57,7 @@ using mesos::internal::master::allocator::HierarchicalDRFAllocatorProcess;
 
 using mesos::internal::master::Master;
 using mesos::internal::master::Registrar;
+using mesos::internal::master::Repairer;
 
 using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Slave;
@@ -78,6 +80,7 @@ static AllocatorProcess* allocatorProcess = NULL;
 static state::Storage* storage = NULL;
 static state::protobuf::State* state = NULL;
 static Registrar* registrar = NULL;
+static Repairer* repairer = NULL;
 static Master* master = NULL;
 static map<Containerizer*, Slave*> slaves;
 static StandaloneMasterDetector* detector = NULL;
@@ -127,12 +130,19 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
 
     state = new state::protobuf::State(storage);
     registrar = new Registrar(state);
+    repairer = new Repairer();
 
     contender = new StandaloneMasterContender();
     detector = new StandaloneMasterDetector();
     master =
-      new Master(_allocator, registrar, files, contender, detector, flags);
-
+      new Master(
+        _allocator,
+        registrar,
+        repairer,
+        files,
+        contender,
+        detector,
+        flags);
     detector->appoint(master->info());
   }
 
@@ -204,6 +214,9 @@ void shutdown()
     delete registrar;
     registrar = NULL;
 
+    delete repairer;
+    repairer = NULL;
+
     delete state;
     state = NULL;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index ab4846e..723d534 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -41,6 +41,7 @@
 #include "master/hierarchical_allocator_process.hpp"
 #include "master/master.hpp"
 #include "master/registrar.hpp"
+#include "master/repairer.hpp"
 
 #include "state/leveldb.hpp"
 #include "state/protobuf.hpp"
@@ -157,6 +158,7 @@ int main(int argc, char** argv)
 
   state::protobuf::State* state = new state::protobuf::State(storage);
   Registrar* registrar = new Registrar(state);
+  Repairer* repairer = new Repairer();
 
   Files files;
 
@@ -177,8 +179,15 @@ int main(int argc, char** argv)
 
   LOG(INFO) << "Starting Mesos master";
 
-  Master* master = new Master(
-      allocator, registrar, &files, contender, detector, flags);
+  Master* master =
+    new Master(
+      allocator,
+      registrar,
+      repairer,
+      &files,
+      contender,
+      detector,
+      flags);
 
   if (zk == "") {
     // It means we are using the standalone detector so we need to
@@ -194,6 +203,7 @@ int main(int argc, char** argv)
   delete allocatorProcess;
 
   delete registrar;
+  delete repairer;
   delete state;
   delete storage;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index cb46869..de40884 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -190,6 +190,7 @@ private:
 Master::Master(
     Allocator* _allocator,
     Registrar* _registrar,
+    Repairer* _repairer,
     Files* _files,
     MasterContender* _contender,
     MasterDetector* _detector,
@@ -199,6 +200,7 @@ Master::Master(
     flags(_flags),
     allocator(_allocator),
     registrar(_registrar),
+    repairer(_repairer),
     files(_files),
     contender(_contender),
     detector(_detector),

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 9d1b56c..768dc3d 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -72,6 +72,7 @@ class Allocator;
 
 }
 
+class Repairer;
 class SlaveObserver;
 class WhitelistWatcher;
 
@@ -86,6 +87,7 @@ class Master : public ProtobufProcess<Master>
 public:
   Master(allocator::Allocator* allocator,
          Registrar* registrar,
+         Repairer* repairer,
          Files* files,
          MasterContender* contender,
          MasterDetector* detector,
@@ -333,6 +335,7 @@ private:
   allocator::Allocator* allocator;
   WhitelistWatcher* whitelistWatcher;
   Registrar* registrar;
+  Repairer* repairer;
   Files* files;
 
   MasterContender* contender;

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/master/repairer.cpp
----------------------------------------------------------------------
diff --git a/src/master/repairer.cpp b/src/master/repairer.cpp
new file mode 100644
index 0000000..151b4ed
--- /dev/null
+++ b/src/master/repairer.cpp
@@ -0,0 +1,46 @@
+/**
+ * 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 "logging/logging.hpp"
+
+#include "master/repairer.hpp"
+
+namespace mesos {
+namespace internal {
+namespace master {
+
+using std::string;
+
+void Repairer::observe(
+    const string& host,
+    const string& monitor,
+    bool isHealthy)
+{
+  LOG(INFO) << "Repairer::observed " << isHealthy << " for monitor '"
+    << monitor << "' on host '" << host << "'";
+}
+
+
+Repairer::~Repairer()
+{
+}
+
+
+} // namespace mesos {
+} // namespace internal {
+} // namespace master {

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/master/repairer.hpp
----------------------------------------------------------------------
diff --git a/src/master/repairer.hpp b/src/master/repairer.hpp
new file mode 100644
index 0000000..d5c6b84
--- /dev/null
+++ b/src/master/repairer.hpp
@@ -0,0 +1,44 @@
+/**
+ * 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_MASTER_REPAIRER_HPP__
+#define __MESOS_MASTER_REPAIRER_HPP__
+
+#include <mesos/mesos.hpp>
+
+namespace mesos {
+namespace internal {
+namespace master {
+
+// An abstraction for recieve notification of unhealty nodes
+// and automating repairs to them.
+class Repairer
+{
+public:
+  virtual void observe(
+      const std::string& host,
+      const std::string& metric,
+      bool isHealthy);
+  virtual ~Repairer();
+};
+
+} // namespace master {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_MASTER_REPAIRER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/51056f4e/src/tests/cluster.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.hpp b/src/tests/cluster.hpp
index 5100959..d1bf680 100644
--- a/src/tests/cluster.hpp
+++ b/src/tests/cluster.hpp
@@ -42,6 +42,7 @@
 #include "master/flags.hpp"
 #include "master/master.hpp"
 #include "master/registrar.hpp"
+#include "master/repairer.hpp"
 
 #include "slave/flags.hpp"
 #include "slave/containerizer/containerizer.hpp"
@@ -115,6 +116,7 @@ public:
       state::Storage* storage;
       state::protobuf::State* state;
       master::Registrar* registrar;
+      master::Repairer* repairer;
       MasterContender* contender;
       MasterDetector* detector;
     };
@@ -261,6 +263,7 @@ inline Try<process::PID<master::Master> > Cluster::Masters::start(
 
   master.state = new state::protobuf::State(master.storage);
   master.registrar = new master::Registrar(master.state);
+  master.repairer = new master::Repairer();
 
   if (url.isSome()) {
     master.contender = new ZooKeeperMasterContender(url.get());
@@ -273,6 +276,7 @@ inline Try<process::PID<master::Master> > Cluster::Masters::start(
   master.master = new master::Master(
       master.allocator,
       master.registrar,
+      master.repairer,
       &cluster->files,
       master.contender,
       master.detector,
@@ -321,6 +325,7 @@ inline Try<process::PID<master::Master> > Cluster::Masters::start(
 
   master.state = new state::protobuf::State(master.storage);
   master.registrar = new master::Registrar(master.state);
+  master.repairer = new master::Repairer();
 
   if (url.isSome()) {
     master.contender = new ZooKeeperMasterContender(url.get());
@@ -333,6 +338,7 @@ inline Try<process::PID<master::Master> > Cluster::Masters::start(
   master.master = new master::Master(
       master.allocator,
       master.registrar,
+      master.repairer,
       &cluster->files,
       master.contender,
       master.detector,
@@ -369,6 +375,7 @@ inline Try<Nothing> Cluster::Masters::stop(
   delete master.allocatorProcess; // May be NULL.
 
   delete master.registrar;
+  delete master.repairer;
   delete master.state;
   delete master.storage;