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 2011/06/05 11:27:28 UTC

svn commit: r1132336 - in /incubator/mesos/trunk/src: examples/Makefile.in examples/long_lived_executor.cpp examples/long_lived_framework.cpp local/main.cpp master/master.cpp master/master.hpp

Author: benh
Date: Sun Jun  5 09:27:27 2011
New Revision: 1132336

URL: http://svn.apache.org/viewvc?rev=1132336&view=rev
Log:
Adding some REST endpoints and a long-lived example for helping the testing process.

Added:
    incubator/mesos/trunk/src/examples/long_lived_executor.cpp
    incubator/mesos/trunk/src/examples/long_lived_framework.cpp
Modified:
    incubator/mesos/trunk/src/examples/Makefile.in
    incubator/mesos/trunk/src/local/main.cpp
    incubator/mesos/trunk/src/master/master.cpp
    incubator/mesos/trunk/src/master/master.hpp

Modified: incubator/mesos/trunk/src/examples/Makefile.in
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/Makefile.in?rev=1132336&r1=1132335&r2=1132336&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/Makefile.in (original)
+++ incubator/mesos/trunk/src/examples/Makefile.in Sun Jun  5 09:27:27 2011
@@ -2,6 +2,8 @@
 
 SHELL = '/bin/sh'
 
+SRCDIR = @srcdir@
+INCLUDEDIR = @top_builddir@/include
 BINDIR = @top_builddir@/bin
 LIBDIR = @top_builddir@/lib
 
@@ -23,25 +25,32 @@ WITH_ZOOKEEPER = @WITH_ZOOKEEPER@
 
 WITH_INCLUDED_ZOOKEEPER = @WITH_INCLUDED_ZOOKEEPER@
 
+BOOST = third_party/boost-1.37.0
+PROTOBUF = third_party/protobuf-2.3.0
+GLOG = third_party/glog-0.3.1
 LIBPROCESS = third_party/libprocess
-
 LIBEV = $(LIBPROCESS)/third_party/libev-3.8
-
-GLOG = third_party/glog-0.3.1
-
 ZOOKEEPER = third_party/zookeeper-3.3.1/src/c
 
 # Ensure that we get better debugging info.
 CFLAGS += -g
 CXXFLAGS += -g
 
-# Add include to CFLAGS and CXXFLAGS.
-CFLAGS += -I@top_srcdir@/include
-CXXFLAGS += -I@top_srcdir@/include
+# Add include and build include to CFLAGS and CXXFLAGS.
+CFLAGS += -I@top_srcdir@/include -I$(INCLUDEDIR)
+CXXFLAGS += -I@top_srcdir@/include -I$(INCLUDEDIR)
 
 # Add boost to CFLAGS and CXXFLAGS.
-CFLAGS += -I@top_srcdir@/third_party/boost-1.37.0
-CXXFLAGS += -I@top_srcdir@/third_party/boost-1.37.0
+CFLAGS += -I@top_srcdir@/$(BOOST)
+CXXFLAGS += -I@top_srcdir@/$(BOOST)
+
+# Add glog to include and lib paths.
+CXXFLAGS += -I@top_srcdir@/$(GLOG)/src -I@top_builddir@/$(GLOG)/src
+LDFLAGS += -L@top_builddir@/$(GLOG)/.libs
+
+# Add protobuf to include and lib paths.
+CXXFLAGS += -I@top_srcdir@/$(PROTOBUF)/src
+LDFLAGS += -L@top_builddir@/$(PROTOBUF)/src/.libs
 
 # Add libprocess to CFLAGS, CXXFLAGS, and LDFLAGS.
 CFLAGS += -I@top_srcdir@/$(LIBPROCESS)
