You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2017/10/06 04:33:42 UTC

[1/2] kudu git commit: [webui] Allow custom response codes and headers

Repository: kudu
Updated Branches:
  refs/heads/master caff45f5d -> 858bf73b3


[webui] Allow custom response codes and headers

Previously, the path handlers used to implement pages in the web ui
could only return 200 OK and could not set any headers, as these two
aspects of the HTTP response were handled in the underlying webserver
code. This patch introduces WebResponse and PrerenderedWebResponse
structs that wrap and replace the 'output' EasyJson and ostringstream
pointers, respectively, used before, and which have fields for the
response code and additional headers.

The ability to add headers isn't currently used, but it's nice to have.
The response codes are adjusted where necessary to match what one
would expect, e.g. navigating to /tablet?id=foo returns
404.

Change-Id: I9ff890785eeb2df3eed9e7c54d0daf760c8b3924
Reviewed-on: http://gerrit.cloudera.org:8080/8141
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/1ee49525
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/1ee49525
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/1ee49525

Branch: refs/heads/master
Commit: 1ee495255d5e01e22f6675d93e46f9080bf6a6aa
Parents: caff45f
Author: Will Berkeley <wd...@apache.org>
Authored: Sat Sep 16 15:02:04 2017 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Thu Oct 5 22:18:07 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/master-path-handlers.cc   | 31 ++++++---
 src/kudu/master/master-path-handlers.h    | 11 ++-
 src/kudu/server/default-path-handlers.cc  | 23 ++++--
 src/kudu/server/pprof-path-handlers.cc    | 45 +++++++-----
 src/kudu/server/rpcz-path-handler.cc      |  7 +-
 src/kudu/server/tracing-path-handlers.cc  |  6 +-
 src/kudu/server/webserver.cc              | 94 +++++++++++++++++--------
 src/kudu/server/webserver.h               |  9 ++-
 src/kudu/tserver/tserver-path-handlers.cc | 96 +++++++++++++++-----------
 src/kudu/tserver/tserver-path-handlers.h  | 20 +++---
 src/kudu/util/thread.cc                   | 11 +--
 src/kudu/util/web_callback_registry.h     | 53 +++++++++++---
 12 files changed, 269 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/master/master-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-path-handlers.cc b/src/kudu/master/master-path-handlers.cc
index dc621eb..28b604a 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -20,11 +20,13 @@
 #include <algorithm>
 #include <array>
 #include <cstdint>
+#include <iosfwd>
 #include <map>
 #include <memory>
 #include <sstream>
 #include <string>
 #include <type_traits>
+#include <unordered_map>
 #include <utility>
 #include <vector>
 
