You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2013/07/02 00:11:58 UTC

[2/4] git commit: Moved master/slave HTTP route handlers into inner class.

Moved master/slave HTTP route handlers into inner class.

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


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

Branch: refs/heads/master
Commit: 663ebd02010a9b755215f6565883fe9ea1136645
Parents: 5c784f4
Author: Benjamin Hindman <be...@twitter.com>
Authored: Fri Jun 7 18:17:54 2013 -0700
Committer: Benjamin Hindman <be...@twitter.com>
Committed: Mon Jul 1 18:10:25 2013 -0400

----------------------------------------------------------------------
 src/Makefile.am       |  4 +--
 src/master/http.cpp   | 24 +++--------------
 src/master/http.hpp   | 65 ----------------------------------------------
 src/master/master.cpp | 12 +++++----
 src/master/master.hpp | 47 +++++++++++++++++++--------------
 src/slave/http.cpp    | 20 +++-----------
 src/slave/http.hpp    | 61 -------------------------------------------
 src/slave/slave.cpp   |  9 ++++---
 src/slave/slave.hpp   | 33 +++++++++++++----------
 9 files changed, 68 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 9337435..5d6f309 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -223,10 +223,10 @@ libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	logging/flags.hpp logging/logging.hpp				\
 	master/allocator.hpp						\
 	master/constants.hpp master/drf_sorter.hpp master/flags.hpp	\
-	master/hierarchical_allocator_process.hpp master/http.hpp	\
+	master/hierarchical_allocator_process.hpp			\
 	master/master.hpp master/sorter.hpp				\
 	messages/messages.hpp slave/constants.hpp			\
-	slave/flags.hpp slave/gc.hpp slave/monitor.hpp slave/http.hpp	\
+	slave/flags.hpp slave/gc.hpp slave/monitor.hpp			\
 	slave/isolator.hpp						\
 	slave/cgroups_isolator.hpp					\
 	slave/paths.hpp slave/state.hpp					\

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 47471d6..4317bfc 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -39,7 +39,6 @@
 
 #include "logging/logging.hpp"
 
