You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by wd...@apache.org on 2017/10/17 23:10:40 UTC

[2/3] kudu git commit: Rename *-path-handlers.* to *_path_handlers.*

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/default_path_handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/default_path_handlers.cc b/src/kudu/server/default_path_handlers.cc
new file mode 100644
index 0000000..040aaaf
--- /dev/null
+++ b/src/kudu/server/default_path_handlers.cc
@@ -0,0 +1,319 @@
+// 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 "kudu/server/default_path_handlers.h"
+
+#include <sys/stat.h>
+
+#include <cstddef>
+#include <cstdint>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <boost/algorithm/string/predicate.hpp>
+#include <boost/algorithm/string/replace.hpp>
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <boost/iterator/iterator_traits.hpp>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gperftools/malloc_extension.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/server/pprof_path_handlers.h"
+#include "kudu/server/webserver.h"
+#include "kudu/util/easy_json.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/process_memory.h"
+#include "kudu/util/status.h"
+#include "kudu/util/web_callback_registry.h"
+
+using std::ifstream;
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+DEFINE_int64(web_log_bytes, 1024 * 1024,
+    "The maximum number of bytes to display on the debug webserver's log page");
+TAG_FLAG(web_log_bytes, advanced);
+TAG_FLAG(web_log_bytes, runtime);
+
+// For configuration dashboard
+DECLARE_string(redact);
+DECLARE_string(rpc_encryption);
+DECLARE_string(rpc_authentication);
+DECLARE_string(webserver_certificate_file);
+
+namespace kudu {
+
+using std::shared_ptr;
+
+namespace {
+// Html/Text formatting tags
+struct Tags {
+  string pre_tag, end_pre_tag, line_break, header, end_header;
+
+  // If as_text is true, set the html tags to a corresponding raw text representation.
+  explicit Tags(bool as_text) {
+    if (as_text) {
+      pre_tag = "";
+      end_pre_tag = "\n";
+      line_break = "\n";
+      header = "";
+      end_header = "";
+    } else {
+      pre_tag = "<pre>";
+      end_pre_tag = "</pre>";
+      line_break = "<br/>";
+      header = "<h2>";
+      end_header = "</h2>";
+    }
+  }
+};
+} // anonymous namespace
+
+// Writes the last FLAGS_web_log_bytes of the INFO logfile to a webpage
+// Note to get best performance, set GLOG_logbuflevel=-1 to prevent log buffering
+static void LogsHandler(const Webserver::WebRequest& req, Webserver::WebResponse* resp) {
+  EasyJson* output = resp->output;
+  (*output)["raw"] = (req.parsed_args.find("raw") != req.parsed_args.end());
+  string logfile;
+  GetFullLogFilename(google::INFO, &logfile);
+  (*output)["logfile"] = logfile;
+  struct stat file_stat;
+  if (stat(logfile.c_str(), &file_stat) == 0) {
+    size_t size = file_stat.st_size;
+    size_t seekpos = size < FLAGS_web_log_bytes ? 0L : size - FLAGS_web_log_bytes;
+    ifstream log(logfile.c_str(), std::ios::in);
+    // Note if the file rolls between stat and seek, this could fail
+    // (and we could wind up reading the whole file). But because the
+    // file is likely to be small, this is unlikely to be an issue in
+    // practice.
+    log.seekg(seekpos);
+    (*output)["web_log_bytes"] = FLAGS_web_log_bytes;
+    std::ostringstream ss;
+    ss << log.rdbuf();
+    (*output)["log"] = ss.str();
+  }
+}
+
+// Registered to handle "/flags", and prints out all command-line flags and their HTML
+// escaped values. If --redact is set with 'flag', the values of flags tagged as
+// sensitive will be redacted. The values would not be HTML escaped if in the raw text
+// mode, e.g. "/varz?raw".
+static void FlagsHandler(const Webserver::WebRequest& req,
+                         Webserver::PrerenderedWebResponse* resp) {
+  std::ostringstream* output = resp->output;
+  bool as_text = (req.parsed_args.find("raw") != req.parsed_args.end());
+  Tags tags(as_text);
+
+  (*output) << tags.header << "Command-line Flags" << tags.end_header;
+  (*output) << tags.pre_tag
+            << CommandlineFlagsIntoString(as_text ? EscapeMode::NONE : EscapeMode::HTML)
+            << tags.end_pre_tag;
+}
+
+// Registered to handle "/memz", and prints out memory allocation statistics.
+static void MemUsageHandler(const Webserver::WebRequest& req,
+                            Webserver::PrerenderedWebResponse* resp) {
+  std::ostringstream* output = resp->output;
+  bool as_text = (req.parsed_args.find("raw") != req.parsed_args.end());
+  Tags tags(as_text);
+
+  (*output) << tags.pre_tag;
+#ifndef TCMALLOC_ENABLED
+  (*output) << "Memory tracking is not available unless tcmalloc is enabled.";
+#else
+  faststring buf;
+  buf.resize(32 * 1024);
+  MallocExtension::instance()->GetStats(reinterpret_cast<char*>(buf.data()), buf.size());
+  // Replace new lines with <br> for html
+  string tmp(reinterpret_cast<char*>(buf.data()));
+  boost::replace_all(tmp, "\n", tags.line_break);
+  (*output) << tmp << tags.end_pre_tag;
+#endif
+}
+
+// Registered to handle "/mem-trackers", and prints out to handle memory tracker information.
+static void MemTrackersHandler(const Webserver::WebRequest& /*req*/,
+                               Webserver::PrerenderedWebResponse* resp) {
+  std::ostringstream* output = resp->output;
+  int64_t current_consumption = process_memory::CurrentConsumption();
+  int64_t hard_limit = process_memory::HardLimit();
+  *output << "<h1>Process memory usage</h1>\n";
+  *output << "<table class='table table-striped'>\n";
+  *output << Substitute("  <tr><th>Total consumption</th><td>$0</td></tr>\n",
+                        HumanReadableNumBytes::ToString(current_consumption));
+  *output << Substitute("  <tr><th>Memory limit</th><td>$0</td></tr>\n",
+                        HumanReadableNumBytes::ToString(hard_limit));
+  if (hard_limit > 0) {
+    double percentage = 100 * static_cast<double>(current_consumption) / hard_limit;
+    *output << Substitute("  <tr><th>Percentage consumed</th><td>$0%</td></tr>\n",
+                          StringPrintf("%.2f", percentage));
+  }
+  *output << "</table>\n";
+#ifndef TCMALLOC_ENABLED
+  *output << R"(
+      <div class="alert alert-warning">
+        <strong>NOTE:</strong> This build of Kudu has not enabled tcmalloc.
+        The above process memory stats will be inaccurate.
+      </div>
+               )";
+#endif
+
+  *output << "<h1>Memory usage by subsystem</h1>\n";
+  *output << "<table class='table table-striped'>\n";
+  *output << "  <thead><tr><th>Id</th><th>Parent</th><th>Limit</th><th>Current Consumption</th>"
+      "<th>Peak consumption</th></tr></thead>\n";
+  *output << "<tbody>\n";
+
+  vector<shared_ptr<MemTracker> > trackers;
+  MemTracker::ListTrackers(&trackers);
+  for (const shared_ptr<MemTracker>& tracker : trackers) {
+    string parent = tracker->parent() == nullptr ? "none" : tracker->parent()->id();
+    string limit_str = tracker->limit() == -1 ? "none" :
+                       HumanReadableNumBytes::ToString(tracker->limit());
+    string current_consumption_str = HumanReadableNumBytes::ToString(tracker->consumption());
+    string peak_consumption_str = HumanReadableNumBytes::ToString(tracker->peak_consumption());
+    (*output) << Substitute("  <tr><td>$0</td><td>$1</td><td>$2</td>" // id, parent, limit
+                            "<td>$3</td><td>$4</td></tr>\n", // current, peak
+                            tracker->id(), parent, limit_str, current_consumption_str,
+                            peak_consumption_str);
+  }
+  *output << "</tbody></table>\n";
+}
+
+static void ConfigurationHandler(const Webserver::WebRequest& /* req */,
+                                 Webserver::WebResponse* resp) {
+  EasyJson* output = resp->output;
+  EasyJson security_configs = output->Set("security_configs", EasyJson::kArray);
+
+  EasyJson rpc_encryption = security_configs.PushBack(EasyJson::kObject);
+  rpc_encryption["name"] = "RPC Encryption";
+  rpc_encryption["value"] = FLAGS_rpc_encryption;
+  rpc_encryption["secure"] = boost::iequals(FLAGS_rpc_encryption, "required");
+  rpc_encryption["id"] = "rpc_encryption";
+  rpc_encryption["explanation"] = "Configure with --rpc_encryption. Most secure value is "
+                                  "'required'.";
+
+  EasyJson rpc_authentication = security_configs.PushBack(EasyJson::kObject);
+  rpc_authentication["name"] = "RPC Authentication";
+  rpc_authentication["value"] = FLAGS_rpc_authentication;
+  rpc_authentication["secure"] = boost::iequals(FLAGS_rpc_authentication, "required");
+  rpc_authentication["id"] = "rpc_authentication";
+  rpc_authentication["explanation"] = "Configure with --rpc_authentication. Most secure value is "
+                                      "'required'.";
+
+  EasyJson webserver_encryption = security_configs.PushBack(EasyJson::kObject);
+  webserver_encryption["name"] = "Webserver Encryption";
+  webserver_encryption["value"] = FLAGS_webserver_certificate_file.empty() ? "off" : "on";
+  webserver_encryption["secure"] = !FLAGS_webserver_certificate_file.empty();
+  webserver_encryption["id"] = "webserver_encryption";
+  webserver_encryption["explanation"] = "Configure with --webserver_certificate_file and "
+                                        "webserver_private_key_file.";
+
+  EasyJson webserver_redaction = security_configs.PushBack(EasyJson::kObject);
+  webserver_redaction["name"] = "Webserver Redaction";
+  webserver_redaction["value"] = FLAGS_redact;
+  webserver_redaction["secure"] = boost::iequals(FLAGS_redact, "all");
+  webserver_redaction["id"] = "webserver_redaction";
+  webserver_redaction["explanation"] = "Configure with --redact. Most secure value is 'all'.";
+}
+
+void AddDefaultPathHandlers(Webserver* webserver) {
+  bool styled = true;
+  bool on_nav_bar = true;
+  webserver->RegisterPathHandler("/logs", "Logs", LogsHandler, styled, on_nav_bar);
+  webserver->RegisterPrerenderedPathHandler("/varz", "Flags", FlagsHandler, styled, on_nav_bar);
+  webserver->RegisterPrerenderedPathHandler("/memz", "Memory (total)", MemUsageHandler,
+                                            styled, on_nav_bar);
+  webserver->RegisterPrerenderedPathHandler("/mem-trackers", "Memory (detail)", MemTrackersHandler,
+                                            styled, on_nav_bar);
+  webserver->RegisterPathHandler("/config", "Configuration", ConfigurationHandler,
+                                  styled, on_nav_bar);
+
+  AddPprofPathHandlers(webserver);
+}
+
+
+static void WriteMetricsAsJson(const MetricRegistry* const metrics,
+                               const Webserver::WebRequest& req,
+                               Webserver::PrerenderedWebResponse* resp) {
+  std::ostringstream* output = resp->output;
+  const string* requested_metrics_param = FindOrNull(req.parsed_args, "metrics");
+  vector<string> requested_metrics;
+  MetricJsonOptions opts;
+
+  {
+    string arg = FindWithDefault(req.parsed_args, "include_raw_histograms", "false");
+    opts.include_raw_histograms = ParseLeadingBoolValue(arg.c_str(), false);
+  }
+  {
+    string arg = FindWithDefault(req.parsed_args, "include_schema", "false");
+    opts.include_schema_info = ParseLeadingBoolValue(arg.c_str(), false);
+  }
+  JsonWriter::Mode json_mode;
+  {
+    string arg = FindWithDefault(req.parsed_args, "compact", "false");
+    json_mode = ParseLeadingBoolValue(arg.c_str(), false) ?
+      JsonWriter::COMPACT : JsonWriter::PRETTY;
+  }
+
+  JsonWriter writer(output, json_mode);
+
+  if (requested_metrics_param != nullptr) {
+    SplitStringUsing(*requested_metrics_param, ",", &requested_metrics);
+  } else {
+    // Default to including all metrics.
+    requested_metrics.emplace_back("*");
+  }
+
+  WARN_NOT_OK(metrics->WriteAsJson(&writer, requested_metrics, opts),
+              "Couldn't write JSON metrics over HTTP");
+}
+
+void RegisterMetricsJsonHandler(Webserver* webserver, const MetricRegistry* const metrics) {
+  Webserver::PrerenderedPathHandlerCallback callback = boost::bind(WriteMetricsAsJson, metrics,
+                                                                   _1, _2);
+  bool not_styled = false;
+  bool not_on_nav_bar = false;
+  bool is_on_nav_bar = true;
+  webserver->RegisterPrerenderedPathHandler("/metrics", "Metrics", callback,
+                                            not_styled, is_on_nav_bar);
+
+  // The old name -- this is preserved for compatibility with older releases of
+  // monitoring software which expects the old name.
+  webserver->RegisterPrerenderedPathHandler("/jsonmetricz", "Metrics", callback,
+                                            not_styled, not_on_nav_bar);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/default_path_handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/default_path_handlers.h b/src/kudu/server/default_path_handlers.h
new file mode 100644
index 0000000..82f3e2b
--- /dev/null
+++ b/src/kudu/server/default_path_handlers.h
@@ -0,0 +1,35 @@
+// 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 KUDU_SERVER_DEFAULT_PATH_HANDLERS_H
+#define KUDU_SERVER_DEFAULT_PATH_HANDLERS_H
+
+namespace kudu {
+
+class MetricRegistry;
+class Webserver;
+
+// Adds a set of default path handlers to the webserver to display
+// logs and configuration flags.
+void AddDefaultPathHandlers(Webserver* webserver);
+
+// Adds an endpoint to get metrics in JSON format.
+void RegisterMetricsJsonHandler(Webserver* webserver, const MetricRegistry* const metrics);
+
+} // namespace kudu
+
+#endif // KUDU_SERVER_DEFAULT_PATH_HANDLERS_H

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/pprof-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/pprof-path-handlers.cc b/src/kudu/server/pprof-path-handlers.cc
deleted file mode 100644
index d668889..0000000
--- a/src/kudu/server/pprof-path-handlers.cc
+++ /dev/null
@@ -1,260 +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.
-
-#include "kudu/server/pprof-path-handlers.h"
-
-#include <unistd.h>
-
-#include <cstdint>
-#include <cstdlib>
-#include <fstream>
-#include <string>
-#include <unordered_map>
-#include <utility>
-#include <vector>
-
-#include <gflags/gflags_declare.h>
-#include <glog/logging.h>
-#include <gperftools/heap-profiler.h>
-#include <gperftools/malloc_extension.h>
-#include <gperftools/profiler.h>
-
-#include "kudu/gutil/map-util.h"
-#include "kudu/gutil/strings/numbers.h"
-#include "kudu/gutil/strings/split.h"
-#include "kudu/gutil/strings/stringpiece.h"
-#include "kudu/gutil/strings/strip.h"
-#include "kudu/gutil/strings/substitute.h"
-#include "kudu/gutil/sysinfo.h"
-#include "kudu/server/webserver.h"
-#include "kudu/util/env.h"
-#include "kudu/util/faststring.h"
-#include "kudu/util/monotime.h"
-#include "kudu/util/spinlock_profiling.h"
-#include "kudu/util/status.h"
-#include "kudu/util/web_callback_registry.h"
-
-DECLARE_bool(enable_process_lifetime_heap_profiling);
-DECLARE_string(heap_profile_path);
-
-
-using std::endl;
-using std::ifstream;
-using std::ostringstream;
-using std::string;
-using std::vector;
-
-// GLog already implements symbolization. Just import their hidden symbol.
-namespace google {
-// Symbolizes a program counter.  On success, returns true and write the
-// symbol name to "out".  The symbol name is demangled if possible
-// (supports symbols generated by GCC 3.x or newer).  Otherwise,
-// returns false.
-bool Symbolize(void *pc, char *out, int out_size);
-}
-
-namespace kudu {
-
-const int kPprofDefaultSampleSecs = 30; // pprof default sample time in seconds.
-
-// pprof asks for the url /pprof/cmdline to figure out what application it's profiling.
-// The server should respond by sending the executable path.
-static void PprofCmdLineHandler(const Webserver::WebRequest& /*req*/,
-                                Webserver::PrerenderedWebResponse* resp) {
-  string executable_path;
-  Env* env = Env::Default();
-  WARN_NOT_OK(env->GetExecutablePath(&executable_path), "Failed to get executable path");
-  *resp->output << executable_path;
-}
-
-// pprof asks for the url /pprof/heap to get heap information. This should be implemented
-// by calling HeapProfileStart(filename), continue to do work, and then, some number of
-// seconds later, call GetHeapProfile() followed by HeapProfilerStop().
-static void PprofHeapHandler(const Webserver::WebRequest& req,
-                             Webserver::PrerenderedWebResponse* resp) {
-  ostringstream* output = resp->output;
-#ifndef TCMALLOC_ENABLED
-  *output << "Heap profiling is not available without tcmalloc.";
-#else
-  // Remote (on-demand) profiling is disabled if the process is already being profiled.
-  if (FLAGS_enable_process_lifetime_heap_profiling) {
-    *output << "Heap profiling is running for the process lifetime.";
-    return;
-  }
-
-  auto it = req.parsed_args.find("seconds");
-  int seconds = kPprofDefaultSampleSecs;
-  if (it != req.parsed_args.end()) {
-    seconds = atoi(it->second.c_str());
-  }
-
-  HeapProfilerStart(FLAGS_heap_profile_path.c_str());
-  // Sleep to allow for some samples to be collected.
-  SleepFor(MonoDelta::FromSeconds(seconds));
-  const char* profile = GetHeapProfile();
-  HeapProfilerStop();
-  *output << profile;
-  delete profile;
-#endif
-}
-
-// pprof asks for the url /pprof/profile?seconds=XX to get cpu-profiling information.
-// The server should respond by calling ProfilerStart(), continuing to do its work,
-// and then, XX seconds later, calling ProfilerStop().
-static void PprofCpuProfileHandler(const Webserver::WebRequest& req,
-                                   Webserver::PrerenderedWebResponse* resp) {
-  ostringstream* output = resp->output;
-#ifndef TCMALLOC_ENABLED
-  *output << "CPU profiling is not available without tcmalloc.";
-#else
-  auto it = req.parsed_args.find("seconds");
-  int seconds = kPprofDefaultSampleSecs;
-  if (it != req.parsed_args.end()) {
-    seconds = atoi(it->second.c_str());
-  }
-  // Build a temporary file name that is hopefully unique.
-  string tmp_prof_file_name = strings::Substitute("/tmp/kudu_cpu_profile.$0.$1", getpid(), rand());
-  ProfilerStart(tmp_prof_file_name.c_str());
-  SleepFor(MonoDelta::FromSeconds(seconds));
-  ProfilerStop();
-  ifstream prof_file(tmp_prof_file_name.c_str(), std::ios::in);
-  if (!prof_file.is_open()) {
-    *output << "Unable to open cpu profile: " << tmp_prof_file_name;
-    return;
-  }
-  *output << prof_file.rdbuf();
-  prof_file.close();
-#endif
-}
-
-// pprof asks for the url /pprof/growth to get heap-profiling delta (growth) information.
-// The server should respond by calling:
-// MallocExtension::instance()->GetHeapGrowthStacks(&output);
-static void PprofGrowthHandler(const Webserver::WebRequest& /*req*/,
-                               Webserver::PrerenderedWebResponse* resp) {
-#ifndef TCMALLOC_ENABLED
-  *resp->output << "Growth profiling is not available without tcmalloc.";
-#else
-  string heap_growth_stack;
-  MallocExtension::instance()->GetHeapGrowthStacks(&heap_growth_stack);
-  *resp->output << heap_growth_stack;
-#endif
-}
-
-// Lock contention profiling
-static void PprofContentionHandler(const Webserver::WebRequest& req,
-                                   Webserver::PrerenderedWebResponse* resp) {
-  ostringstream* output = resp->output;
-  string secs_str = FindWithDefault(req.parsed_args, "seconds", "");
-  int32_t seconds = ParseLeadingInt32Value(secs_str.c_str(), kPprofDefaultSampleSecs);
-  int64_t discarded_samples = 0;
-
-  *output << "--- contention" << endl;
-  *output << "sampling period = 1" << endl;
-  *output << "cycles/second = " << base::CyclesPerSecond() << endl;
-
-  MonoTime end = MonoTime::Now() + MonoDelta::FromSeconds(seconds);
-  StartSynchronizationProfiling();
-  while (MonoTime::Now() < end) {
-    SleepFor(MonoDelta::FromMilliseconds(500));
-    FlushSynchronizationProfile(output, &discarded_samples);
-  }
-  StopSynchronizationProfiling();
-  FlushSynchronizationProfile(output, &discarded_samples);
-
-  // pprof itself ignores this value, but we can at least look at it in the textual
-  // output.
-  *output << "discarded samples = " << discarded_samples << std::endl;
-
-#if defined(__linux__)
-  // procfs only exists on Linux.
-  faststring maps;
-  ReadFileToString(Env::Default(), "/proc/self/maps", &maps);
-  *output << maps.ToString();
-#endif // defined(__linux__)
-}
-
-
-// pprof asks for the url /pprof/symbol to map from hex addresses to variable names.
-// When the server receives a GET request for /pprof/symbol, it should return a line
-// formatted like: num_symbols: ###
-// where ### is the number of symbols found in the binary. For now, the only important
-// distinction is whether the value is 0, which it is for executables that lack debug
-// information, or not-0).
-//
-// In addition to the GET request for this url, the server must accept POST requests.
-// This means that after the HTTP headers, pprof will pass in a list of hex addresses
-// connected by +, like:
-//   curl -d '0x0824d061+0x0824d1cf' http://remote_host:80/pprof/symbol
-// The server should read the POST data, which will be in one line, and for each hex value
-// should write one line of output to the output stream, like so:
-// <hex address><tab><function name>
-// For instance:
-// 0x08b2dabd    _Update
-static void PprofSymbolHandler(const Webserver::WebRequest& req,
-                               Webserver::PrerenderedWebResponse* resp) {
-  if (req.request_method == "GET") {
-    // Per the above comment, pprof doesn't expect to know the actual number of symbols.
-    // Any non-zero value indicates that we support symbol lookup.
-    *resp->output << "num_symbols: 1";
-    return;
-  }
-
-  int missing_symbols = 0;
-  int invalid_addrs = 0;
-
-  // Symbolization request.
-  vector<StringPiece> pieces = strings::Split(req.post_data, "+");
-  for (StringPiece p : pieces) {
-    string hex_addr;
-    if (!TryStripPrefixString(p, "0x", &hex_addr)) {
-      invalid_addrs++;
-      continue;
-    }
-    uint64_t addr;
-    if (!safe_strtou64_base(hex_addr.c_str(), &addr, 16)) {
-      invalid_addrs++;
-      continue;
-    }
-    char symbol_buf[1024];
-    if (google::Symbolize(reinterpret_cast<void*>(addr), symbol_buf, sizeof(symbol_buf))) {
-      *resp->output << p << "\t" << symbol_buf << std::endl;
-    } else {
-      missing_symbols++;
-    }
-  }
-
-  LOG(INFO) << strings::Substitute(
-      "Handled request for /pprof/symbol: requested=$0 invalid_addrs=$1 missing=$2",
-      pieces.size(), invalid_addrs, missing_symbols);
-}
-
-void AddPprofPathHandlers(Webserver* webserver) {
-  // Path handlers for remote pprof profiling. For information see:
-  // https://gperftools.googlecode.com/svn/trunk/doc/pprof_remote_servers.html
-  webserver->RegisterPrerenderedPathHandler("/pprof/cmdline", "", PprofCmdLineHandler,
-                                            false, false);
-  webserver->RegisterPrerenderedPathHandler("/pprof/heap", "", PprofHeapHandler, false, false);
-  webserver->RegisterPrerenderedPathHandler("/pprof/growth", "", PprofGrowthHandler, false, false);
-  webserver->RegisterPrerenderedPathHandler("/pprof/profile", "", PprofCpuProfileHandler,
-                                            false, false);
-  webserver->RegisterPrerenderedPathHandler("/pprof/symbol", "", PprofSymbolHandler, false, false);
-  webserver->RegisterPrerenderedPathHandler("/pprof/contention", "", PprofContentionHandler,
-                                            false, false);
-}
-
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/pprof-path-handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/pprof-path-handlers.h b/src/kudu/server/pprof-path-handlers.h
deleted file mode 100644
index e8c0329..0000000
--- a/src/kudu/server/pprof-path-handlers.h
+++ /dev/null
@@ -1,27 +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 KUDU_SERVER_PPROF_DEFAULT_PATH_HANDLERS_H
-#define KUDU_SERVER_PPROF_DEFAULT_PATH_HANDLERS_H
-
-namespace kudu {
-class Webserver;
-
-// Adds set of path handlers to support pprof profiling of a remote server.
-void AddPprofPathHandlers(Webserver* webserver);
-}
-
-#endif // KUDU_SERVER_PPROF_DEFAULT_PATH_HANDLERS_H

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/pprof_path_handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/pprof_path_handlers.cc b/src/kudu/server/pprof_path_handlers.cc
new file mode 100644
index 0000000..1425941
--- /dev/null
+++ b/src/kudu/server/pprof_path_handlers.cc
@@ -0,0 +1,260 @@
+// 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 "kudu/server/pprof_path_handlers.h"
+
+#include <unistd.h>
+
+#include <cstdint>
+#include <cstdlib>
+#include <fstream>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gperftools/heap-profiler.h>
+#include <gperftools/malloc_extension.h>
+#include <gperftools/profiler.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/server/webserver.h"
+#include "kudu/util/env.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/spinlock_profiling.h"
+#include "kudu/util/status.h"
+#include "kudu/util/web_callback_registry.h"
+
+DECLARE_bool(enable_process_lifetime_heap_profiling);
+DECLARE_string(heap_profile_path);
+
+
+using std::endl;
+using std::ifstream;
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+// GLog already implements symbolization. Just import their hidden symbol.
+namespace google {
+// Symbolizes a program counter.  On success, returns true and write the
+// symbol name to "out".  The symbol name is demangled if possible
+// (supports symbols generated by GCC 3.x or newer).  Otherwise,
+// returns false.
+bool Symbolize(void *pc, char *out, int out_size);
+}
+
+namespace kudu {
+
+const int kPprofDefaultSampleSecs = 30; // pprof default sample time in seconds.
+
+// pprof asks for the url /pprof/cmdline to figure out what application it's profiling.
+// The server should respond by sending the executable path.
+static void PprofCmdLineHandler(const Webserver::WebRequest& /*req*/,
+                                Webserver::PrerenderedWebResponse* resp) {
+  string executable_path;
+  Env* env = Env::Default();
+  WARN_NOT_OK(env->GetExecutablePath(&executable_path), "Failed to get executable path");
+  *resp->output << executable_path;
+}
+
+// pprof asks for the url /pprof/heap to get heap information. This should be implemented
+// by calling HeapProfileStart(filename), continue to do work, and then, some number of
+// seconds later, call GetHeapProfile() followed by HeapProfilerStop().
+static void PprofHeapHandler(const Webserver::WebRequest& req,
+                             Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
+#ifndef TCMALLOC_ENABLED
+  *output << "Heap profiling is not available without tcmalloc.";
+#else
+  // Remote (on-demand) profiling is disabled if the process is already being profiled.
+  if (FLAGS_enable_process_lifetime_heap_profiling) {
+    *output << "Heap profiling is running for the process lifetime.";
+    return;
+  }
+
+  auto it = req.parsed_args.find("seconds");
+  int seconds = kPprofDefaultSampleSecs;
+  if (it != req.parsed_args.end()) {
+    seconds = atoi(it->second.c_str());
+  }
+
+  HeapProfilerStart(FLAGS_heap_profile_path.c_str());
+  // Sleep to allow for some samples to be collected.
+  SleepFor(MonoDelta::FromSeconds(seconds));
+  const char* profile = GetHeapProfile();
+  HeapProfilerStop();
+  *output << profile;
+  delete profile;
+#endif
+}
+
+// pprof asks for the url /pprof/profile?seconds=XX to get cpu-profiling information.
+// The server should respond by calling ProfilerStart(), continuing to do its work,
+// and then, XX seconds later, calling ProfilerStop().
+static void PprofCpuProfileHandler(const Webserver::WebRequest& req,
+                                   Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
+#ifndef TCMALLOC_ENABLED
+  *output << "CPU profiling is not available without tcmalloc.";
+#else
+  auto it = req.parsed_args.find("seconds");
+  int seconds = kPprofDefaultSampleSecs;
+  if (it != req.parsed_args.end()) {
+    seconds = atoi(it->second.c_str());
+  }
+  // Build a temporary file name that is hopefully unique.
+  string tmp_prof_file_name = strings::Substitute("/tmp/kudu_cpu_profile.$0.$1", getpid(), rand());
+  ProfilerStart(tmp_prof_file_name.c_str());
+  SleepFor(MonoDelta::FromSeconds(seconds));
+  ProfilerStop();
+  ifstream prof_file(tmp_prof_file_name.c_str(), std::ios::in);
+  if (!prof_file.is_open()) {
+    *output << "Unable to open cpu profile: " << tmp_prof_file_name;
+    return;
+  }
+  *output << prof_file.rdbuf();
+  prof_file.close();
+#endif
+}
+
+// pprof asks for the url /pprof/growth to get heap-profiling delta (growth) information.
+// The server should respond by calling:
+// MallocExtension::instance()->GetHeapGrowthStacks(&output);
+static void PprofGrowthHandler(const Webserver::WebRequest& /*req*/,
+                               Webserver::PrerenderedWebResponse* resp) {
+#ifndef TCMALLOC_ENABLED
+  *resp->output << "Growth profiling is not available without tcmalloc.";
+#else
+  string heap_growth_stack;
+  MallocExtension::instance()->GetHeapGrowthStacks(&heap_growth_stack);
+  *resp->output << heap_growth_stack;
+#endif
+}
+
+// Lock contention profiling
+static void PprofContentionHandler(const Webserver::WebRequest& req,
+                                   Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
+  string secs_str = FindWithDefault(req.parsed_args, "seconds", "");
+  int32_t seconds = ParseLeadingInt32Value(secs_str.c_str(), kPprofDefaultSampleSecs);
+  int64_t discarded_samples = 0;
+
+  *output << "--- contention" << endl;
+  *output << "sampling period = 1" << endl;
+  *output << "cycles/second = " << base::CyclesPerSecond() << endl;
+
+  MonoTime end = MonoTime::Now() + MonoDelta::FromSeconds(seconds);
+  StartSynchronizationProfiling();
+  while (MonoTime::Now() < end) {
+    SleepFor(MonoDelta::FromMilliseconds(500));
+    FlushSynchronizationProfile(output, &discarded_samples);
+  }
+  StopSynchronizationProfiling();
+  FlushSynchronizationProfile(output, &discarded_samples);
+
+  // pprof itself ignores this value, but we can at least look at it in the textual
+  // output.
+  *output << "discarded samples = " << discarded_samples << std::endl;
+
+#if defined(__linux__)
+  // procfs only exists on Linux.
+  faststring maps;
+  ReadFileToString(Env::Default(), "/proc/self/maps", &maps);
+  *output << maps.ToString();
+#endif // defined(__linux__)
+}
+
+
+// pprof asks for the url /pprof/symbol to map from hex addresses to variable names.
+// When the server receives a GET request for /pprof/symbol, it should return a line
+// formatted like: num_symbols: ###
+// where ### is the number of symbols found in the binary. For now, the only important
+// distinction is whether the value is 0, which it is for executables that lack debug
+// information, or not-0).
+//
+// In addition to the GET request for this url, the server must accept POST requests.
+// This means that after the HTTP headers, pprof will pass in a list of hex addresses
+// connected by +, like:
+//   curl -d '0x0824d061+0x0824d1cf' http://remote_host:80/pprof/symbol
+// The server should read the POST data, which will be in one line, and for each hex value
+// should write one line of output to the output stream, like so:
+// <hex address><tab><function name>
+// For instance:
+// 0x08b2dabd    _Update
+static void PprofSymbolHandler(const Webserver::WebRequest& req,
+                               Webserver::PrerenderedWebResponse* resp) {
+  if (req.request_method == "GET") {
+    // Per the above comment, pprof doesn't expect to know the actual number of symbols.
+    // Any non-zero value indicates that we support symbol lookup.
+    *resp->output << "num_symbols: 1";
+    return;
+  }
+
+  int missing_symbols = 0;
+  int invalid_addrs = 0;
+
+  // Symbolization request.
+  vector<StringPiece> pieces = strings::Split(req.post_data, "+");
+  for (StringPiece p : pieces) {
+    string hex_addr;
+    if (!TryStripPrefixString(p, "0x", &hex_addr)) {
+      invalid_addrs++;
+      continue;
+    }
+    uint64_t addr;
+    if (!safe_strtou64_base(hex_addr.c_str(), &addr, 16)) {
+      invalid_addrs++;
+      continue;
+    }
+    char symbol_buf[1024];
+    if (google::Symbolize(reinterpret_cast<void*>(addr), symbol_buf, sizeof(symbol_buf))) {
+      *resp->output << p << "\t" << symbol_buf << std::endl;
+    } else {
+      missing_symbols++;
+    }
+  }
+
+  LOG(INFO) << strings::Substitute(
+      "Handled request for /pprof/symbol: requested=$0 invalid_addrs=$1 missing=$2",
+      pieces.size(), invalid_addrs, missing_symbols);
+}
+
+void AddPprofPathHandlers(Webserver* webserver) {
+  // Path handlers for remote pprof profiling. For information see:
+  // https://gperftools.googlecode.com/svn/trunk/doc/pprof_remote_servers.html
+  webserver->RegisterPrerenderedPathHandler("/pprof/cmdline", "", PprofCmdLineHandler,
+                                            false, false);
+  webserver->RegisterPrerenderedPathHandler("/pprof/heap", "", PprofHeapHandler, false, false);
+  webserver->RegisterPrerenderedPathHandler("/pprof/growth", "", PprofGrowthHandler, false, false);
+  webserver->RegisterPrerenderedPathHandler("/pprof/profile", "", PprofCpuProfileHandler,
+                                            false, false);
+  webserver->RegisterPrerenderedPathHandler("/pprof/symbol", "", PprofSymbolHandler, false, false);
+  webserver->RegisterPrerenderedPathHandler("/pprof/contention", "", PprofContentionHandler,
+                                            false, false);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/pprof_path_handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/pprof_path_handlers.h b/src/kudu/server/pprof_path_handlers.h
new file mode 100644
index 0000000..e8c0329
--- /dev/null
+++ b/src/kudu/server/pprof_path_handlers.h
@@ -0,0 +1,27 @@
+// 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 KUDU_SERVER_PPROF_DEFAULT_PATH_HANDLERS_H
+#define KUDU_SERVER_PPROF_DEFAULT_PATH_HANDLERS_H
+
+namespace kudu {
+class Webserver;
+
+// Adds set of path handlers to support pprof profiling of a remote server.
+void AddPprofPathHandlers(Webserver* webserver);
+}
+
+#endif // KUDU_SERVER_PPROF_DEFAULT_PATH_HANDLERS_H

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index e17358f..9edd463 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -49,7 +49,7 @@
 #include "kudu/rpc/rpc_context.h"
 #include "kudu/rpc/service_if.h"
 #include "kudu/security/init.h"
-#include "kudu/server/default-path-handlers.h"
+#include "kudu/server/default_path_handlers.h"
 #include "kudu/server/generic_service.h"
 #include "kudu/server/glog_metrics.h"
 #include "kudu/server/rpc_server.h"
@@ -57,7 +57,7 @@
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base_options.h"
 #include "kudu/server/tcmalloc_metrics.h"
-#include "kudu/server/tracing-path-handlers.h"
+#include "kudu/server/tracing_path_handlers.h"
 #include "kudu/server/webserver.h"
 #include "kudu/util/atomic.h"
 #include "kudu/util/env.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/tracing-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/tracing-path-handlers.cc b/src/kudu/server/tracing-path-handlers.cc
deleted file mode 100644
index 7849c13..0000000
--- a/src/kudu/server/tracing-path-handlers.cc
+++ /dev/null
@@ -1,287 +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.
-#include "kudu/server/tracing-path-handlers.h"
-
-#include <map>
-#include <memory>
-#include <ostream>
-#include <string>
-#include <utility>
-#include <vector>
-
-
-#include <boost/bind.hpp> // IWYU pragma: keep
-#include <glog/logging.h>
-#include <rapidjson/document.h>
-
-#include "kudu/gutil/strings/escaping.h"
-#include "kudu/server/webserver.h"
-#include "kudu/util/debug/trace_event_impl.h"
-#include "kudu/util/jsonwriter.h"
-#include "kudu/util/monotime.h"
-#include "kudu/util/status.h"
-#include "kudu/util/web_callback_registry.h"
-#include "kudu/util/zlib.h"
-
-using std::map;
-using std::ostringstream;
-using std::pair;
-using std::string;
-using std::unique_ptr;
-using std::vector;
-
-using kudu::debug::CategoryFilter;
-using kudu::debug::TraceLog;
-using kudu::debug::TraceResultBuffer;
-
-namespace kudu {
-namespace server {
-
-enum Handler {
-  kBeginMonitoring,
-  kEndMonitoring,
-  kCaptureMonitoring,
-  kGetMonitoringStatus,
-  kCategories,
-  kBeginRecording,
-  kGetBufferPercentFull,
-  kEndRecording,
-  kEndRecordingCompressed,
-  kSimpleDump
-};
-
-namespace {
-
-Status ParseBase64JsonRequest(const string& json_base64,
-                              rapidjson::Document* doc) {
-  string json_str;
-  if (!strings::Base64Unescape(json_base64, &json_str)) {
-    return Status::InvalidArgument("Invalid base64-encoded JSON");
-  }
-
-  doc->Parse<0>(json_str.c_str());
-  if (!doc->IsObject()) {
-    return Status::InvalidArgument("Invalid JSON", json_str);
-  }
-  return Status::OK();
-}
-
-Status GetTracingOptions(const std::string& json_base64,
-                       std::string* category_filter_string,
-                       int* tracing_options) {
-  rapidjson::Document doc;
-  RETURN_NOT_OK(ParseBase64JsonRequest(json_base64, &doc));
-
-  bool use_continuous_tracing = false;
-  bool use_sampling = false;
-
-  if (!doc.HasMember("categoryFilter") ||
-      !doc["categoryFilter"].IsString()) {
-    return Status::InvalidArgument("missing categoryFilter");
-  }
-  *category_filter_string = doc["categoryFilter"].GetString();
-
-  if (doc.HasMember("useContinuousTracing") &&
-      doc["useContinuousTracing"].IsBool()) {
-    use_continuous_tracing = doc["useContinuousTracing"].GetBool();
-  }
-
-  if (doc.HasMember("useSampling") &&
-      doc["useSampling"].IsBool()) {
-    use_sampling = doc["useSampling"].GetBool();
-  }
-
-  *tracing_options = 0;
-  if (use_sampling)
-    *tracing_options |= TraceLog::ENABLE_SAMPLING;
-  if (use_continuous_tracing)
-    *tracing_options |= TraceLog::RECORD_CONTINUOUSLY;
-  return Status::OK();
-}
-
-Status BeginRecording(const Webserver::WebRequest& req,
-                      TraceLog::Mode mode) {
-  string filter_str;
-  int options;
-  RETURN_NOT_OK(GetTracingOptions(req.query_string, &filter_str, &options));
-
-  kudu::debug::TraceLog::GetInstance()->SetEnabled(
-    CategoryFilter(filter_str),
-    mode,
-    static_cast<TraceLog::Options>(options));
-  return Status::OK();
-}
-
-Status EndRecording(const Webserver::WebRequest& req,
-                    bool compressed,
-                    ostringstream* out) {
-  TraceLog* tl = TraceLog::GetInstance();
-  tl->SetDisabled();
-  string json = TraceResultBuffer::FlushTraceLogToString();
-
-  if (compressed) {
-    RETURN_NOT_OK_PREPEND(zlib::Compress(json, out),
-                          "Could not compress output");
-  } else {
-    *out << json;
-  }
-
-  return Status::OK();
-}
-
-Status CaptureMonitoring(ostringstream* out) {
-  TraceLog* tl = TraceLog::GetInstance();
-  if (!tl->IsEnabled()) {
-    return Status::IllegalState("monitoring not enabled");
-  }
-  *out << TraceResultBuffer::FlushTraceLogToStringButLeaveBufferIntact();
-  return Status::OK();
-}
-
-void GetCategories(ostringstream* out) {
-  vector<string> groups;
-  kudu::debug::TraceLog::GetInstance()->GetKnownCategoryGroups(&groups);
-  JsonWriter j(out, JsonWriter::COMPACT);
-  j.StartArray();
-  for (const string& g : groups) {
-    j.String(g);
-  }
-  j.EndArray();
-}
-
-void GetMonitoringStatus(ostringstream* out) {
-  TraceLog* tl = TraceLog::GetInstance();
-  bool is_monitoring = tl->IsEnabled();
-  std::string category_filter = tl->GetCurrentCategoryFilter().ToString();
-  int options = static_cast<int>(tl->trace_options());
-
-  ostringstream json_out;
-  JsonWriter j(&json_out, JsonWriter::COMPACT);
-  j.StartObject();
-
-  j.String("isMonitoring");
-  j.Bool(is_monitoring);
-
-  j.String("categoryFilter");
-  j.String(category_filter);
-
-  j.String("useContinuousTracing");
-  j.Bool((options & TraceLog::RECORD_CONTINUOUSLY) != 0);
-
-  j.String("useSampling");
-  j.Bool((options & TraceLog::ENABLE_SAMPLING) != 0);
-
-  j.EndObject();
-
-  string encoded;
-  strings::Base64Escape(json_out.str(), &encoded);
-  *out << encoded;
-}
-
-void HandleTraceJsonPage(const Webserver::ArgumentMap &args,
-                         std::ostringstream* output) {
-  TraceLog* tl = TraceLog::GetInstance();
-  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
-                 TraceLog::RECORDING_MODE,
-                 TraceLog::RECORD_CONTINUOUSLY);
-  SleepFor(MonoDelta::FromSeconds(10));
-  tl->SetDisabled();
-
-  *output << TraceResultBuffer::FlushTraceLogToString();
-}
-
-Status DoHandleRequest(Handler handler,
-                       const Webserver::WebRequest& req,
-                       std::ostringstream* output) {
-  VLOG(2) << "Tracing request type=" << handler << ": " << req.query_string;
-
-  switch (handler) {
-    case kBeginMonitoring:
-      RETURN_NOT_OK(BeginRecording(req, TraceLog::MONITORING_MODE));
-      break;
-    case kCaptureMonitoring:
-      RETURN_NOT_OK(CaptureMonitoring(output));
-      break;
-    case kGetMonitoringStatus:
-      GetMonitoringStatus(output);
-      break;
-    case kCategories:
-      GetCategories(output);
-      break;
-    case kBeginRecording:
-      RETURN_NOT_OK(BeginRecording(req, TraceLog::RECORDING_MODE));
-      break;
-    case kGetBufferPercentFull:
-      *output << TraceLog::GetInstance()->GetBufferPercentFull();
-      break;
-    case kEndMonitoring:
-    case kEndRecording:
-      RETURN_NOT_OK(EndRecording(req, false, output));
-      break;
-    case kEndRecordingCompressed:
-      RETURN_NOT_OK(EndRecording(req, true, output));
-      break;
-    case kSimpleDump:
-      HandleTraceJsonPage(req.parsed_args, output);
-      break;
-  }
-
-  return Status::OK();
-}
-
-
-void HandleRequest(Handler handler,
-                   const Webserver::WebRequest& req,
-                   Webserver::PrerenderedWebResponse* resp) {
-  Status s = DoHandleRequest(handler, req, resp->output);
-  if (!s.ok()) {
-    LOG(WARNING) << "Tracing error for handler " << handler << ": "
-                 << s.ToString();
-    // The trace-viewer JS expects '##ERROR##' to indicate that an error
-    // occurred. TODO: change the JS to bubble up the actual error message
-    // to the user.
-    *resp->output << "##ERROR##";
-  }
-}
-} // anonymous namespace
-
-
-void TracingPathHandlers::RegisterHandlers(Webserver* server) {
-  // All of the tracing-related hand
-  std::map<string, Handler> handlers = {
-    { "/tracing/json/begin_monitoring", kBeginMonitoring },
-    { "/tracing/json/end_monitoring", kEndMonitoring },
-    { "/tracing/json/capture_monitoring", kCaptureMonitoring },
-    { "/tracing/json/get_monitoring_status", kGetMonitoringStatus },
-    { "/tracing/json/categories", kCategories },
-    { "/tracing/json/begin_recording", kBeginRecording },
-    { "/tracing/json/get_buffer_percent_full", kGetBufferPercentFull },
-    { "/tracing/json/end_recording", kEndRecording },
-    { "/tracing/json/end_recording_compressed", kEndRecordingCompressed },
-    { "/tracing/json/simple_dump", kSimpleDump } };
-
-  typedef pair<const string, Handler> HandlerPair;
-  for (const HandlerPair& e : handlers) {
-    server->RegisterPrerenderedPathHandler(
-      e.first, "",
-      boost::bind(&HandleRequest, e.second, _1, _2),
-      false /* styled */, false /* is_on_nav_bar */);
-  }
-}
-
-} // namespace server
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/tracing-path-handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/tracing-path-handlers.h b/src/kudu/server/tracing-path-handlers.h
deleted file mode 100644
index 7e1feef..0000000
--- a/src/kudu/server/tracing-path-handlers.h
+++ /dev/null
@@ -1,40 +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 KUDU_SERVER_TRACING_PATH_HANDLERS_H
-#define KUDU_SERVER_TRACING_PATH_HANDLERS_H
-
-#include "kudu/gutil/macros.h"
-
-namespace kudu {
-
-class Webserver;
-
-namespace server {
-
-// Web handlers for Chromium tracing.
-// These handlers provide AJAX endpoints for /tracing.html provided by
-// the trace-viewer package.
-class TracingPathHandlers {
- public:
-  static void RegisterHandlers(Webserver* server);
-
-  DISALLOW_IMPLICIT_CONSTRUCTORS(TracingPathHandlers);
-};
-
-} // namespace server
-} // namespace kudu
-#endif /* KUDU_SERVER_TRACING_PATH_HANDLERS_H */

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/tracing_path_handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/tracing_path_handlers.cc b/src/kudu/server/tracing_path_handlers.cc
new file mode 100644
index 0000000..5e57212
--- /dev/null
+++ b/src/kudu/server/tracing_path_handlers.cc
@@ -0,0 +1,285 @@
+// 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 "kudu/server/tracing_path_handlers.h"
+
+#include <map>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+
+#include <boost/bind.hpp> // IWYU pragma: keep
+#include <glog/logging.h>
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/strings/escaping.h"
+#include "kudu/server/webserver.h"
+#include "kudu/util/debug/trace_event_impl.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/web_callback_registry.h"
+#include "kudu/util/zlib.h"
+
+using std::map;
+using std::ostringstream;
+using std::pair;
+using std::string;
+using std::vector;
+
+using kudu::debug::CategoryFilter;
+using kudu::debug::TraceLog;
+using kudu::debug::TraceResultBuffer;
+
+namespace kudu {
+namespace server {
+
+enum Handler {
+  kBeginMonitoring,
+  kEndMonitoring,
+  kCaptureMonitoring,
+  kGetMonitoringStatus,
+  kCategories,
+  kBeginRecording,
+  kGetBufferPercentFull,
+  kEndRecording,
+  kEndRecordingCompressed,
+  kSimpleDump
+};
+
+namespace {
+
+Status ParseBase64JsonRequest(const string& json_base64,
+                              rapidjson::Document* doc) {
+  string json_str;
+  if (!strings::Base64Unescape(json_base64, &json_str)) {
+    return Status::InvalidArgument("Invalid base64-encoded JSON");
+  }
+
+  doc->Parse<0>(json_str.c_str());
+  if (!doc->IsObject()) {
+    return Status::InvalidArgument("Invalid JSON", json_str);
+  }
+  return Status::OK();
+}
+
+Status GetTracingOptions(const std::string& json_base64,
+                       std::string* category_filter_string,
+                       int* tracing_options) {
+  rapidjson::Document doc;
+  RETURN_NOT_OK(ParseBase64JsonRequest(json_base64, &doc));
+
+  bool use_continuous_tracing = false;
+  bool use_sampling = false;
+
+  if (!doc.HasMember("categoryFilter") ||
+      !doc["categoryFilter"].IsString()) {
+    return Status::InvalidArgument("missing categoryFilter");
+  }
+  *category_filter_string = doc["categoryFilter"].GetString();
+
+  if (doc.HasMember("useContinuousTracing") &&
+      doc["useContinuousTracing"].IsBool()) {
+    use_continuous_tracing = doc["useContinuousTracing"].GetBool();
+  }
+
+  if (doc.HasMember("useSampling") &&
+      doc["useSampling"].IsBool()) {
+    use_sampling = doc["useSampling"].GetBool();
+  }
+
+  *tracing_options = 0;
+  if (use_sampling)
+    *tracing_options |= TraceLog::ENABLE_SAMPLING;
+  if (use_continuous_tracing)
+    *tracing_options |= TraceLog::RECORD_CONTINUOUSLY;
+  return Status::OK();
+}
+
+Status BeginRecording(const Webserver::WebRequest& req,
+                      TraceLog::Mode mode) {
+  string filter_str;
+  int options;
+  RETURN_NOT_OK(GetTracingOptions(req.query_string, &filter_str, &options));
+
+  kudu::debug::TraceLog::GetInstance()->SetEnabled(
+    CategoryFilter(filter_str),
+    mode,
+    static_cast<TraceLog::Options>(options));
+  return Status::OK();
+}
+
+Status EndRecording(const Webserver::WebRequest& /*req*/,
+                    bool compressed,
+                    ostringstream* out) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetDisabled();
+  string json = TraceResultBuffer::FlushTraceLogToString();
+
+  if (compressed) {
+    RETURN_NOT_OK_PREPEND(zlib::Compress(json, out),
+                          "Could not compress output");
+  } else {
+    *out << json;
+  }
+
+  return Status::OK();
+}
+
+Status CaptureMonitoring(ostringstream* out) {
+  TraceLog* tl = TraceLog::GetInstance();
+  if (!tl->IsEnabled()) {
+    return Status::IllegalState("monitoring not enabled");
+  }
+  *out << TraceResultBuffer::FlushTraceLogToStringButLeaveBufferIntact();
+  return Status::OK();
+}
+
+void GetCategories(ostringstream* out) {
+  vector<string> groups;
+  kudu::debug::TraceLog::GetInstance()->GetKnownCategoryGroups(&groups);
+  JsonWriter j(out, JsonWriter::COMPACT);
+  j.StartArray();
+  for (const string& g : groups) {
+    j.String(g);
+  }
+  j.EndArray();
+}
+
+void GetMonitoringStatus(ostringstream* out) {
+  TraceLog* tl = TraceLog::GetInstance();
+  bool is_monitoring = tl->IsEnabled();
+  std::string category_filter = tl->GetCurrentCategoryFilter().ToString();
+  int options = static_cast<int>(tl->trace_options());
+
+  ostringstream json_out;
+  JsonWriter j(&json_out, JsonWriter::COMPACT);
+  j.StartObject();
+
+  j.String("isMonitoring");
+  j.Bool(is_monitoring);
+
+  j.String("categoryFilter");
+  j.String(category_filter);
+
+  j.String("useContinuousTracing");
+  j.Bool((options & TraceLog::RECORD_CONTINUOUSLY) != 0);
+
+  j.String("useSampling");
+  j.Bool((options & TraceLog::ENABLE_SAMPLING) != 0);
+
+  j.EndObject();
+
+  string encoded;
+  strings::Base64Escape(json_out.str(), &encoded);
+  *out << encoded;
+}
+
+void HandleTraceJsonPage(const Webserver::ArgumentMap& /*args*/,
+                         std::ostringstream* output) {
+  TraceLog* tl = TraceLog::GetInstance();
+  tl->SetEnabled(CategoryFilter(CategoryFilter::kDefaultCategoryFilterString),
+                 TraceLog::RECORDING_MODE,
+                 TraceLog::RECORD_CONTINUOUSLY);
+  SleepFor(MonoDelta::FromSeconds(10));
+  tl->SetDisabled();
+
+  *output << TraceResultBuffer::FlushTraceLogToString();
+}
+
+Status DoHandleRequest(Handler handler,
+                       const Webserver::WebRequest& req,
+                       std::ostringstream* output) {
+  VLOG(2) << "Tracing request type=" << handler << ": " << req.query_string;
+
+  switch (handler) {
+    case kBeginMonitoring:
+      RETURN_NOT_OK(BeginRecording(req, TraceLog::MONITORING_MODE));
+      break;
+    case kCaptureMonitoring:
+      RETURN_NOT_OK(CaptureMonitoring(output));
+      break;
+    case kGetMonitoringStatus:
+      GetMonitoringStatus(output);
+      break;
+    case kCategories:
+      GetCategories(output);
+      break;
+    case kBeginRecording:
+      RETURN_NOT_OK(BeginRecording(req, TraceLog::RECORDING_MODE));
+      break;
+    case kGetBufferPercentFull:
+      *output << TraceLog::GetInstance()->GetBufferPercentFull();
+      break;
+    case kEndMonitoring:
+    case kEndRecording:
+      RETURN_NOT_OK(EndRecording(req, false, output));
+      break;
+    case kEndRecordingCompressed:
+      RETURN_NOT_OK(EndRecording(req, true, output));
+      break;
+    case kSimpleDump:
+      HandleTraceJsonPage(req.parsed_args, output);
+      break;
+  }
+
+  return Status::OK();
+}
+
+
+void HandleRequest(Handler handler,
+                   const Webserver::WebRequest& req,
+                   Webserver::PrerenderedWebResponse* resp) {
+  Status s = DoHandleRequest(handler, req, resp->output);
+  if (!s.ok()) {
+    LOG(WARNING) << "Tracing error for handler " << handler << ": "
+                 << s.ToString();
+    // The trace-viewer JS expects '##ERROR##' to indicate that an error
+    // occurred. TODO: change the JS to bubble up the actual error message
+    // to the user.
+    *resp->output << "##ERROR##";
+  }
+}
+} // anonymous namespace
+
+
+void TracingPathHandlers::RegisterHandlers(Webserver* server) {
+  // All of the tracing-related hand
+  std::map<string, Handler> handlers = {
+    { "/tracing/json/begin_monitoring", kBeginMonitoring },
+    { "/tracing/json/end_monitoring", kEndMonitoring },
+    { "/tracing/json/capture_monitoring", kCaptureMonitoring },
+    { "/tracing/json/get_monitoring_status", kGetMonitoringStatus },
+    { "/tracing/json/categories", kCategories },
+    { "/tracing/json/begin_recording", kBeginRecording },
+    { "/tracing/json/get_buffer_percent_full", kGetBufferPercentFull },
+    { "/tracing/json/end_recording", kEndRecording },
+    { "/tracing/json/end_recording_compressed", kEndRecordingCompressed },
+    { "/tracing/json/simple_dump", kSimpleDump } };
+
+  typedef pair<const string, Handler> HandlerPair;
+  for (const HandlerPair& e : handlers) {
+    server->RegisterPrerenderedPathHandler(
+      e.first, "",
+      boost::bind(&HandleRequest, e.second, _1, _2),
+      false /* styled */, false /* is_on_nav_bar */);
+  }
+}
+
+} // namespace server
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/tracing_path_handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/tracing_path_handlers.h b/src/kudu/server/tracing_path_handlers.h
new file mode 100644
index 0000000..7e1feef
--- /dev/null
+++ b/src/kudu/server/tracing_path_handlers.h
@@ -0,0 +1,40 @@
+// 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 KUDU_SERVER_TRACING_PATH_HANDLERS_H
+#define KUDU_SERVER_TRACING_PATH_HANDLERS_H
+
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+
+class Webserver;
+
+namespace server {
+
+// Web handlers for Chromium tracing.
+// These handlers provide AJAX endpoints for /tracing.html provided by
+// the trace-viewer package.
+class TracingPathHandlers {
+ public:
+  static void RegisterHandlers(Webserver* server);
+
+  DISALLOW_IMPLICIT_CONSTRUCTORS(TracingPathHandlers);
+};
+
+} // namespace server
+} // namespace kudu
+#endif /* KUDU_SERVER_TRACING_PATH_HANDLERS_H */

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/server/webserver-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver-test.cc b/src/kudu/server/webserver-test.cc
index d8b3ef4..ca80822 100644
--- a/src/kudu/server/webserver-test.cc
+++ b/src/kudu/server/webserver-test.cc
@@ -31,7 +31,7 @@
 #include "kudu/gutil/strings/util.h"
 #include "kudu/security/test/test_certs.h"
 #include "kudu/security/test/test_pass.h"
-#include "kudu/server/default-path-handlers.h"
+#include "kudu/server/default_path_handlers.h"
 #include "kudu/server/webserver.h"
 #include "kudu/server/webserver_options.h"
 #include "kudu/util/curl_util.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/tserver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/CMakeLists.txt b/src/kudu/tserver/CMakeLists.txt
index c848f95..e062e2d 100644
--- a/src/kudu/tserver/CMakeLists.txt
+++ b/src/kudu/tserver/CMakeLists.txt
@@ -114,7 +114,7 @@ set(TSERVER_SRCS
   tablet_server_options.cc
   tablet_service.cc
   ts_tablet_manager.cc
-  tserver-path-handlers.cc
+  tserver_path_handlers.cc
 )
 
 add_library(tserver ${TSERVER_SRCS})

http://git-wip-us.apache.org/repos/asf/kudu/blob/1c90b2f5/src/kudu/tserver/tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server.cc b/src/kudu/tserver/tablet_server.cc
index 2beea2b..95b2b1e 100644
--- a/src/kudu/tserver/tablet_server.cc
+++ b/src/kudu/tserver/tablet_server.cc
@@ -36,7 +36,7 @@
 #include "kudu/tserver/tablet_copy_service.h"
 #include "kudu/tserver/tablet_service.h"
 #include "kudu/tserver/ts_tablet_manager.h"
-#include "kudu/tserver/tserver-path-handlers.h"
+#include "kudu/tserver/tserver_path_handlers.h"
 #include "kudu/util/maintenance_manager.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/status.h"