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:59 UTC

[3/4] git commit: Added help pages to libprocess.

Added help pages to libprocess.

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


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

Branch: refs/heads/master
Commit: 0c38dabd7fa778d75f108b5dc824b572dcc5e88d
Parents: 663ebd0
Author: Benjamin Hindman <be...@twitter.com>
Authored: Thu Jun 6 23:23:21 2013 -0700
Committer: Benjamin Hindman <be...@twitter.com>
Committed: Mon Jul 1 18:10:27 2013 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/Makefile.am                 |   1 +
 3rdparty/libprocess/include/process/help.hpp    | 297 +++++++++++++++++++
 3rdparty/libprocess/include/process/logging.hpp |   4 +-
 3rdparty/libprocess/include/process/process.hpp |  14 +-
 .../libprocess/include/process/profiler.hpp     |   7 +-
 3rdparty/libprocess/src/process.cpp             |  72 +++++
 3rdparty/libprocess/src/statistics.cpp          |  34 ++-
 3rdparty/libprocess/src/tests/http_tests.cpp    |   5 +-
 8 files changed, 418 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/Makefile.am b/3rdparty/libprocess/Makefile.am
index 2819734..a13ea70 100644
--- a/3rdparty/libprocess/Makefile.am
+++ b/3rdparty/libprocess/Makefile.am
@@ -72,6 +72,7 @@ libprocess_la_SOURCES +=					\
   $(top_srcdir)/include/process/gc.hpp				\
   $(top_srcdir)/include/process/gmock.hpp			\
   $(top_srcdir)/include/process/gtest.hpp			\