@@ -51,10 +60,6 @@ LDFLAGS += -L@top_builddir@/$(LIBPROCESS
 # Add libev to LDFLAGS.
 LDFLAGS += -L@top_builddir@/$(LIBEV)/.libs
 
-# Add glog to include and lib paths.
-CXXFLAGS += -I@top_srcdir@/$(GLOG)/src -I@top_builddir@/$(GLOG)/src
-LDFLAGS += -L@top_builddir@/$(GLOG)/.libs
-
 # Add included ZooKeeper to include and lib paths if necessary.
 ifeq ($(WITH_INCLUDED_ZOOKEEPER),1)
   CXXFLAGS += -I@top_srcdir@/$(ZOOKEEPER)/include -I@top_srcdir@/$(ZOOKEEPER)/generated
@@ -65,32 +70,21 @@ endif
 CFLAGS += -MMD -MP
 CXXFLAGS += -MMD -MP
 
-# Add build date to CFLAGS, CXXFLAGS
-CFLAGS += -DBUILD_DATE="\"$$(date '+%Y-%m-%d %H:%M:%S')\""
-CXXFLAGS += -DBUILD_DATE="\"$$(date '+%Y-%m-%d %H:%M:%S')\""
-
-# Add build user to CFLAGS, CXXFLAGS
-CFLAGS += -DBUILD_USER="\"$$USER\""
-CXXFLAGS += -DBUILD_USER="\"$$USER\""
-
-# Add libev to LDFLAGS.
-LDFLAGS += -L$(LIBEV)/.libs
-
-# Add glog, libev, libprocess, pthread, and dl to LIBS.
-LIBS += -lglog -lprocess -lev -lpthread -ldl
+# Add protobuf, glog, libev, libprocess, pthread, and dl to LIBS.
+LIBS += -lprotobuf -lglog -lprocess -lev -lpthread -ldl
 
 # Add ZooKeeper if necessary.
 ifeq ($(WITH_ZOOKEEPER),1)
-  LIBS += -lzookeeper_st
+  LIBS += -lzookeeper_mt
 endif
 
-SCHED_EXES = $(BINDIR)/examples/test-framework		\
-             $(BINDIR)/examples/cpp-test-framework	\
+SCHED_EXES = $(BINDIR)/examples/cpp-test-framework	\
+	     $(BINDIR)/examples/long-lived-framework	\
              $(BINDIR)/examples/memhog			\
              $(BINDIR)/examples/scheduled-memhog
 
-EXEC_EXES = $(BINDIR)/examples/test-executor		\
-            $(BINDIR)/examples/cpp-test-executor	\
+EXEC_EXES = $(BINDIR)/examples/cpp-test-executor	\
+            $(BINDIR)/examples/long-lived-executor	\
             $(BINDIR)/examples/memhog-executor
 
 EXAMPLES_EXES = $(SCHED_EXES) $(EXEC_EXES)
@@ -102,6 +96,7 @@ MESOS_EXEC_LIB = $(LIBDIR)/libmesos_exec
 # automagically by configure.
 DIRECTORIES = $(BINDIR)/examples
 
+
 default: all
 
 -include $(patsubst %, %.d, $(EXAMPLES_EXES))
@@ -123,6 +118,12 @@ $(BINDIR)/examples/cpp-test-framework: @
 $(BINDIR)/examples/cpp-test-executor: @srcdir@/cpp_test_executor.cpp $(MESOS_EXEC_LIB)
 	$(CXX) $(CXXFLAGS) -o $@ $< $(MESOS_EXEC_LIB) $(LDFLAGS) $(LIBS)
 
+$(BINDIR)/examples/long-lived-framework: @srcdir@/long_lived_framework.cpp $(MESOS_SCHED_LIB)
+	$(CXX) $(CXXFLAGS) -o $@ $< $(MESOS_SCHED_LIB) $(LDFLAGS) $(LIBS)
+
+$(BINDIR)/examples/long-lived-executor: @srcdir@/long_lived_executor.cpp $(MESOS_EXEC_LIB)
+	$(CXX) $(CXXFLAGS) -o $@ $< $(MESOS_EXEC_LIB) $(LDFLAGS) $(LIBS)
+
 $(BINDIR)/examples/memhog: @srcdir@/memhog.cpp $(MESOS_SCHED_LIB)
 	$(CXX) $(CXXFLAGS) -o $@ $< $(MESOS_SCHED_LIB) $(LDFLAGS) $(LIBS)
 

Added: incubator/mesos/trunk/src/examples/long_lived_executor.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/long_lived_executor.cpp?rev=1132336&view=auto
==============================================================================
--- incubator/mesos/trunk/src/examples/long_lived_executor.cpp (added)
+++ incubator/mesos/trunk/src/examples/long_lived_executor.cpp Sun Jun  5 09:27:27 2011
@@ -0,0 +1,82 @@
+#include <pthread.h>
+
+#include <cstdlib>
+#include <iostream>
+
+#include <tr1/functional>
+
+#include <mesos/executor.hpp>
+
+using namespace mesos;
+using namespace std;
+using namespace std::tr1;
+
+
+void run(ExecutorDriver* driver, const TaskDescription& task)
+{
+  sleep(100);
+  TaskStatus status;
+  *status.mutable_task_id() = task.task_id();
+  *status.mutable_slave_id() = task.slave_id();
+  status.set_state(TASK_FINISHED);
+  driver->sendStatusUpdate(status);
+}
+
+
+void* start(void* arg)
+{
+  function<void(void)>* thunk = (function<void(void)>*) arg;
+  (*thunk)();
+  delete thunk;
+  return NULL;
+}
+
+
+class MyExecutor : public Executor
+{
+public:
+  virtual ~MyExecutor() {}
+
+  virtual void init(ExecutorDriver*, const ExecutorArgs& args)
+  {
+    cout << "Initalized executor on " << args.hostname() << endl;
+  }
+
+  virtual void launchTask(ExecutorDriver* driver, const TaskDescription& task)
+  {
+    cout << "Starting task " << task.task_id().value() << endl;
+
+    function<void(void)>* thunk =
+      new function<void(void)>(bind(&run, driver, task));
+
+    pthread_t pthread;
+    if (pthread_create(&pthread, NULL, &start, thunk) != 0) {
+      TaskStatus status;
+      *status.mutable_task_id() = task.task_id();
+      *status.mutable_slave_id() = task.slave_id();
+      status.set_state(TASK_FAILED);
+      driver->sendStatusUpdate(status);
+    } else {
+      pthread_detach(pthread);
+    }
+  }
+
+  virtual void killTask(ExecutorDriver* driver, const TaskID& taskId) {}
+
+  virtual void frameworkMessage(ExecutorDriver* driver,
+                                const string& data) {}
+
+  virtual void shutdown(ExecutorDriver* driver) {}
+
+  virtual void error(ExecutorDriver* driver, int code,
+                     const std::string& message) {}
+};
+
+
+int main(int argc, char** argv)
+{
+  MyExecutor exec;
+  MesosExecutorDriver driver(&exec);
+  driver.run();
+  return 0;
+}

Added: incubator/mesos/trunk/src/examples/long_lived_framework.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/long_lived_framework.cpp?rev=1132336&view=auto
==============================================================================
--- incubator/mesos/trunk/src/examples/long_lived_framework.cpp (added)
+++ incubator/mesos/trunk/src/examples/long_lived_framework.cpp Sun Jun  5 09:27:27 2011
@@ -0,0 +1,147 @@
+#include <libgen.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <sstream>
+
+#include <boost/lexical_cast.hpp>
+
+#include <mesos/scheduler.hpp>
+
+using namespace mesos;
+using namespace std;
+
+using boost::lexical_cast;
+
+
+const int32_t CPUS_PER_TASK = 1;
+const int32_t MEM_PER_TASK = 32;
+
+
+class MyScheduler : public Scheduler
+{
+public:
+  MyScheduler(const string& _uri)
+    : uri(_uri), tasksLaunched(0) {}
+
+  virtual ~MyScheduler() {}
+
+  virtual string getFrameworkName(SchedulerDriver*)
+  {
+    return "C++ Test Framework";
+  }
+
+  virtual ExecutorInfo getExecutorInfo(SchedulerDriver*)
+  {
+    ExecutorInfo executor;
+    executor.mutable_executor_id()->set_value("default");
+    executor.set_uri(uri);
+    return executor;
+  }
+
+  virtual void registered(SchedulerDriver*, const FrameworkID&)
+  {
+    cout << "Registered!" << endl;
+  }
+
+  virtual void resourceOffer(SchedulerDriver* driver,
+                             const OfferID& offerId,
+                             const vector<SlaveOffer>& offers)
+  {
+    cout << "." << flush;
+    vector<TaskDescription> tasks;
+    vector<SlaveOffer>::const_iterator iterator = offers.begin();
+    for (; iterator != offers.end(); ++iterator) {
+      const SlaveOffer& offer = *iterator;
+      // Lookup resources we care about.
+      // TODO(benh): It would be nice to ultimately have some helper
+      // functions for looking up resources.
+      double cpus = 0;
+      double mem = 0;
+
+      for (int i = 0; i < offer.resources_size(); i++) {
+        const Resource& resource = offer.resources(i);
+        if (resource.name() == "cpus" &&
+            resource.type() == Resource::SCALAR) {
+          cpus = resource.scalar().value();
+        } else if (resource.name() == "mem" &&
+                   resource.type() == Resource::SCALAR) {
+          mem = resource.scalar().value();
+        }
+      }
+
+      // Launch tasks.
+      if (cpus >= CPUS_PER_TASK && mem >= MEM_PER_TASK) {
+        int taskId = tasksLaunched++;
+
+        cout << "Starting task " << taskId << " on "
+             << offer.hostname() << endl;
+
+        TaskDescription task;
+        task.set_name("Task " + lexical_cast<string>(taskId));
+        task.mutable_task_id()->set_value(lexical_cast<string>(taskId));
+        task.mutable_slave_id()->MergeFrom(offer.slave_id());
+
+        Resource* resource;
+
+        resource = task.add_resources();
+        resource->set_name("cpus");
+        resource->set_type(Resource::SCALAR);
+        resource->mutable_scalar()->set_value(CPUS_PER_TASK);
+
+        resource = task.add_resources();
+        resource->set_name("mem");
+        resource->set_type(Resource::SCALAR);
+        resource->mutable_scalar()->set_value(MEM_PER_TASK);
+
+        tasks.push_back(task);
+
+        cpus -= CPUS_PER_TASK;
+        mem -= MEM_PER_TASK;
+      }
+    }
+
+    driver->replyToOffer(offerId, tasks);
+  }
+
+  virtual void offerRescinded(SchedulerDriver* driver,
+                              const OfferID& offerId) {}
+
+  virtual void statusUpdate(SchedulerDriver* driver, const TaskStatus& status)
+  {
+    int taskId = lexical_cast<int>(status.task_id().value());
+    cout << "Task " << taskId << " is in state " << status.state() << endl;
+  }
+
+  virtual void frameworkMessage(SchedulerDriver* driver,
+				const SlaveID& slaveId,
+				const ExecutorID& executorId,
+                                const string& data) {}
+
+  virtual void slaveLost(SchedulerDriver* driver, const SlaveID& sid) {}
+
+  virtual void error(SchedulerDriver* driver, int code,
+                     const string& message) {}
+
+private:
+  string uri;
+  int tasksLaunched;
+};
+
+
+int main(int argc, char** argv)
+{
+  if (argc != 2) {
+    cerr << "Usage: " << argv[0] << " <masterPid>" << endl;
+    return -1;
+  }
+  // Find this executable's directory to locate executor
+  char buf[4096];
+  realpath(dirname(argv[0]), buf);
+  string executor = string(buf) + "/long-lived-executor";
+  // Run a Mesos scheduler
+  MyScheduler sched(executor);
+  MesosSchedulerDriver driver(&sched, argv[1]);
+  driver.run();
+  return 0;
+}

Modified: incubator/mesos/trunk/src/local/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/main.cpp?rev=1132336&r1=1132335&r2=1132336&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/main.cpp (original)
+++ incubator/mesos/trunk/src/local/main.cpp Sun Jun  5 09:27:27 2011
@@ -40,7 +40,6 @@ void usage(const char* programName, cons
 int main(int argc, char **argv)
 {
   Configurator configurator;
-  Logging::registerOptions(&configurator);
   local::registerOptions(&configurator);
   configurator.addOption<int>("port", 'p', "Port to listen on", 5050);
   configurator.addOption<string>("ip", "IP address to listen on");

Modified: incubator/mesos/trunk/src/master/master.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.cpp?rev=1132336&r1=1132335&r2=1132336&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.cpp (original)
+++ incubator/mesos/trunk/src/master/master.cpp Sun Jun  5 09:27:27 2011
@@ -525,8 +525,12 @@ void Master::initialize()
   install(process::EXITED, &Master::exited);
 
   // Install HTTP request handlers.
-  Process<Master>::install("vars", &Master::vars);
-  Process<Master>::install("stats", &Master::stats);
+  Process<Master>::install("info.json", &Master::http_info_json);
+  Process<Master>::install("frameworks.json", &Master::http_frameworks_json);
+  Process<Master>::install("slaves.json", &Master::http_slaves_json);
+  Process<Master>::install("tasks.json", &Master::http_tasks_json);
+  Process<Master>::install("stats.json", &Master::http_stats_json);
+  Process<Master>::install("vars", &Master::http_vars);
 }
 
 
@@ -1202,34 +1206,146 @@ void Master::exited()
 }
 
 
-Promise<HttpResponse> Master::vars(const HttpRequest& request)
+Promise<HttpResponse> Master::http_info_json(const HttpRequest& request)
 {
-  LOG(INFO) << "Request for 'vars'";
+  LOG(INFO) << "HTTP request for '/master/info.json'";
 
   ostringstream out;
 
   out <<
-    "build_date " << build::DATE << "\n" <<
-    "build_user " << build::USER << "\n" <<
-    "build_flags " << build::FLAGS << "\n" <<
-    "frameworks_count " << frameworks.size() << "\n";
+    "{" <<
+    "\"built_date\":\"" << build::DATE << "\"," <<
+    "\"build_user\":\"" << build::USER << "\"," <<
+    "\"pid\":\"" << self() << "\"" <<
+    "}";
 
-  // Also add the configuration values.
-  foreachpair (const string& key, const string& value, conf.getMap()) {
-    out << key << " " << value << "\n";
+  HttpOKResponse response;
+  response.headers["Content-Type"] = "text/x-json;charset=UTF-8";
+  response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
+  response.body = out.str().data();
+  return response;
+}
+
+
+Promise<HttpResponse> Master::http_frameworks_json(const HttpRequest& request)
+{
+  LOG(INFO) << "HTTP request for '/master/frameworks.json'";
+
+  ostringstream out;
+
+  out << "[";
+
+  foreachpair (_, Framework* framework, frameworks) {
+    out <<
+      "{" <<
+      "\"id\":\"" << framework->frameworkId << "\"," <<
+      "\"name\":\"" << framework->info.name() << "\"," <<
+      "\"user\":\"" << framework->info.user() << "\""
+      "},";
+  }
+
+  // Backup the put pointer to overwrite the last comma (hack).
+  if (frameworks.size() > 0) {
+    long pos = out.tellp();
+    out.seekp(pos - 1);
   }
 
+  out << "]";
+
   HttpOKResponse response;
-  response.headers["Content-Type"] = "text/plain";
+  response.headers["Content-Type"] = "text/x-json;charset=UTF-8";
+  response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
+  response.body = out.str().data();
+  return response;
+}
+
+
+Promise<HttpResponse> Master::http_slaves_json(const HttpRequest& request)
+{
+  LOG(INFO) << "HTTP request for '/master/slaves.json'";
+
+  ostringstream out;
+
+  out << "[";
+
+  foreachpair (_, Slave* slave, slaves) {
+    // TODO(benh): Send all of the resources (as JSON).
+    Resources resources(slave->info.resources());
+    Resource::Scalar cpus = resources.getScalar("cpus", Resource::Scalar());
+    Resource::Scalar mem = resources.getScalar("mem", Resource::Scalar());
+    out <<
+      "{" <<
+      "\"id\":\"" << slave->slaveId << "\"," <<
+      "\"hostname\":\"" << slave->info.hostname() << "\"," <<
+      "\"cpus\":" << cpus.value() << "," <<
+      "\"mem\":" << mem.value() <<
+      "},";
+  }
+
+  // Backup the put pointer to overwrite the last comma (hack).
+  if (slaves.size() > 0) {
+    long pos = out.tellp();
+    out.seekp(pos - 1);
+  }
+
+  out << "]";
+
+  HttpOKResponse response;
+  response.headers["Content-Type"] = "text/x-json;charset=UTF-8";
   response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
   response.body = out.str().data();
   return response;
 }
 
 
-Promise<HttpResponse> Master::stats(const HttpRequest& request)
+Promise<HttpResponse> Master::http_tasks_json(const HttpRequest& request)
 {
-  LOG(INFO) << "Request for 'stats'";
+  LOG(INFO) << "HTTP request for '/master/tasks.json'";
+
+  ostringstream out;
+
+  out << "[";
+
+  foreachpair (_, Framework* framework, frameworks) {
+    foreachpair (_, Task* task, framework->tasks) {
+      // TODO(benh): Send all of the resources (as JSON).
+      Resources resources(task->resources());
+      Resource::Scalar cpus = resources.getScalar("cpus", Resource::Scalar());
+      Resource::Scalar mem = resources.getScalar("mem", Resource::Scalar());
+      const string& state =
+        TaskState_descriptor()->FindValueByNumber(task->state())->name();
+      out <<
+        "{" <<
+        "\"task_id\":\"" << task->task_id() << "\"," <<
+        "\"framework_id\":\"" << task->framework_id() << "\"," <<
+        "\"slave_id\":\"" << task->slave_id() << "\"," <<
+        "\"name\":\"" << task->name() << "\"," <<
+        "\"state\":\"" << state << "\"," <<
+        "\"cpus\":" << cpus.value() << "," <<
+        "\"mem\":" << mem.value() <<
+        "},";
+    }
+  }
+
+  // Backup the put pointer to overwrite the last comma (hack).
+  if (frameworks.size() > 0) {
+    long pos = out.tellp();
+    out.seekp(pos - 1);
+  }
+
+  out << "]";
+
+  HttpOKResponse response;
+  response.headers["Content-Type"] = "text/x-json;charset=UTF-8";
+  response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
+  response.body = out.str().data();
+  return response;
+}
+
+
+Promise<HttpResponse> Master::http_stats_json(const HttpRequest& request)
+{
+  LOG(INFO) << "Http request for '/master/stats.json'";
 
   ostringstream out;
 
@@ -1258,6 +1374,31 @@ Promise<HttpResponse> Master::stats(cons
 }
 
 
+Promise<HttpResponse> Master::http_vars(const HttpRequest& request)
+{
+  LOG(INFO) << "HTTP request for '/master/vars'";
+
+  ostringstream out;
+
+  out <<
+    "build_date " << build::DATE << "\n" <<
+    "build_user " << build::USER << "\n" <<
+    "build_flags " << build::FLAGS << "\n" <<
+    "frameworks_count " << frameworks.size() << "\n";
+
+  // Also add the configuration values.
+  foreachpair (const string& key, const string& value, conf.getMap()) {
+    out << key << " " << value << "\n";
+  }
+
+  HttpOKResponse response;
+  response.headers["Content-Type"] = "text/plain";
+  response.headers["Content-Length"] = lexical_cast<string>(out.str().size());
+  response.body = out.str().data();
+  return response;
+}
+
+
 OfferID Master::makeOffer(Framework* framework,
                           const vector<SlaveResources>& resources)
 {

Modified: incubator/mesos/trunk/src/master/master.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.hpp?rev=1132336&r1=1132335&r2=1132336&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.hpp (original)
+++ incubator/mesos/trunk/src/master/master.hpp Sun Jun  5 09:27:27 2011
@@ -151,9 +151,6 @@ public:
   void frameworkExpired(const FrameworkID& frameworkId);
   void exited();
 
-  process::Promise<process::HttpResponse> vars(const process::HttpRequest& request);
-  process::Promise<process::HttpResponse> stats(const process::HttpRequest& request);
-
   Framework* lookupFramework(const FrameworkID& frameworkId);
   Slave* lookupSlave(const SlaveID& slaveId);
   SlotOffer* lookupSlotOffer(const OfferID& offerId);
@@ -214,6 +211,14 @@ protected:
   const Configuration& getConfiguration();
 
 private:
+  // TODO(benh): Better naming and name scope for these http handlers.
+  process::Promise<process::HttpResponse> http_info_json(const process::HttpRequest& request);
+  process::Promise<process::HttpResponse> http_frameworks_json(const process::HttpRequest& request);
+  process::Promise<process::HttpResponse> http_slaves_json(const process::HttpRequest& request);
+  process::Promise<process::HttpResponse> http_tasks_json(const process::HttpRequest& request);
+  process::Promise<process::HttpResponse> http_stats_json(const process::HttpRequest& request);
+  process::Promise<process::HttpResponse> http_vars(const process::HttpRequest& request);
+
   const Configuration conf;
 
   SlavesManager* slavesManager;