-#include "master/http.hpp"
 #include "master/master.hpp"
 
 namespace mesos {
@@ -63,7 +62,6 @@ using std::vector;
 // TODO(bmahler): Kill these in favor of automatic Proto->JSON Conversion (when
 // it becomes available).
 
-
 // Returns a JSON object modeled on a Resources.
 JSON::Object model(const Resources& resources)
 {
@@ -211,11 +209,7 @@ JSON::Object model(const Slave& slave)
 }
 
 
-namespace http {
-
-Future<Response> vars(
-    const Master& master,
-    const Request& request)
+Future<Response> Master::Http::vars(const Request& request)
 {
   VLOG(1) << "HTTP request for '" << request.path << "'";
 
@@ -236,9 +230,7 @@ Future<Response> vars(
   return OK(out.str(), request.query.get("jsonp"));
 }
 
-Future<Response> redirect(
-    const Master& master,
-    const Request& request)
+Future<Response> Master::Http::redirect(const Request& request)
 {
   VLOG(1) << "HTTP request for '" << request.path << "'";
 
@@ -255,11 +247,7 @@ Future<Response> redirect(
 }
 
 
-namespace json {
-
-Future<Response> stats(
-    const Master& master,
-    const Request& request)
+Future<Response> Master::Http::stats(const Request& request)
 {
   VLOG(1) << "HTTP request for '" << request.path << "'";
 
@@ -307,9 +295,7 @@ Future<Response> stats(
 }
 
 
-Future<Response> state(
-    const Master& master,
-    const Request& request)
+Future<Response> Master::Http::state(const Request& request)
 {
   VLOG(1) << "HTTP request for '" << request.path << "'";
 
@@ -378,8 +364,6 @@ Future<Response> state(
   return OK(object, request.query.get("jsonp"));
 }
 
-} // namespace json {
-} // namespace http {
 } // namespace master {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/master/http.hpp
----------------------------------------------------------------------
diff --git a/src/master/http.hpp b/src/master/http.hpp
deleted file mode 100644
index 9eb0a31..0000000
--- a/src/master/http.hpp
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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 __MASTER_HTTP_HPP__
-#define __MASTER_HTTP_HPP__
-
-#include <process/future.hpp>
-#include <process/http.hpp>
-
-namespace mesos {
-namespace internal {
-namespace master {
-
-// Forward declaration (necessary to break circular dependency).
-class Master;
-
-namespace http {
-
-// Returns current vars in "key value\n" format (keys do not contain
-// spaces, values may contain spaces but are ended by a newline).
-process::Future<process::http::Response> vars(
-    const Master& master,
-    const process::http::Request& request);
-
-// Redirects immediately to the current leader. If there's no leader, this
-// redirects to the master.
-process::Future<process::http::Response> redirect(
-    const Master& master,
-    const process::http::Request& request);
-
-namespace json {
-
-// Returns current statistics of the master.
-process::Future<process::http::Response> stats(
-    const Master& master,
-    const process::http::Request& request);
-
-
-// Returns current state of the cluster that the master knows about.
-process::Future<process::http::Response> state(
-    const Master& master,
-    const process::http::Request& request);
-
-} // namespace json {
-} // namespace http {
-} // namespace master {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MASTER_HTTP_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 695fb93..cf95101 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -188,6 +188,7 @@ private:
 
 Master::Master(Allocator* _allocator, Files* _files)
   : ProcessBase("master"),
+    http(*this),
     flags(),
     allocator(_allocator),
     files(_files),
@@ -196,6 +197,7 @@ Master::Master(Allocator* _allocator, Files* _files)
 
 Master::Master(Allocator* _allocator, Files* _files, const Flags& _flags)
   : ProcessBase("master"),
+    http(*this),
     flags(_flags),
     allocator(_allocator),
     files(_files),
@@ -354,11 +356,11 @@ void Master::initialize()
       &ExitedExecutorMessage::executor_id,
       &ExitedExecutorMessage::status);
 
-  // Setup HTTP request handlers.
-  route("/redirect", bind(&http::redirect, cref(*this), params::_1));
-  route("/vars", bind(&http::vars, cref(*this), params::_1));
-  route("/stats.json", bind(&http::json::stats, cref(*this), params::_1));
-  route("/state.json", bind(&http::json::state, cref(*this), params::_1));
+  // Setup HTTP routes.
+  route("/redirect", bind(&Http::redirect, http, params::_1));
+  route("/vars", bind(&Http::vars, http, params::_1));
+  route("/stats.json", bind(&Http::stats, http, params::_1));
+  route("/state.json", bind(&Http::state, http, params::_1));
 
   // Provide HTTP assets from a "webui" directory. This is either
   // specified via flags (which is necessary for running out of the

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 86c5232..edbc3f8 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -45,7 +45,6 @@
 
 #include "master/constants.hpp"
 #include "master/flags.hpp"
-#include "master/http.hpp"
 
 #include "messages/messages.hpp"
 
@@ -191,30 +190,38 @@ protected:
   SlaveID newSlaveId();
 
 private:
-  Master(const Master&);              // No copying.
-  Master& operator = (const Master&); // No assigning.
+  // Inner class used to namespace HTTP route handlers (see
+  // master/http.cpp for implementations).
+  class Http
+  {
+  public:
+    Http(const Master& _master) : master(_master) {}
 
-  friend struct SlaveRegistrar;
-  friend struct SlaveReregistrar;
+    // /master/vars
+    process::Future<process::http::Response> vars(
+        const process::http::Request& request);
+
+    // /master/redirect
+    process::Future<process::http::Response> redirect(
+        const process::http::Request& request);
 
-  // HTTP handlers, friends of the master in order to access state,
-  // they get invoked from within the master so there is no need to
-  // use synchronization mechanisms to protect state.
-  friend Future<process::http::Response> http::vars(
-      const Master& master,
-      const process::http::Request& request);
+    // /master/stats.json
+    process::Future<process::http::Response> stats(
+        const process::http::Request& request);
 
-  friend Future<process::http::Response> http::redirect(
-      const Master& master,
-      const process::http::Request& request);
+    // /master/state.json
+    process::Future<process::http::Response> state(
+        const process::http::Request& request);
 
-  friend Future<process::http::Response> http::json::stats(
-      const Master& master,
-      const process::http::Request& request);
+  private:
+    const Master& master;
+  } http;
 
-  friend Future<process::http::Response> http::json::state(
-      const Master& master,
-      const process::http::Request& request);
+  Master(const Master&);              // No copying.
+  Master& operator = (const Master&); // No assigning.
+
+  friend struct SlaveRegistrar;
+  friend struct SlaveReregistrar;
 
   const Flags flags;
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index e362c30..582b3a2 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -52,11 +52,9 @@ using std::map;
 using std::string;
 using std::vector;
 
-
 // TODO(bmahler): Kill these in favor of automatic Proto->JSON Conversion (when
 // in becomes available).
 
-
 // Returns a JSON object modeled on a Resources.
 JSON::Object model(const Resources& resources)
 {
@@ -247,11 +245,7 @@ JSON::Object model(const Framework& framework)
 }
 
 
-namespace http {
-
-Future<Response> vars(
-    const Slave& slave,
-    const Request& request)
+Future<Response> Slave::Http::vars(const Request& request)
 {
   LOG(INFO) << "HTTP request for '" << request.path << "'";
 
@@ -273,11 +267,7 @@ Future<Response> vars(
 }
 
 
-namespace json {
-
-Future<Response> stats(
-    const Slave& slave,
-    const Request& request)
+Future<Response> Slave::Http::stats(const Request& request)
 {
   LOG(INFO) << "HTTP request for '" << request.path << "'";
 
@@ -297,9 +287,7 @@ Future<Response> stats(
 }
 
 
-Future<Response> state(
-    const Slave& slave,
-    const Request& request)
+Future<Response> Slave::Http::state(const Request& request)
 {
   LOG(INFO) << "HTTP request for '" << request.path << "'";
 
@@ -345,8 +333,6 @@ Future<Response> state(
   return OK(object, request.query.get("jsonp"));
 }
 
-} // namespace json {
-} // namespace http {
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/slave/http.hpp
----------------------------------------------------------------------
diff --git a/src/slave/http.hpp b/src/slave/http.hpp
deleted file mode 100644
index 9801470..0000000
--- a/src/slave/http.hpp
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * 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_HTTP_HPP__
-#define __SLAVE_HTTP_HPP__
-
-#include <process/future.hpp>
-#include <process/http.hpp>
-
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-// Forward declaration (necessary to break circular dependency).
-class Slave;
-
-namespace http {
-
-// Returns current vars in "key value\n" format (keys do not contain
-// spaces, values may contain spaces but are ended by a newline).
-process::Future<process::http::Response> vars(
-    const Slave& slave,
-    const process::http::Request& request);
-
-
-namespace json {
-
-// Returns current statistics of the slave.
-process::Future<process::http::Response> stats(
-    const Slave& slave,
-    const process::http::Request& request);
-
-
-// Returns current state of the cluster that the slave knows about.
-process::Future<process::http::Response> state(
-    const Slave& slave,
-    const process::http::Request& request);
-
-} // namespace json {
-} // namespace http {
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __SLAVE_HTTP_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 1752689..7271097 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -82,6 +82,7 @@ Slave::Slave(const slave::Flags& _flags,
              Files* _files)
   : ProcessBase(ID::generate("slave")),
     state(RECOVERING),
+    http(*this),
     flags(_flags),
     local(_local),
     completedFrameworks(MAX_COMPLETED_FRAMEWORKS),
@@ -352,10 +353,10 @@ void Slave::initialize()
   // Install the ping message handler.
   install("PING", &Slave::ping);
 
-  // Setup some HTTP routes.
-  route("/vars", bind(&http::vars, cref(*this), params::_1));
-  route("/stats.json", bind(&http::json::stats, cref(*this), params::_1));
-  route("/state.json", bind(&http::json::state, cref(*this), params::_1));
+  // Setup HTTP routes.
+  route("/vars", bind(&Http::vars, http, params::_1));
+  route("/stats.json", bind(&Http::stats, http, params::_1));
+  route("/state.json", bind(&Http::state, http, params::_1));
 
   if (flags.log_dir.isSome()) {
     Try<string> log = logging::getLogFile(google::INFO);

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/663ebd02/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 1fca208..c679447 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -43,7 +43,6 @@
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
 #include "slave/gc.hpp"
-#include "slave/http.hpp"
 #include "slave/isolator.hpp"
 #include "slave/monitor.hpp"
 #include "slave/paths.hpp"
@@ -252,27 +251,35 @@ protected:
   void remove(Framework* framework);
 
 private:
-  Slave(const Slave&);              // No copying.
-  Slave& operator = (const Slave&); // No assigning.
-
-  // HTTP handlers, friends of the slave in order to access state,
-  // they get invoked from within the slave so there is no need to
-  // use synchronization mechanisms to protect state.
-  friend Future<process::http::Response> http::vars(
-      const Slave& slave,
+  // Inner class used to namespace HTTP route handlers (see
+  // slave/http.cpp for implementations).
+  class Http
+  {
+  public:
+    Http(const Slave& _slave) : slave(_slave) {}
+
+    // /slave/vars
+    process::Future<process::http::Response> vars(
       const process::http::Request& request);
 
-  friend Future<process::http::Response> http::json::stats(
-      const Slave& slave,
+    // /slave/stats.json
+    process::Future<process::http::Response> stats(
       const process::http::Request& request);
 
-  friend Future<process::http::Response> http::json::state(
-      const Slave& slave,
+    // /slave/state.json
+    process::Future<process::http::Response> state(
       const process::http::Request& request);
 
+  private:
+    const Slave& slave;
+  } http;
+
   friend class Framework;
   friend class Executor;
 
+  Slave(const Slave&);              // No copying.
+  Slave& operator = (const Slave&); // No assigning.
+
   const Flags flags;
 
   bool local;