+  $(top_srcdir)/include/process/help.hpp			\
   $(top_srcdir)/include/process/http.hpp			\
   $(top_srcdir)/include/process/id.hpp				\
   $(top_srcdir)/include/process/io.hpp				\

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/include/process/help.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/help.hpp b/3rdparty/libprocess/include/process/help.hpp
new file mode 100644
index 0000000..65b0590
--- /dev/null
+++ b/3rdparty/libprocess/include/process/help.hpp
@@ -0,0 +1,297 @@
+#ifndef __PROCESS_HELP_HPP__
+#define __PROCESS_HELP_HPP__
+
+#include <map>
+#include <string>
+#include <vector>
+
+#include <process/future.hpp>
+#include <process/http.hpp>
+#include <process/process.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/json.hpp>
+#include <stout/option.hpp>
+#include <stout/preprocessor.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+
+namespace process {
+
+// Constructs a Markdown based help "page" for a route with the
+// following template:
+//
+//     ### TL;DR; ###
+//     tldr
+//
+//     ### USAGE ###
+//     usage
+//
+//     ### DESCRIPTION ###
+//     description
+//
+//     references
+//
+// See the 'TLDR', 'USAGE', 'DESCRIPTION', and 'REFERENCES' helpers
+// below to more easily construct your help pages.
+inline std::string HELP(
+    std::string tldr,
+    std::string usage,
+    std::string description,
+    const Option<std::string>& references = None())
+{
+  // Make sure 'tldr', 'usage', and 'description' end with a newline.
+  if (!strings::endsWith(tldr, "\n")) {
+    tldr += "\n";
+  }
+
+  if (!strings::endsWith(usage, "\n")) {
+    usage += "\n";
+  }
+
+  if (!strings::endsWith(description, "\n")) {
+    description += "\n";
+  }
+
+  // Construct the help string.
+  std::string help =
+    "### TL;DR; ###\n" +
+    tldr +
+    "\n" +
+    "### USAGE ###\n" +
+    usage +
+    "\n" +
+    "### DESCRIPTION ###\n" +
+    description;
+
+  if (references.isSome()) {
+    help += "\n";
+    help += references.get();
+  }
+
+  return help;
+}
+
+
+// Helper for single-line TL;DR; that adds a newline.
+inline std::string TLDR(const std::string& tldr)
+{
+  return tldr + "\n";
+}
+
+
+// Helper for single-line usage that puts it in a blockquote as code
+// and adds a newline.
+inline std::string USAGE(const std::string& usage)
+{
+  return ">        " + usage + "\n";
+}
+
+
+// Helpers for adding newlines to each line of a multi-line
+// description or references.
+#define LINE_TEMPLATE(Z, N, DATA) + CAT(line, N) + "\n"
+#define TEMPLATE(Z, N, DATA)                                            \
+  inline std::string DESCRIPTION(                                       \
+      ENUM_PARAMS(N, const std::string& line))                          \
+  {                                                                     \
+    return                                                              \
+      ""                                                                \
+      REPEAT_FROM_TO(0, N, LINE_TEMPLATE, _);                           \
+  }                                                                     \
+                                                                        \
+                                                                        \
+  inline std::string REFERENCES(                                        \
+      ENUM_PARAMS(N, const std::string& line))                          \
+  {                                                                     \
+    return                                                              \
+      ""                                                                \
+      REPEAT_FROM_TO(0, N, LINE_TEMPLATE, _);                           \
+  }
+
+  REPEAT_FROM_TO(1, 201, TEMPLATE, _) // Lines 1 -> 200.
+#undef TEMPLATE
+#undef LINE_TEMPLATE
+
+
+// Help process for serving /help, /help/id, and /help/id/name (see
+// Help::help below for more information).
+class Help : public Process<Help>
+{
+public:
+  Help() : ProcessBase("help") {}
+
+  // Adds 'help' for the route 'name' of the process with the
+  // specified 'id' (i.e., 'http://ip:port/id/name'). It's expected
+  // that 'help' is written using Markdown. When serving help to a
+  // browser the Markdown will be rendered into HTML while a tool like
+  // 'curl' or 'http' will just be given the Markdown directly (thus
+  // making it easy to get help without opening a browser).
+  // NOTE: There is no need to dispatch this directly; this gets
+  // automagically dispatched by 'ProcessBase::route'.
+  void add(const std::string& id,
+           const std::string& name,
+           const Option<std::string>& help)
+  {
+    if (id != "help") { // TODO(benh): Enable help for help.
+      if (help.isSome()) {
+        helps[id][name] = help.get();
+      } else {
+        helps[id][name] = "## No help page for `/" + id + name + "`\n";
+      }
+      route("/" + id, "Help for " + id, &Help::help);
+    }
+  }
+
+protected:
+  virtual void initialize()
+  {
+    route("/", None(), &Help::help);
+  }
+
+private:
+  // Handles the following:
+  //
+  //   (1) http://ip:port/help
+  //   (2) http://ip:port/help/id
+  //   (3) http://ip:port/help/id/name
+  //
+  // Where 'id' and 'name' are replaced with a process ID and route
+  // name respectively. (1) provides a "table of contents" for all
+  // available processes while (2) provides a "table of contents" for
+  // all endpoints associated with a particular process and (3)
+  // provides the help associated with a particular endpoint of a
+  // process.
+  Future<http::Response> help(const http::Request& request)
+  {
+    // Split the path by '/'.
+    std::vector<std::string> tokens = strings::tokenize(request.path, "/");
+
+    Option<std::string> id = None();
+    Option<std::string> name = None();
+
+    if (tokens.size() > 3) {
+      return http::BadRequest("Malformed URL, expecting '/help/id/name/'\n");
+    } else if (tokens.size() == 3) {
+      id = tokens[1];
+      name = tokens[2];
+    } else if (tokens.size() > 1) {
+      id = tokens[1];
+    }
+
+    std::string document;
+    std::string references;
+
+    if (id.isNone()) {             // http://ip:port/help
+      document += "## HELP\n";
+      foreachkey (const std::string& id, helps) {
+        document += "> [/" + id + "][" + id + "]\n";
+        references += "[" + id + "]: /help/" + id + "\n";
+      }
+    } else if (name.isNone()) {    // http://ip:port/help/id
+      if (helps.count(id.get()) == 0) {
+        return http::BadRequest(
+            "No help available for '/" + id.get() + "'.\n");
+      }
+
+      document += "## `/" + id.get() + "` ##\n";
+      foreachkey (const std::string& name, helps[id.get()]) {
+        const std::string& path = id.get() + name;
+        document += "> [/" +  path + "][" + path + "]\n";
+        references += "[" + path + "]: /help/" + path + "\n";
+      }
+    } else {                       // http://ip:port/help/id/name
+      if (helps.count(id.get()) == 0) {
+        return http::BadRequest(
+            "No help available for '/" + id.get() + "'.\n");
+      } else if (helps[id.get()].count("/" + name.get()) == 0) {
+        return http::BadRequest(
+            "No help available for '/" + id.get() + "/" + name.get() + "'.\n");
+      }
+
+      document += helps[id.get()]["/" + name.get()];
+    }
+
+    // Final Markdown is 'document' followed by the 'references'.
+    std::string markdown = document + "\n" + references;
+
+    // Just send the Markdown if we aren't speaking to a browser. For
+    // now we only check for the 'curl' or 'http' utilities.
+    Option<std::string> agent = request.headers.get("User-Agent");
+
+    if (agent.isSome() &&
+        (strings::startsWith(agent.get(), "curl") ||
+         strings::startsWith(agent.get(), "HTTPie"))) {
+      http::Response response = http::OK(markdown);
+      response.headers["Content-Type"] = "text/x-markdown";
+      return response;
+    }
+
+    // Need to JSONify the markdown for embedding into JavaScript.
+    markdown = stringify(JSON::String(markdown));
+
+    // URL for jQuery.
+    const std::string jquery =
+      "https://ajax.googleapis.com/ajax/libs/jquery/1.10.1/jquery.min.js";
+
+    // Assuming client has Internet access, provide some JavaScript to
+    // render the Markdown into some aesthetically pleasing HTML. ;)
+    // This currently uses GitHub to render the Markdown instead of
+    // doing it client-side in the browser (e.g., using something like
+    // 'showdown.js').
+    return http::OK(
+        "<html>"
+        "<head>"
+        "<title>Help</title>"
+        "<script src=\"" + jquery + "\"></script>"
+        "<script>"
+        "  function loaded() {"
+        "    var markdown = " + markdown + ";"
+        "    if (typeof $ === 'undefined') {"
+        "      document.body.innerHTML = '<pre>' + markdown + '</pre>';"
+        "    } else {"
+        "      var data = { text: markdown, mode: 'gfm' };"
+        "      $.ajax({"
+        "        type: 'POST',"
+        "        url: 'https://api.github.com/markdown',"
+        "        data: JSON.stringify(data),"
+        "        success: function(data) {"
+        "          document.body.innerHTML = data;"
+        "        }"
+        "      });"
+        "    }"
+        "  }"
+        "</script>"
+        "<style>"
+        "body {"
+        "  font-family: Helvetica, arial, sans-serif;"
+        "  font-size: 14px;"
+        "  line-height: 1.6;"
+        "  padding-top: 10px;"
+        "  padding-bottom: 10px;"
+        "  background-color: white;"
+        "  padding: 30px;"
+        "}"
+        "blockquote {"
+        "  border-left: 5px solid #dddddd;"
+        "  padding: 0 10px;"
+        "  color: #777777;"
+        "  margin: 0 0 20px;"
+        "}"
+        "a {"
+        "  color: #0088cc;"
+        "  text-decoration: none;"
+        "}"
+        "</style>"
+        "</head>"
+        "<body onload=\"loaded()\">"
+        "</body>"
+        "</html>");
+  }
+
+  std::map<std::string, std::map<std::string, std::string> > helps;
+};
+
+} // namespace process {
+
+#endif // __PROCESS_HELP_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/include/process/logging.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/logging.hpp b/3rdparty/libprocess/include/process/logging.hpp
index cba2fd4..f4fb619 100644
--- a/3rdparty/libprocess/include/process/logging.hpp
+++ b/3rdparty/libprocess/include/process/logging.hpp
@@ -36,7 +36,7 @@ public:
 protected:
   virtual void initialize()
   {
-    route("/toggle", &This::toggle);
+    route("/toggle", TOGGLE_HELP, &This::toggle);
   }
 
 private:
@@ -101,6 +101,8 @@ private:
     }
   }
 
+  static const std::string TOGGLE_HELP;
+
   Timeout timeout;
 
   const int32_t original; // Original value of FLAGS_v.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/include/process/process.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/process.hpp b/3rdparty/libprocess/include/process/process.hpp
index e70b4f7..df7e143 100644
--- a/3rdparty/libprocess/include/process/process.hpp
+++ b/3rdparty/libprocess/include/process/process.hpp
@@ -18,6 +18,7 @@
 #include <process/pid.hpp>
 
 #include <stout/duration.hpp>
+#include <stout/option.hpp>
 #include <stout/thread.hpp>
 
 namespace process {
@@ -126,18 +127,13 @@ protected:
   // Setup a handler for an HTTP request.
   bool route(
       const std::string& name,
-      const HttpRequestHandler& handler)
-  {
-    if (name.find('/') != 0) {
-      return false;
-    }
-    handlers.http[name.substr(1)] = handler;
-    return true;
-  }
+      const Option<std::string>& help,
+      const HttpRequestHandler& handler);
 
   template <typename T>
   bool route(
       const std::string& name,
+      const Option<std::string>& help,
       Future<http::Response> (T::*method)(const http::Request&))
   {
     // Note that we use dynamic_cast here so a process can use
@@ -146,7 +142,7 @@ protected:
     HttpRequestHandler handler =
       std::tr1::bind(method, dynamic_cast<T*>(this),
                      std::tr1::placeholders::_1);
-    return route(name, handler);
+    return route(name, help, handler);
   }
 
   // Provide the static asset(s) at the specified _absolute_ path for

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/include/process/profiler.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/profiler.hpp b/3rdparty/libprocess/include/process/profiler.hpp
index 64cf622..c886d7e 100644
--- a/3rdparty/libprocess/include/process/profiler.hpp
+++ b/3rdparty/libprocess/include/process/profiler.hpp
@@ -30,11 +30,14 @@ public:
 protected:
   virtual void initialize()
   {
-    route("/start", &Profiler::start);
-    route("/stop", &Profiler::stop);
+    route("/start", START_HELP, &Profiler::start);
+    route("/stop", STOP_HELP, &Profiler::stop);
   }
 
 private:
+  static const std::string START_HELP;
+  static const std::string STOP_HELP;
+
   // HTTP endpoints.
 
   // Starts the profiler. There are no request parameters.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index 3ffe0b5..3bd7015 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -53,6 +53,7 @@
 #include <process/filter.hpp>
 #include <process/future.hpp>
 #include <process/gc.hpp>
+#include <process/help.hpp>
 #include <process/id.hpp>
 #include <process/io.hpp>
 #include <process/logging.hpp>
@@ -411,6 +412,56 @@ private:
 };
 
 
+// Help strings.
+const string Logging::TOGGLE_HELP = HELP(
+    TLDR(
+        "Sets the logging verbosity level for a specified duration."),
+    USAGE(
+        "/logging/toggle?level=VALUE&duration=VALUE"),
+    DESCRIPTION(
+        "The libprocess library uses [glog][glog] for logging. The library",
+        "only uses verbose logging which means nothing will be output unless",
+        "the verbosity level is set (by default it's 0, libprocess uses"
+        "levels 1, 2, and 3).",
+        "",
+        "**NOTE:** If your application uses glog this will also affect",
+        "your verbose logging.",
+        "",
+        "Required query parameters:",
+        "",
+        ">        level=VALUE          Verbosity level (e.g., 1, 2, 3)",
+        ">        duration=VALUE       Duration to keep verbosity level",
+        ">                             toggled (e.g., 10secs, 15mins, etc.)"),
+    REFERENCES(
+        "[glog]: https://code.google.com/p/google-glog"));
+
+
+const string Profiler::START_HELP = HELP(
+    TLDR(
+        "Starts profiling ..."),
+    USAGE(
+        "/profiler/start..."),
+    DESCRIPTION(
+        "...",
+        "",
+        "Query parameters:",
+        "",
+        ">        param=VALUE          Some description here"));
+
+
+const string Profiler::STOP_HELP = HELP(
+    TLDR(
+        "Stops profiling ..."),
+    USAGE(
+        "/profiler/stop..."),
+    DESCRIPTION(
+        "...",
+        "",
+        "Query parameters:",
+        "",
+        ">        param=VALUE          Some description here"));
+
+
 // Unique id that can be assigned to each process.
 static uint32_t __id__ = 0;
 
@@ -472,6 +523,9 @@ static synchronizable(filterer) = SYNCHRONIZED_INITIALIZER_RECURSIVE;
 // Global garbage collector.
 PID<GarbageCollector> gc;
 