@@ -83,8 +85,9 @@ namespace master {
 MasterPathHandlers::~MasterPathHandlers() {
 }
 
-void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& req,
-                                             ostringstream* output) {
+void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& /*req*/,
+                                             Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   vector<std::shared_ptr<TSDescriptor> > descs;
   master_->ts_manager()->GetAllDescriptors(&descs);
 
@@ -159,7 +162,9 @@ int ExtractRedirectsFromRequest(const Webserver::WebRequest& req) {
 
 } // anonymous namespace
 
-void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req, EasyJson* output) {
+void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req,
+                                              Webserver::WebResponse* resp) {
+  EasyJson* output = resp->output;
   CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
   if (!l.catalog_status().ok()) {
     (*output)["error"] = Substitute("Master is not ready: $0",  l.catalog_status().ToString());
@@ -220,11 +225,12 @@ bool CompareByRole(const pair<TabletDetailPeerInfo, RaftPeerPB::Role>& a,
 
 
 void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
-                                         EasyJson* output) {
+                                         Webserver::WebResponse* resp) {
+  EasyJson* output = resp->output;
   // Parse argument.
   string table_id;
   if (!FindCopy(req.parsed_args, "id", &table_id)) {
-    // TODO(wdb): Webserver should give a way to return a non-200 response code.
+    resp->status_code = HttpStatusCode::BadRequest;
     (*output)["error"] = "Missing 'id' argument";
     return;
   }
@@ -235,7 +241,10 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
     return;
   }
   if (!l.leader_status().ok()) {
-    // Track redirects to prevent a redirect loop.
+    // It's possible to respond 307 Temporary Redirect and automatically redirect with
+    // a Location header, but this would likely confuse users about which master's web ui
+    // they are looking at. Instead, we show a link users can click to go to the leader master.
+    // We track redirects to prevent a redirect loop.
     int redirects = ExtractRedirectsFromRequest(req);
     SetupLeaderMasterRedirect(Substitute("table?id=$0", table_id), redirects, output);
     return;
@@ -244,11 +253,13 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
   scoped_refptr<TableInfo> table;
   Status s = master_->catalog_manager()->GetTableInfo(table_id, &table);
   if (!s.ok()) {
+    resp->status_code = HttpStatusCode::ServiceUnavailable;
     (*output)["error"] = Substitute("Master is not ready: $0", s.ToString());
     return;
   }
 
   if (!table) {
+    resp->status_code = HttpStatusCode::NotFound;
     (*output)["error"] = "Table not found";
     return;
   }
@@ -376,7 +387,8 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
 }
 
 void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& /*req*/,
-                                       ostringstream* output) {
+                                       Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   vector<ServerEntryPB> masters;
   Status s = master_->ListMasters(&masters);
   if (!s.ok()) {
@@ -516,8 +528,9 @@ void JsonError(const Status& s, ostringstream* out) {
 }
 } // anonymous namespace
 
-void MasterPathHandlers::HandleDumpEntities(const Webserver::WebRequest& req,
-                                            ostringstream* output) {
+void MasterPathHandlers::HandleDumpEntities(const Webserver::WebRequest& /*req*/,
+                                            Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   Status s = master_->catalog_manager()->CheckOnline();
   if (!s.ok()) {
     JsonError(s, output);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/master/master-path-handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-path-handlers.h b/src/kudu/master/master-path-handlers.h
index 72cff3f..280b397 100644
--- a/src/kudu/master/master-path-handlers.h
+++ b/src/kudu/master/master-path-handlers.h
@@ -17,7 +17,6 @@
 #ifndef KUDU_MASTER_MASTER_PATH_HANDLERS_H
 #define KUDU_MASTER_MASTER_PATH_HANDLERS_H
 
-#include <iosfwd>
 #include <string>
 #include <utility>
 
@@ -47,15 +46,15 @@ class MasterPathHandlers {
 
  private:
   void HandleTabletServers(const Webserver::WebRequest& req,
-                           std::ostringstream* output);
+                           Webserver::PrerenderedWebResponse* resp);
   void HandleCatalogManager(const Webserver::WebRequest& req,
-                            EasyJson* output);
+                            Webserver::WebResponse* resp);
   void HandleTablePage(const Webserver::WebRequest& req,
-                       EasyJson* output);
+                       Webserver::WebResponse* resp);
   void HandleMasters(const Webserver::WebRequest& req,
-                     std::ostringstream* output);
+                     Webserver::PrerenderedWebResponse* resp);
   void HandleDumpEntities(const Webserver::WebRequest& req,
-                          std::ostringstream* output);
+                          Webserver::PrerenderedWebResponse* resp);
 
   // Returns a pair (text, target) given a tserver's TSDescriptor and a tablet id.
   // - text is the http host and port for the tserver, if available, or the tserver's uuid.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/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
index d3050f1..e71922b 100644
--- a/src/kudu/server/default-path-handlers.cc
+++ b/src/kudu/server/default-path-handlers.cc
@@ -102,7 +102,8 @@ struct Tags {
 
 // 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, EasyJson* output) {
+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);
@@ -128,7 +129,9 @@ static void LogsHandler(const Webserver::WebRequest& req, EasyJson* output) {
 // 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, std::ostringstream* output) {
+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);
 
@@ -139,7 +142,9 @@ static void FlagsHandler(const Webserver::WebRequest& req, std::ostringstream* o
 }
 
 // Registered to handle "/memz", and prints out memory allocation statistics.
-static void MemUsageHandler(const Webserver::WebRequest& req, std::ostringstream* output) {
+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);
 
@@ -157,7 +162,9 @@ static void MemUsageHandler(const Webserver::WebRequest& req, std::ostringstream
 }
 
 // Registered to handle "/mem-trackers", and prints out to handle memory tracker information.
-static void MemTrackersHandler(const Webserver::WebRequest& /*req*/, std::ostringstream* output) {
+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";
@@ -203,7 +210,9 @@ static void MemTrackersHandler(const Webserver::WebRequest& /*req*/, std::ostrin
   *output << "</tbody></table>\n";
 }
 
-static void ConfigurationHandler(const Webserver::WebRequest& /* req */, EasyJson* output) {
+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);
@@ -255,7 +264,9 @@ void AddDefaultPathHandlers(Webserver* webserver) {
 
 
 static void WriteMetricsAsJson(const MetricRegistry* const metrics,
-                               const Webserver::WebRequest& req, std::ostringstream* output) {
+                               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;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/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
index 1efb909..d668889 100644
--- a/src/kudu/server/pprof-path-handlers.cc
+++ b/src/kudu/server/pprof-path-handlers.cc
@@ -22,8 +22,8 @@
 #include <cstdint>
 #include <cstdlib>
 #include <fstream>
-#include <map>
 #include <string>
+#include <unordered_map>
 #include <utility>
 #include <vector>
 
@@ -73,23 +73,26 @@ 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, ostringstream* output) {
+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");
-  *output << 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, ostringstream* output) {
+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.";
+  *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.";
+    *output << "Heap profiling is running for the process lifetime.";
     return;
   }
 
@@ -104,7 +107,7 @@ static void PprofHeapHandler(const Webserver::WebRequest& req, ostringstream* ou
   SleepFor(MonoDelta::FromSeconds(seconds));
   const char* profile = GetHeapProfile();
   HeapProfilerStop();
-  (*output) << profile;
+  *output << profile;
   delete profile;
 #endif
 }
@@ -112,9 +115,11 @@ static void PprofHeapHandler(const Webserver::WebRequest& req, ostringstream* ou
 // 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, ostringstream* output) {
+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.";
+  *output << "CPU profiling is not available without tcmalloc.";
 #else
   auto it = req.parsed_args.find("seconds");
   int seconds = kPprofDefaultSampleSecs;
@@ -128,10 +133,10 @@ static void PprofCpuProfileHandler(const Webserver::WebRequest& req, ostringstre
   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;
+    *output << "Unable to open cpu profile: " << tmp_prof_file_name;
     return;
   }
-  (*output) << prof_file.rdbuf();
+  *output << prof_file.rdbuf();
   prof_file.close();
 #endif
 }
@@ -139,18 +144,21 @@ static void PprofCpuProfileHandler(const Webserver::WebRequest& req, ostringstre
 // 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, ostringstream* output) {
+static void PprofGrowthHandler(const Webserver::WebRequest& /*req*/,
+                               Webserver::PrerenderedWebResponse* resp) {
 #ifndef TCMALLOC_ENABLED
-  (*output) << "Growth profiling is not available without tcmalloc.";
+  *resp->output << "Growth profiling is not available without tcmalloc.";
 #else
   string heap_growth_stack;
   MallocExtension::instance()->GetHeapGrowthStacks(&heap_growth_stack);
-  (*output) << heap_growth_stack;
+  *resp->output << heap_growth_stack;
 #endif
 }
 
 // Lock contention profiling
-static void PprofContentionHandler(const Webserver::WebRequest& req, ostringstream* output) {
+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;
@@ -197,11 +205,12 @@ static void PprofContentionHandler(const Webserver::WebRequest& req, ostringstre
 // <hex address><tab><function name>
 // For instance:
 // 0x08b2dabd    _Update
-static void PprofSymbolHandler(const Webserver::WebRequest& req, ostringstream* output) {
+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.
-    (*output) << "num_symbols: 1";
+    *resp->output << "num_symbols: 1";
     return;
   }
 
@@ -223,7 +232,7 @@ static void PprofSymbolHandler(const Webserver::WebRequest& req, ostringstream*
     }
     char symbol_buf[1024];
     if (google::Symbolize(reinterpret_cast<void*>(addr), symbol_buf, sizeof(symbol_buf))) {
-      *output << p << "\t" << symbol_buf << std::endl;
+      *resp->output << p << "\t" << symbol_buf << std::endl;
     } else {
       missing_symbols++;
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/server/rpcz-path-handler.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/rpcz-path-handler.cc b/src/kudu/server/rpcz-path-handler.cc
index 48c4f5d..80a2840 100644
--- a/src/kudu/server/rpcz-path-handler.cc
+++ b/src/kudu/server/rpcz-path-handler.cc
@@ -17,10 +17,10 @@
 
 #include "kudu/server/rpcz-path-handler.h"
 
-#include <map>
 #include <memory>
 #include <sstream>
 #include <string>
+#include <unordered_map>
 
 #include <boost/bind.hpp> // IWYU pragma: keep
 
@@ -47,7 +47,8 @@ namespace kudu {
 namespace {
 
 void RpczPathHandler(const shared_ptr<Messenger>& messenger,
-                     const Webserver::WebRequest& req, ostringstream* output) {
+                     const Webserver::WebRequest& req,
+                     Webserver::PrerenderedWebResponse* resp) {
   DumpRunningRpcsResponsePB running_rpcs;
   {
     DumpRunningRpcsRequestPB dump_req;
@@ -63,7 +64,7 @@ void RpczPathHandler(const shared_ptr<Messenger>& messenger,
     messenger->rpcz_store()->DumpPB(dump_req, &sampled_rpcs);
   }
 
-  JsonWriter writer(output, JsonWriter::PRETTY);
+  JsonWriter writer(resp->output, JsonWriter::PRETTY);
   writer.StartObject();
   writer.String("running");
   writer.Protobuf(running_rpcs);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/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
index 6d8c3a2..7849c13 100644
--- a/src/kudu/server/tracing-path-handlers.cc
+++ b/src/kudu/server/tracing-path-handlers.cc
@@ -246,15 +246,15 @@ Status DoHandleRequest(Handler handler,
 
 void HandleRequest(Handler handler,
                    const Webserver::WebRequest& req,
-                   std::ostringstream* output) {
-  Status s = DoHandleRequest(handler, req, output);
+                   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.
-    *output << "##ERROR##";
+    *resp->output << "##ERROR##";
   }
 }
 } // anonymous namespace

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/server/webserver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver.cc b/src/kudu/server/webserver.cc
index 9127a10..58a7f8e 100644
--- a/src/kudu/server/webserver.cc
+++ b/src/kudu/server/webserver.cc
@@ -28,6 +28,7 @@
 #include <mutex>
 #include <sstream>
 #include <string>
+#include <unordered_set>
 #include <utility>
 #include <vector>
 
@@ -83,6 +84,27 @@ TAG_FLAG(webserver_x_frame_options, advanced);
 namespace {
   // Last error message from the webserver.
   string kWebserverLastErrMsg;
+
+  string HttpStatusCodeToString(kudu::HttpStatusCode code) {
+    switch (code) {
+      case kudu::HttpStatusCode::Ok:
+        return "200 OK";
+      case kudu::HttpStatusCode::BadRequest:
+        return "400 Bad Request";
+      case kudu::HttpStatusCode::NotFound:
+        return "404 Not Found";
+      case kudu::HttpStatusCode::LengthRequired:
+        return "411 Length Required";
+      case kudu::HttpStatusCode::RequestEntityTooLarge:
+        return "413 Request Entity Too Large";
+      case kudu::HttpStatusCode::InternalServerError:
+        return "500 Internal Server Error";
+      case kudu::HttpStatusCode::ServiceUnavailable:
+        return "503 Service Unavailable";
+    }
+    LOG(FATAL) << "Unexpected HTTP response code";
+  }
+
 }  // anonymous namespace
 
 namespace kudu {
@@ -99,8 +121,9 @@ Webserver::~Webserver() {
   STLDeleteValues(&path_handlers_);
 }
 
-void Webserver::RootHandler(const Webserver::WebRequest& /* args */, EasyJson* output) {
-  EasyJson path_handlers = output->Set("path_handlers", EasyJson::kArray);
+void Webserver::RootHandler(const Webserver::WebRequest& /* args */,
+                            Webserver::WebResponse* resp) {
+  EasyJson path_handlers = resp->output->Set("path_handlers", EasyJson::kArray);
   for (const PathHandlerMap::value_type& handler : path_handlers_) {
     if (handler.second->is_on_nav_bar()) {
       EasyJson path_handler = path_handlers.PushBack(EasyJson::kObject);
@@ -108,7 +131,7 @@ void Webserver::RootHandler(const Webserver::WebRequest& /* args */, EasyJson* o
       path_handler["alias"] = handler.second->alias();
     }
   }
-  (*output)["version_info"] = EscapeForHtmlToString(VersionInfo::GetAllVersionInfo());
+  (*resp->output)["version_info"] = EscapeForHtmlToString(VersionInfo::GetAllVersionInfo());
 }
 
 void Webserver::BuildArgumentMap(const string& args, ArgumentMap* output) {
@@ -360,12 +383,12 @@ int Webserver::BeginRequestCallback(struct sq_connection* connection,
       if (!opts_.doc_root.empty() && opts_.enable_doc_root) {
         VLOG(2) << "HTTP File access: " << request_info->uri;
         return 0;
-      } else {
-        sq_printf(connection, "HTTP/1.1 404 Not Found\r\n"
-                  "Content-Type: text/plain\r\n\r\n");
-        sq_printf(connection, "No handler for URI %s\r\n\r\n", request_info->uri);
-        return 1;
       }
+      sq_printf(connection,
+                "HTTP/1.1 %s\r\nContent-Type: text/plain\r\n\r\n",
+                HttpStatusCodeToString(HttpStatusCode::NotFound).c_str());
+      sq_printf(connection, "No handler for URI %s\r\n\r\n", request_info->uri);
+      return 1;
     }
     handler = it->second;
   }
@@ -390,14 +413,18 @@ int Webserver::RunPathHandler(const PathHandler& handler,
     int32_t content_len = 0;
     if (content_len_str == nullptr ||
         !safe_strto32(content_len_str, &content_len)) {
-      sq_printf(connection, "HTTP/1.1 411 Length Required\r\n");
+      sq_printf(connection,
+                "HTTP/1.1 %s\r\n",
+                HttpStatusCodeToString(HttpStatusCode::LengthRequired).c_str());
       return 1;
     }
     if (content_len > FLAGS_webserver_max_post_length_bytes) {
-      // TODO: for this and other HTTP requests, we should log the
+      // TODO(wdb): for this and other HTTP requests, we should log the
       // remote IP, etc.
       LOG(WARNING) << "Rejected POST with content length " << content_len;
-      sq_printf(connection, "HTTP/1.1 413 Request Entity Too Large\r\n");
+      sq_printf(connection,
+                "HTTP/1.1 %s\r\n",
+                HttpStatusCodeToString(HttpStatusCode::RequestEntityTooLarge).c_str());
       return 1;
     }
 
@@ -409,7 +436,9 @@ int Webserver::RunPathHandler(const PathHandler& handler,
         LOG(WARNING) << "error reading POST data: expected "
                      << content_len << " bytes but only read "
                      << req.post_data.size();
-        sq_printf(connection, "HTTP/1.1 500 Internal Server Error\r\n");
+        sq_printf(connection,
+                  "HTTP/1.1 %s\r\n",
+                  HttpStatusCodeToString(HttpStatusCode::InternalServerError).c_str());
         return 1;
       }
 
@@ -423,7 +452,8 @@ int Webserver::RunPathHandler(const PathHandler& handler,
   }
 
   ostringstream content;
-  handler.callback()(req, &content);
+  PrerenderedWebResponse resp { HttpStatusCode::Ok, HttpResponseHeaders{}, &content };
+  handler.callback()(req, &resp);
 
   string full_content;
   if (use_style) {
@@ -434,17 +464,24 @@ int Webserver::RunPathHandler(const PathHandler& handler,
     full_content = content.str();
   }
 
-  // Without styling, render the page as plain text
-  string headers = strings::Substitute(
-      "HTTP/1.1 200 OK\r\n"
-      "Content-Type: $0\r\n"
-      "Content-Length: $1\r\n"
-      "X-Frame-Options: $2\r\n"
-      "\r\n",
-      use_style ? "text/html" : "text/plain",
-      full_content.length(),
-      FLAGS_webserver_x_frame_options);
-  // Make sure to use sq_write for printing the body; sq_printf truncates at 8kb
+  ostringstream headers_stream;
+  headers_stream << Substitute("HTTP/1.1 $0\r\n", HttpStatusCodeToString(resp.status_code));
+  headers_stream << Substitute("Content-Type: $0\r\n", use_style ? "text/html" : "text/plain");
+  headers_stream << Substitute("Content-Length: $0\r\n", full_content.length());
+  headers_stream << Substitute("X-Frame-Options: $0\r\n", FLAGS_webserver_x_frame_options);
+  std::unordered_set<string> invalid_headers{"Content-Type", "Content-Length", "X-Frame-Options"};
+  for (const auto& entry : resp.response_headers) {
+    // It's forbidden to override the above headers.
+    if (ContainsKey(invalid_headers, entry.first)) {
+      LOG(FATAL) << "Reserved header " << entry.first << " was overridden "
+          "by handler for " << handler.alias();
+    }
+    headers_stream << Substitute("$0: $1\r\n", entry.first, entry.second);
+  }
+  headers_stream << "\r\n";
+  string headers = headers_stream.str();
+
+  // Make sure to use sq_write for printing the body; sq_printf truncates at 8KB.
   sq_write(connection, headers.c_str(), headers.length());
   sq_write(connection, full_content.c_str(), full_content.length());
   return 1;
@@ -453,12 +490,15 @@ int Webserver::RunPathHandler(const PathHandler& handler,
 void Webserver::RegisterPathHandler(const string& path, const string& alias,
     const PathHandlerCallback& callback, bool is_styled, bool is_on_nav_bar) {
   string render_path = (path == "/") ? "/home" : path;
-  auto wrapped_cb = [=](const WebRequest& args, ostringstream* output) {
+  auto wrapped_cb = [=](const WebRequest& args, PrerenderedWebResponse* rendered_resp) {
     EasyJson ej;
-    callback(args, &ej);
+    WebResponse resp { HttpStatusCode::Ok, HttpResponseHeaders{}, &ej };
+    callback(args, &resp);
     stringstream out;
     Render(render_path, ej, is_styled, &out);
-    (*output) << out.rdbuf();
+    rendered_resp->status_code = resp.status_code;
+    rendered_resp->response_headers = std::move(resp.response_headers);
+    *rendered_resp->output << out.rdbuf();
   };
   RegisterPrerenderedPathHandler(path, alias, wrapped_cb, is_styled, is_on_nav_bar);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/server/webserver.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver.h b/src/kudu/server/webserver.h
index f3f8ebc..0c47479 100644
--- a/src/kudu/server/webserver.h
+++ b/src/kudu/server/webserver.h
@@ -63,10 +63,16 @@ class Webserver : public WebCallbackRegistry {
   // to the world. Requires that the server has been Start()ed.
   Status GetAdvertisedAddresses(std::vector<Sockaddr>* addresses) const WARN_UNUSED_RESULT;
 
+  // Register a route 'path' to be rendered via template.
+  // The appropriate template to use is determined by 'path'.
+  // If 'is_styled' is true, the page will be styled and include a header and footer.
+  // If 'is_on_nav_bar' is true, a link to the page will be placed on the navbar
+  // in the header of styled pages. The link text is given by 'alias'.
   void RegisterPathHandler(const std::string& path, const std::string& alias,
                            const PathHandlerCallback& callback,
                            bool is_styled, bool is_on_nav_bar) override;
 
+  // Register a route 'path'. See the RegisterPathHandler for details.
   void RegisterPrerenderedPathHandler(const std::string& path, const std::string& alias,
                                       const PrerenderedPathHandlerCallback& callback,
                                       bool is_styled,
@@ -77,6 +83,7 @@ class Webserver : public WebCallbackRegistry {
 
   // True if serving all traffic over SSL, false otherwise
   bool IsSecure() const;
+
  private:
   // Container class for a list of path handler callbacks for a single URL.
   class PathHandler {
@@ -144,7 +151,7 @@ class Webserver : public WebCallbackRegistry {
                                       const char* message);
 
   // Registered to handle "/", and prints a list of available URIs
-  void RootHandler(const WebRequest& args, EasyJson* output);
+  void RootHandler(const WebRequest& args, WebResponse* resp);
 
   // Builds a map of argument name to argument value from a typical URL argument
   // string (that is, "key1=value1&key2=value2.."). If no value is given for a

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/tserver/tserver-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver-path-handlers.cc b/src/kudu/tserver/tserver-path-handlers.cc
index c348341..49b2df2 100644
--- a/src/kudu/tserver/tserver-path-handlers.cc
+++ b/src/kudu/tserver/tserver-path-handlers.cc
@@ -20,11 +20,13 @@
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <iosfwd>
 #include <map>
 #include <memory>
 #include <sstream>
 #include <string>
 #include <type_traits>
+#include <unordered_map>
 #include <utility>
 #include <vector>
 
@@ -88,6 +90,7 @@ using kudu::tablet::TabletStatusPB;
 using kudu::tablet::Transaction;
 using std::endl;
 using std::map;
+using std::ostringstream;
 using std::shared_ptr;
 using std::string;
 using std::vector;
@@ -141,7 +144,8 @@ Status TabletServerPathHandlers::Register(Webserver* server) {
 }
 
 void TabletServerPathHandlers::HandleTransactionsPage(const Webserver::WebRequest& req,
-                                                      std::ostringstream* output) {
+                                                      Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   bool as_text = ContainsKey(req.parsed_args, "raw");
 
   vector<scoped_refptr<TabletReplica> > replicas;
@@ -179,7 +183,7 @@ void TabletServerPathHandlers::HandleTransactionsPage(const Webserver::WebReques
       }
 
       if (!as_text) {
-        (*output) << Substitute(
+        *output << Substitute(
           "<tr><th>$0</th><th>$1</th><th>$2</th><th>$3</th><th>$4</th></tr>\n",
           EscapeForHtmlToString(replica->tablet_id()),
           EscapeForHtmlToString(SecureShortDebugString(inflight_tx.op_id())),
@@ -187,12 +191,12 @@ void TabletServerPathHandlers::HandleTransactionsPage(const Webserver::WebReques
           total_time_str,
           EscapeForHtmlToString(description));
       } else {
-        (*output) << "Tablet: " << replica->tablet_id() << endl;
-        (*output) << "Op ID: " << SecureShortDebugString(inflight_tx.op_id()) << endl;
-        (*output) << "Type: " << OperationType_Name(inflight_tx.tx_type()) << endl;
-        (*output) << "Running: " << total_time_str;
-        (*output) << description << endl;
-        (*output) << endl;
+        *output << "Tablet: " << replica->tablet_id() << endl;
+        *output << "Op ID: " << SecureShortDebugString(inflight_tx.op_id()) << endl;
+        *output << "Type: " << OperationType_Name(inflight_tx.tx_type()) << endl;
+        *output << "Running: " << total_time_str;
+        *output << description << endl;
+        *output << endl;
       }
     }
   }
@@ -211,8 +215,9 @@ string TabletLink(const string& id) {
 
 } // anonymous namespace
 
-void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& req,
-                                                 std::ostringstream *output) {
+void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*req*/,
+                                                 Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   vector<scoped_refptr<TabletReplica>> replicas;
   tserver_->tablet_manager()->GetTabletReplicas(&replicas);
 
@@ -284,7 +289,7 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& re
                                                        replica->tablet_metadata()->schema());
 
       shared_ptr<consensus::RaftConsensus> consensus = replica->shared_consensus();
-      (*output) << Substitute(
+      *output << Substitute(
           // Table name, tablet id, partition
           "<tr><td>$0</td><td>$1</td><td>$2</td>"
           // State, on-disk size, consensus configuration, last status
@@ -332,7 +337,7 @@ bool CompareByMemberType(const RaftPeerPB& a, const RaftPeerPB& b) {
 } // anonymous namespace
 
 string TabletServerPathHandlers::ConsensusStatePBToHtml(const ConsensusStatePB& cstate) const {
-  std::ostringstream html;
+  ostringstream html;
 
   html << "<ul>\n";
   std::vector<RaftPeerPB> sorted_peers;
@@ -361,32 +366,36 @@ string TabletServerPathHandlers::ConsensusStatePBToHtml(const ConsensusStatePB&
 
 namespace {
 
-bool GetTabletID(const Webserver::WebRequest& req, string* id, std::ostringstream* out) {
+bool GetTabletID(const Webserver::WebRequest& req,
+                 string* id,
+                 Webserver::PrerenderedWebResponse* resp) {
   if (!FindCopy(req.parsed_args, "id", id)) {
-    // TODO: webserver should give a way to return a non-200 response code
-    (*out) << "Tablet missing 'id' argument";
+    resp->status_code = HttpStatusCode::BadRequest;
+    *resp->output << "Tablet missing 'id' argument";
     return false;
   }
   return true;
 }
 
-bool GetTabletReplica(TabletServer* tserver, const Webserver::WebRequest& req,
+bool GetTabletReplica(TabletServer* tserver, const Webserver::WebRequest& /*req*/,
                       scoped_refptr<TabletReplica>* replica, const string& tablet_id,
-                      std::ostringstream* out) {
+                      Webserver::PrerenderedWebResponse* resp) {
   if (!tserver->tablet_manager()->LookupTablet(tablet_id, replica)) {
-    (*out) << "Tablet " << EscapeForHtmlToString(tablet_id) << " not found";
+    resp->status_code = HttpStatusCode::NotFound;
+    *resp->output << "Tablet " << EscapeForHtmlToString(tablet_id) << " not found";
     return false;
   }
   return true;
 }
 
 bool TabletBootstrapping(const scoped_refptr<TabletReplica>& replica, const string& tablet_id,
-                         std::ostringstream* out) {
+                         Webserver::PrerenderedWebResponse* resp) {
   if (replica->state() == tablet::BOOTSTRAPPING) {
-    (*out) << "Tablet " << EscapeForHtmlToString(tablet_id) << " is still bootstrapping";
-    return false;
+    resp->status_code = HttpStatusCode::ServiceUnavailable;
+    *resp->output << "Tablet " << EscapeForHtmlToString(tablet_id) << " is still bootstrapping";
+    return true;
   }
-  return true;
+  return false;
 }
 
 // Returns true if the tablet_id was properly specified, the
@@ -394,20 +403,20 @@ bool TabletBootstrapping(const scoped_refptr<TabletReplica>& replica, const stri
 bool LoadTablet(TabletServer* tserver,
                 const Webserver::WebRequest& req,
                 string* tablet_id, scoped_refptr<TabletReplica>* replica,
-                std::ostringstream* out) {
-  if (!GetTabletID(req, tablet_id, out)) return false;
-  if (!GetTabletReplica(tserver, req, replica, *tablet_id, out)) return false;
-  if (!TabletBootstrapping(*replica, *tablet_id, out)) return false;
-  return true;
+                Webserver::PrerenderedWebResponse* resp) {
+  return GetTabletID(req, tablet_id, resp) &&
+      GetTabletReplica(tserver, req, replica, *tablet_id, resp) &&
+      !TabletBootstrapping(*replica, *tablet_id, resp);
 }
 
 } // anonymous namespace
 
 void TabletServerPathHandlers::HandleTabletPage(const Webserver::WebRequest& req,
-                                                std::ostringstream *output) {
+                                                Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   string tablet_id;
   scoped_refptr<TabletReplica> replica;
-  if (!LoadTablet(tserver_, req, &tablet_id, &replica, output)) return;
+  if (!LoadTablet(tserver_, req, &tablet_id, &replica, resp)) return;
 
   string table_name = replica->tablet_metadata()->table_name();
   RaftPeerPB::Role role = RaftPeerPB::UNKNOWN_ROLE;
@@ -454,10 +463,11 @@ void TabletServerPathHandlers::HandleTabletPage(const Webserver::WebRequest& req
 }
 
 void TabletServerPathHandlers::HandleTabletSVGPage(const Webserver::WebRequest& req,
-                                                   std::ostringstream* output) {
+                                                   Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   string id;
   scoped_refptr<TabletReplica> replica;
-  if (!LoadTablet(tserver_, req, &id, &replica, output)) return;
+  if (!LoadTablet(tserver_, req, &id, &replica, resp)) return;
   shared_ptr<Tablet> tablet = replica->shared_tablet();
   if (!tablet) {
     *output << "Tablet " << EscapeForHtmlToString(id) << " not running";
@@ -471,10 +481,11 @@ void TabletServerPathHandlers::HandleTabletSVGPage(const Webserver::WebRequest&
 }
 
 void TabletServerPathHandlers::HandleLogAnchorsPage(const Webserver::WebRequest& req,
-                                                    std::ostringstream* output) {
+                                                    Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   string tablet_id;
   scoped_refptr<TabletReplica> replica;
-  if (!LoadTablet(tserver_, req, &tablet_id, &replica, output)) return;
+  if (!LoadTablet(tserver_, req, &tablet_id, &replica, resp)) return;
 
   *output << "<h1>Log Anchors for Tablet " << EscapeForHtmlToString(tablet_id) << "</h1>"
           << std::endl;
@@ -484,10 +495,11 @@ void TabletServerPathHandlers::HandleLogAnchorsPage(const Webserver::WebRequest&
 }
 
 void TabletServerPathHandlers::HandleConsensusStatusPage(const Webserver::WebRequest& req,
-                                                         std::ostringstream* output) {
+                                                         Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   string id;
   scoped_refptr<TabletReplica> replica;
-  if (!LoadTablet(tserver_, req, &id, &replica, output)) return;
+  if (!LoadTablet(tserver_, req, &id, &replica, resp)) return;
   shared_ptr<consensus::RaftConsensus> consensus = replica->shared_consensus();
   if (!consensus) {
     *output << "Tablet " << EscapeForHtmlToString(id) << " not initialized";
@@ -496,8 +508,9 @@ void TabletServerPathHandlers::HandleConsensusStatusPage(const Webserver::WebReq
   consensus->DumpStatusHtml(*output);
 }
 
-void TabletServerPathHandlers::HandleScansPage(const Webserver::WebRequest& req,
-                                               std::ostringstream* output) {
+void TabletServerPathHandlers::HandleScansPage(const Webserver::WebRequest& /*req*/,
+                                               Webserver::PrerenderedWebResponse* resp) {
+  std::ostringstream* output = resp->output;
   *output << "<h1>Scans</h1>\n";
   *output << "<table class='table table-striped'>\n";
   *output << "<thead><tr><th>Tablet id</th><th>Scanner id</th><th>Total time in-flight</th>"
@@ -596,9 +609,9 @@ string TabletServerPathHandlers::IteratorStatsToHtml(const Schema& projection,
   return html.str();
 }
 
-void TabletServerPathHandlers::HandleDashboardsPage(const Webserver::WebRequest& req,
-                                                    std::ostringstream* output) {
-
+void TabletServerPathHandlers::HandleDashboardsPage(const Webserver::WebRequest& /*req*/,
+                                                    Webserver::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   *output << "<h3>Dashboards</h3>\n";
   *output << "<table class='table table-striped'>\n";
   *output << "  <thead><tr><th>Dashboard</th><th>Description</th></tr></thead>\n";
@@ -622,7 +635,8 @@ string TabletServerPathHandlers::GetDashboardLine(const std::string& link,
 }
 
 void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::WebRequest& req,
-                                                            EasyJson* output) {
+                                                            Webserver::WebResponse* resp) {
+  EasyJson* output = resp->output;
   MaintenanceManager* manager = tserver_->maintenance_manager();
   MaintenanceManagerStatusPB pb;
   manager->GetMaintenanceManagerStatusDump(&pb);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/tserver/tserver-path-handlers.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver-path-handlers.h b/src/kudu/tserver/tserver-path-handlers.h
index 385b24d..8c00a6f 100644
--- a/src/kudu/tserver/tserver-path-handlers.h
+++ b/src/kudu/tserver/tserver-path-handlers.h
@@ -17,7 +17,6 @@
 #ifndef KUDU_TSERVER_TSERVER_PATH_HANDLERS_H
 #define KUDU_TSERVER_TSERVER_PATH_HANDLERS_H
 
-#include <iosfwd>
 #include <string>
 #include <vector>
 
@@ -27,7 +26,6 @@
 
 namespace kudu {
 
-class EasyJson;
 class Schema;
 struct IteratorStats;
 
@@ -52,23 +50,23 @@ class TabletServerPathHandlers {
 
  private:
   void HandleScansPage(const Webserver::WebRequest& req,
-                       std::ostringstream* output);
+                       Webserver::PrerenderedWebResponse* resp);
   void HandleTabletsPage(const Webserver::WebRequest& req,
-                         std::ostringstream* output);
+                         Webserver::PrerenderedWebResponse* resp);
   void HandleTabletPage(const Webserver::WebRequest& req,
-                        std::ostringstream* output);
+                        Webserver::PrerenderedWebResponse* resp);
   void HandleTransactionsPage(const Webserver::WebRequest& req,
-                              std::ostringstream* output);
+                              Webserver::PrerenderedWebResponse* resp);
   void HandleTabletSVGPage(const Webserver::WebRequest& req,
-                           std::ostringstream* output);
+                           Webserver::PrerenderedWebResponse* resp);
   void HandleLogAnchorsPage(const Webserver::WebRequest& req,
-                            std::ostringstream* output);
+                            Webserver::PrerenderedWebResponse* resp);
   void HandleConsensusStatusPage(const Webserver::WebRequest& req,
-                                 std::ostringstream* output);
+                                 Webserver::PrerenderedWebResponse* resp);
   void HandleDashboardsPage(const Webserver::WebRequest& req,
-                            std::ostringstream* output);
+                            Webserver::PrerenderedWebResponse* resp);
   void HandleMaintenanceManagerPage(const Webserver::WebRequest& req,
-                                    EasyJson* output);
+                                    Webserver::WebResponse* resp);
   std::string ConsensusStatePBToHtml(const consensus::ConsensusStatePB& cstate) const;
   std::string ScannerToHtml(const Scanner& scanner) const;
   std::string IteratorStatsToHtml(const Schema& projection,

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/util/thread.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/thread.cc b/src/kudu/util/thread.cc
index c4b651b..c264da1 100644
--- a/src/kudu/util/thread.cc
+++ b/src/kudu/util/thread.cc
@@ -31,8 +31,9 @@
 #include <map>
 #include <memory>
 #include <sstream>
-#include <vector>
+#include <unordered_map>
 #include <utility>
+#include <vector>
 
 #include <boost/bind.hpp>
 #include <boost/smart_ptr/shared_ptr.hpp>
@@ -220,8 +221,9 @@ class ThreadMgr {
   uint64_t ReadThreadsStarted();
   uint64_t ReadThreadsRunning();
 
-  // Webpage callback; prints all threads by category
-  void ThreadPathHandler(const WebCallbackRegistry::WebRequest& args, ostringstream* output);
+  // Webpage callback; prints all threads by category.
+  void ThreadPathHandler(const WebCallbackRegistry::WebRequest& req,
+                         WebCallbackRegistry::PrerenderedWebResponse* resp);
   void PrintThreadCategoryRows(const ThreadCategory& category, ostringstream* output);
 };
 
@@ -357,7 +359,8 @@ void ThreadMgr::PrintThreadCategoryRows(const ThreadCategory& category,
 }
 
 void ThreadMgr::ThreadPathHandler(const WebCallbackRegistry::WebRequest& req,
-    ostringstream* output) {
+                                  WebCallbackRegistry::PrerenderedWebResponse* resp) {
+  ostringstream* output = resp->output;
   MutexLock l(lock_);
   vector<const ThreadCategory*> categories_to_print;
   auto category_name = req.parsed_args.find("group");

http://git-wip-us.apache.org/repos/asf/kudu/blob/1ee49525/src/kudu/util/web_callback_registry.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/web_callback_registry.h b/src/kudu/util/web_callback_registry.h
index 0779a40..600bddd 100644
--- a/src/kudu/util/web_callback_registry.h
+++ b/src/kudu/util/web_callback_registry.h
@@ -18,8 +18,8 @@
 #define KUDU_UTIL_WEB_CALLBACK_REGISTRY_H
 
 #include <iosfwd>
-#include <map>
 #include <string>
+#include <unordered_map>
 
 #include <boost/function.hpp>
 
@@ -27,6 +27,16 @@
 
 namespace kudu {
 
+enum class HttpStatusCode {
+  Ok, // 200
+  BadRequest, // 400
+  NotFound, // 404
+  LengthRequired, // 411
+  RequestEntityTooLarge, // 413
+  InternalServerError, // 500
+  ServiceUnavailable, // 503
+};
+
 // Interface for registering webserver callbacks.
 //
 // To register a webserver callback for /example/path:
@@ -34,10 +44,10 @@ namespace kudu {
 // 1. Define a PathHandlerCallback that accepts an EasyJson
 //    object and fills out its fields with relevant information.
 // 2. Call RegisterPathHandler("/example/path", ...)
-// 3. Create the file $KUDU_HOME/www/example/path.mustache
+// 3. Create the file $KUDU_HOME/www/example/path.mustache.
 class WebCallbackRegistry {
  public:
-  typedef std::map<std::string, std::string> ArgumentMap;
+  typedef std::unordered_map<std::string, std::string> ArgumentMap;
 
   struct WebRequest {
     // The query string, parsed into key/value argument pairs.
@@ -53,13 +63,40 @@ class WebCallbackRegistry {
     std::string post_data;
   };
 
-  // A function that adds members to the JSON object 'output' to be
-  // rendered in an HTML template.
-  typedef boost::function<void (const WebRequest& args, EasyJson* output)>
+  typedef std::unordered_map<std::string, std::string> HttpResponseHeaders;
+
+  // A response to an HTTP request whose body is rendered by template.
+  struct WebResponse {
+    // Determines the status code of the HTTP response.
+    HttpStatusCode status_code;
+
+    // Additional headers added to the HTTP response.
+    HttpResponseHeaders response_headers;
+
+    // A JSON object to be rendered to HTML by a mustache template.
+    EasyJson* output;
+  };
+
+  // A response to an HTTP request.
+  struct PrerenderedWebResponse {
+    // Determines the status code of the HTTP response.
+    HttpStatusCode status_code;
+
+    // Additional headers added to the HTTP response.
+    HttpResponseHeaders response_headers;
+
+    // The fully-rendered HTML response body.
+    std::ostringstream* output;
+  };
+
+  // A function that handles an HTTP request where the response body will be rendered
+  // with a mustache template from the JSON object held by 'resp'.
+  typedef boost::function<void (const WebRequest& args, WebResponse* resp)>
       PathHandlerCallback;
 
-  // A function that streams fully rendered HTML to 'output'.
-  typedef boost::function<void (const WebRequest& args, std::ostringstream* output)>
+  // A function that handles an HTTP request, where the response body is the contents
+  // of the 'output' member of 'resp'.
+  typedef boost::function<void (const WebRequest& args, PrerenderedWebResponse* resp)>
       PrerenderedPathHandlerCallback;
 
   virtual ~WebCallbackRegistry() {}


[2/2] kudu git commit: KUDU-2167: fix C++ client crash due to bad assumption regarding scan data

Posted by ad...@apache.org.
KUDU-2167: fix C++ client crash due to bad assumption regarding scan data

The new unit test triggered the crash reliably, though it's probably not the
only way (or perhaps not even the best way) to trigger it.

I also modified the tserver to always populate the scan data fields so that
older clients are also protected. This necessitated an additional change to
how a scanner's "data in open" was computed: the existence of the 'data'
field in the scan response isn't enough. The Java client was already doing
this, so it seems like a reasonable change to make on the C++ side.

Change-Id: If1a8a4e22082cf39710b9f00894f644a0b34234e
Reviewed-on: http://gerrit.cloudera.org:8080/8204
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Kudu Jenkins


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/858bf73b
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/858bf73b
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/858bf73b

Branch: refs/heads/master
Commit: 858bf73b380d10f16e7aca04a780a0adae705e9a
Parents: 1ee4952
Author: Adar Dembo <ad...@cloudera.com>
Authored: Tue Oct 3 18:34:18 2017 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Fri Oct 6 04:30:50 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc      | 33 ++++++++++++++++++
 src/kudu/client/scanner-internal.cc | 26 ++++++++------
 src/kudu/tserver/tablet_service.cc  | 59 ++++++++++++--------------------
 3 files changed, 71 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/858bf73b/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index c888513..450449e 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -5307,5 +5307,38 @@ TEST_F(ClientTest, TestVerboseLevelByEnvVar) {
   ASSERT_EQ(0, FLAGS_v);
 }
 
+// Regression test for KUDU-2167: older versions of Kudu could return a scan
+// response without a 'data' field, crashing the client.
+TEST_F(ClientTest, TestSubsequentScanRequestReturnsNoData) {
+  // Insert some rows.
+  NO_FATALS(InsertTestRows(client_table_.get(), FLAGS_test_scan_num_rows));
+
+  // Set up a table scan.
+  KuduScanner scanner(client_table_.get());
+  ASSERT_OK(scanner.SetProjectedColumns({ "key" }));
+
+  // Ensure that the new scan RPC does not return the data.
+  //
+  // It's OK to leave the scanner configured like this; after the new scan RPC
+  // the server will still return at least one block of data per RPC.
+  ASSERT_OK(scanner.SetBatchSizeBytes(0));
+
+  // This scan should not match any of the inserted rows.
+  unique_ptr<KuduPartialRow> row(client_table_->schema().NewRow());
+  ASSERT_OK(row->SetInt32("key", -1));
+  ASSERT_OK(scanner.AddExclusiveUpperBound(*row));
+
+  // Perform the scan.
+  ASSERT_OK(scanner.Open());
+  ASSERT_TRUE(scanner.HasMoreRows());
+  int count = 0;
+  KuduScanBatch batch;
+  while (scanner.HasMoreRows()) {
+    ASSERT_OK(scanner.NextBatch(&batch));
+    count += batch.NumRows();
+  }
+  ASSERT_EQ(0, count);
+}
+
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/858bf73b/src/kudu/client/scanner-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/scanner-internal.cc b/src/kudu/client/scanner-internal.cc
index f61a5a3..92948c9 100644
--- a/src/kudu/client/scanner-internal.cc
+++ b/src/kudu/client/scanner-internal.cc
@@ -444,7 +444,7 @@ Status KuduScanner::Data::OpenTablet(const string& partition_key,
   partition_pruner_.RemovePartitionKeyRange(remote_->partition().partition_key_end());
 
   next_req_.clear_new_scan_request();
-  data_in_open_ = last_response_.has_data();
+  data_in_open_ = last_response_.has_data() && last_response_.data().num_rows() > 0;
   if (last_response_.has_more_results()) {
     next_req_.set_scanner_id(last_response_.scanner_id());
     VLOG(2) << "Opened tablet " << remote_->tablet_id()
@@ -550,8 +550,16 @@ Status KuduScanBatch::Data::Reset(RpcController* controller,
   CHECK(controller->finished());
   controller_.Swap(controller);
   projection_ = projection;
+  projected_row_size_ = CalculateProjectedRowSize(*projection_);
   client_projection_ = client_projection;
   row_format_flags_ = row_format_flags;
+  if (!resp_data) {
+    // No new data; just clear out the old stuff.
+    resp_data_.Clear();
+    return Status::OK();
+  }
+
+  // There's new data. Swap it in and process it.
   resp_data_.Swap(resp_data.get());
 
   // First, rewrite the relative addresses into absolute ones.
@@ -561,16 +569,16 @@ Status KuduScanBatch::Data::Reset(RpcController* controller,
 
   Status s = controller_.GetInboundSidecar(resp_data_.rows_sidecar(), &direct_data_);
   if (!s.ok()) {
-    return Status::Corruption("Server sent invalid response: row data "
-        "sidecar index corrupt", s.ToString());
+    return Status::Corruption("Server sent invalid response: "
+        "row data sidecar index corrupt", s.ToString());
   }
 
   if (resp_data_.has_indirect_data_sidecar()) {
     Status s = controller_.GetInboundSidecar(resp_data_.indirect_data_sidecar(),
-                                      &indirect_data_);
+                                             &indirect_data_);
     if (!s.ok()) {
-      return Status::Corruption("Server sent invalid response: indirect data "
-                                "sidecar index corrupt", s.ToString());
+      return Status::Corruption("Server sent invalid response: "
+          "indirect data sidecar index corrupt", s.ToString());
     }
   }
 
@@ -579,10 +587,8 @@ Status KuduScanBatch::Data::Reset(RpcController* controller,
     pad_unixtime_micros_to_16_bytes = true;
   }
 
-  RETURN_NOT_OK(RewriteRowBlockPointers(*projection_, resp_data_, indirect_data_, &direct_data_,
-                                        pad_unixtime_micros_to_16_bytes));
-  projected_row_size_ = CalculateProjectedRowSize(*projection_);
-  return Status::OK();
+  return RewriteRowBlockPointers(*projection_, resp_data_, indirect_data_, &direct_data_,
+                                 pad_unixtime_micros_to_16_bytes);
 }
 
 void KuduScanBatch::Data::ExtractRows(vector<KuduScanBatch::RowPtr>* rows) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/858bf73b/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index be86cb3..bb4bc59 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -419,9 +419,6 @@ class ScanResultCollector {
   virtual void HandleRowBlock(const Schema* client_projection_schema,
                               const RowBlock& row_block) = 0;
 
-  // Returns number of times HandleRowBlock() was called.
-  virtual int BlocksProcessed() const = 0;
-
   // Returns number of bytes which will be returned in the response.
   virtual int64_t ResponseSize() const = 0;
 
@@ -478,21 +475,17 @@ class ScanResultCopier : public ScanResultCollector {
       : rowblock_pb_(DCHECK_NOTNULL(rowblock_pb)),
         rows_data_(DCHECK_NOTNULL(rows_data)),
         indirect_data_(DCHECK_NOTNULL(indirect_data)),
-        blocks_processed_(0),
         num_rows_returned_(0),
         pad_unixtime_micros_to_16_bytes_(false) {}
 
   void HandleRowBlock(const Schema* client_projection_schema,
                               const RowBlock& row_block) override {
-    blocks_processed_++;
     num_rows_returned_ += row_block.selection_vector()->CountSelected();
     SerializeRowBlock(row_block, rowblock_pb_, client_projection_schema,
                       rows_data_, indirect_data_, pad_unixtime_micros_to_16_bytes_);
     SetLastRow(row_block, &last_primary_key_);
   }
 
-  int BlocksProcessed() const override { return blocks_processed_; }
-
   // Returns number of bytes buffered to return.
   int64_t ResponseSize() const override {
     return rows_data_->size() + indirect_data_->size();
@@ -516,7 +509,6 @@ class ScanResultCopier : public ScanResultCollector {
   RowwiseRowBlockPB* const rowblock_pb_;
   faststring* const rows_data_;
   faststring* const indirect_data_;
-  int blocks_processed_;
   int64_t num_rows_returned_;
   faststring last_primary_key_;
   bool pad_unixtime_micros_to_16_bytes_;
@@ -530,13 +522,11 @@ class ScanResultChecksummer : public ScanResultCollector {
   ScanResultChecksummer()
       : crc_(crc::GetCrc32cInstance()),
         agg_checksum_(0),
-        blocks_processed_(0),
         rows_checksummed_(0) {
   }
 
   virtual void HandleRowBlock(const Schema* client_projection_schema,
                               const RowBlock& row_block) OVERRIDE {
-    blocks_processed_++;
     if (!client_projection_schema) {
       client_projection_schema = &row_block.schema();
     }
@@ -552,8 +542,6 @@ class ScanResultChecksummer : public ScanResultCollector {
     SetLastRow(row_block, &encoded_last_row_);
   }
 
-  virtual int BlocksProcessed() const OVERRIDE { return blocks_processed_; }
-
   // Returns a constant -- we only return checksum based on a time budget.
   virtual int64_t ResponseSize() const OVERRIDE { return sizeof(agg_checksum_); }
 
@@ -602,7 +590,6 @@ class ScanResultChecksummer : public ScanResultCollector {
   faststring tmp_buf_;
   crc::Crc* const crc_;
   uint64_t agg_checksum_;
-  int blocks_processed_;
   int64_t rows_checksummed_;
   faststring encoded_last_row_;
 
@@ -1310,31 +1297,29 @@ void TabletServiceImpl::Scan(const ScanRequestPB* req,
   }
   resp->set_has_more_results(has_more_results);
 
-  DVLOG(2) << "Blocks processed: " << collector.BlocksProcessed();
-  if (collector.BlocksProcessed() > 0) {
-    resp->mutable_data()->CopyFrom(data);
-
-    // Add sidecar data to context and record the returned indices.
-    int rows_idx;
-    CHECK_OK(context->AddOutboundSidecar(RpcSidecar::FromFaststring((std::move(rows_data))),
-            &rows_idx));
-    resp->mutable_data()->set_rows_sidecar(rows_idx);
-
-    // Add indirect data as a sidecar, if applicable.
-    if (indirect_data->size() > 0) {
-      int indirect_idx;
-      CHECK_OK(context->AddOutboundSidecar(RpcSidecar::FromFaststring(
-          std::move(indirect_data)), &indirect_idx));
-      resp->mutable_data()->set_indirect_data_sidecar(indirect_idx);
-    }
+  resp->mutable_data()->CopyFrom(data);
 
-    // Set the last row found by the collector.
-    // We could have an empty batch if all the remaining rows are filtered by the predicate,
-    // in which case do not set the last row.
-    const faststring& last = collector.last_primary_key();
-    if (last.length() > 0) {
-      resp->set_last_primary_key(last.ToString());
-    }
+  // Add sidecar data to context and record the returned indices.
+  int rows_idx;
+  CHECK_OK(context->AddOutboundSidecar(
+      RpcSidecar::FromFaststring((std::move(rows_data))), &rows_idx));
+  resp->mutable_data()->set_rows_sidecar(rows_idx);
+
+  // Add indirect data as a sidecar, if applicable.
+  if (indirect_data->size() > 0) {
+    int indirect_idx;
+    CHECK_OK(context->AddOutboundSidecar(
+        RpcSidecar::FromFaststring(std::move(indirect_data)), &indirect_idx));
+    resp->mutable_data()->set_indirect_data_sidecar(indirect_idx);
+  }
+
+  // Set the last row found by the collector.
+  //
+  // We could have an empty batch if all the remaining rows are filtered by the
+  // predicate, in which case do not set the last row.
+  const faststring& last = collector.last_primary_key();
+  if (last.length() > 0) {
+    resp->set_last_primary_key(last.ToString());
   }
   resp->set_propagated_timestamp(server_->clock()->Now().ToUint64());
   SetResourceMetrics(resp->mutable_resource_metrics(), context);