+// Global help.
+PID<Help> help;
+
 // Per thread process pointer.
 ThreadLocal<ProcessBase>* _process_ = new ThreadLocal<ProcessBase>();
 
@@ -1400,6 +1454,9 @@ void initialize(const string& delegate)
   // Create global garbage collector process.
   gc = spawn(new GarbageCollector());
 
+  // Create global help process.
+  help = spawn(new Help(), true);
+
   // Create the global logging process.
   spawn(new Logging(), true);
 
@@ -3036,6 +3093,21 @@ UPID ProcessBase::link(const UPID& to)
 }
 
 
+bool ProcessBase::route(
+    const string& name,
+    const Option<string>& help_,
+    const HttpRequestHandler& handler)
+{
+  if (name.find('/') != 0) {
+    return false;
+  }
+  handlers.http[name.substr(1)] = handler;
+  dispatch(help, &Help::add, pid.id, name, help_);
+  return true;
+}
+
+
+
 UPID spawn(ProcessBase* process, bool manage)
 {
   process::initialize();

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/src/statistics.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/statistics.cpp b/3rdparty/libprocess/src/statistics.cpp
index d8f5ad1..d4ba9f1 100644
--- a/3rdparty/libprocess/src/statistics.cpp
+++ b/3rdparty/libprocess/src/statistics.cpp
@@ -11,6 +11,7 @@
 #include <process/delay.hpp>
 #include <process/dispatch.hpp>
 #include <process/future.hpp>
+#include <process/help.hpp>
 #include <process/http.hpp>
 #include <process/process.hpp>
 #include <process/statistics.hpp>
@@ -101,14 +102,17 @@ public:
 protected:
   virtual void initialize()
   {
-    route("/snapshot.json", &StatisticsProcess::snapshot);
-    route("/series.json", &StatisticsProcess::series);
+    route("/snapshot.json", SNAPSHOT_HELP, &StatisticsProcess::snapshot);
+    route("/series.json", SERIES_HELP, &StatisticsProcess::series);
 
     // Schedule the first truncation.
     delay(STATISTICS_TRUNCATION_INTERVAL, self(), &StatisticsProcess::truncate);
   }
 
 private:
+  static const string SNAPSHOT_HELP;
+  static const string SERIES_HELP;
+
   // Removes values for the specified statistic that occurred outside
   // the time series window.
   // NOTE: We always ensure there is at least 1 value left for a statistic,
@@ -140,6 +144,32 @@ private:
 };
 
 
+const string StatisticsProcess::SERIES_HELP = HELP(
+    TLDR(
+        "Provides the time series for ..."),
+    USAGE(
+        "/statistics/series.json..."),
+    DESCRIPTION(
+        "...",
+        "",
+        "Query parameters:",
+        "",
+        ">        param=VALUE          Some description here"));
+
+
+const string StatisticsProcess::SNAPSHOT_HELP = HELP(
+    TLDR(
+        "Provides a snapshot of the current statistics  ..."),
+    USAGE(
+        "/statistics/snapshot.json..."),
+    DESCRIPTION(
+        "...",
+        "",
+        "Query parameters:",
+        "",
+        ">        param=VALUE          Some description here"));
+
+
 Try<Nothing> StatisticsProcess::meter(
     const string& context,
     const string& name,

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/0c38dabd/3rdparty/libprocess/src/tests/http_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/http_tests.cpp b/3rdparty/libprocess/src/tests/http_tests.cpp
index f677267..89a9105 100644
--- a/3rdparty/libprocess/src/tests/http_tests.cpp
+++ b/3rdparty/libprocess/src/tests/http_tests.cpp
@@ -14,6 +14,7 @@
 #include <process/io.hpp>
 
 #include <stout/gtest.hpp>
+#include <stout/none.hpp>
 #include <stout/nothing.hpp>
 #include <stout/os.hpp>
 
@@ -32,8 +33,8 @@ class HttpProcess : public Process<HttpProcess>
 public:
   HttpProcess()
   {
-    route("/body", &HttpProcess::body);
-    route("/pipe", &HttpProcess::pipe);
+    route("/body", None(), &HttpProcess::body);
+    route("/pipe", None(), &HttpProcess::pipe);
   }
 
   MOCK_METHOD1(body, Future<http::Response>(const http::Request&));