You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2017/09/21 20:15:43 UTC

[1/3] kudu git commit: Add mustache template for /table

Repository: kudu
Updated Branches:
  refs/heads/master edd41cb40 -> 315a464c1


Add mustache template for /table

As a run-up to improving the web ui experience when there are multiple
masters, this patch switches the /table page to be based on a mustache
template.

Change-Id: I9fe78e0701c1cd965650b101244212b5988f11fb
Reviewed-on: http://gerrit.cloudera.org:8080/8067
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/e66fb10d
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/e66fb10d
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/e66fb10d

Branch: refs/heads/master
Commit: e66fb10da44dfd1b5bd5611b9714fe594c0df7d0
Parents: edd41cb
Author: Will Berkeley <wd...@apache.org>
Authored: Tue Sep 12 22:48:28 2017 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Thu Sep 21 19:24:53 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/master-path-handlers.cc | 244 +++++++++++++--------------
 src/kudu/master/master-path-handlers.h  |  18 +-
 src/kudu/server/webui_util.cc           |  70 ++++----
 src/kudu/server/webui_util.h            |  20 ++-
 www/table.mustache                      | 143 ++++++++++++++++
 www/tables.mustache                     |   2 +-
 6 files changed, 315 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e66fb10d/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 a497f9e..ebd42f2 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -24,6 +24,7 @@
 #include <memory>
 #include <sstream>
 #include <string>
+#include <type_traits>
 #include <utility>
 #include <vector>
 
@@ -170,6 +171,15 @@ void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req,
 
 namespace {
 
+
+// Holds info about a peer for use in the tablet detail table.
+struct TabletDetailPeerInfo {
+  string text;
+  string target;
+  string role;
+  bool is_leader;
+};
+
 int RoleToSortIndex(RaftPeerPB::Role r) {
   switch (r) {
     case RaftPeerPB::LEADER: return 0;
@@ -177,8 +187,8 @@ int RoleToSortIndex(RaftPeerPB::Role r) {
   }
 }
 
-bool CompareByRole(const pair<string, RaftPeerPB::Role>& a,
-                   const pair<string, RaftPeerPB::Role>& b) {
+bool CompareByRole(const pair<TabletDetailPeerInfo, RaftPeerPB::Role>& a,
+                   const pair<TabletDetailPeerInfo, RaftPeerPB::Role>& b) {
   return RoleToSortIndex(a.second) < RoleToSortIndex(b.second);
 }
 
@@ -186,116 +196,102 @@ bool CompareByRole(const pair<string, RaftPeerPB::Role>& a,
 
 
 void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
-                                         ostringstream* output) {
+                                         EasyJson* 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
-    *output << "Missing 'id' argument";
+    // TODO(wdb): Webserver should give a way to return a non-200 response code.
+    (*output)["error"] = "Missing 'id' argument";
     return;
   }
 
   CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
   if (!l.first_failed_status().ok()) {
-    *output << "Master is not ready: " << l.first_failed_status().ToString();
+    (*output)["error"] = Substitute("Master is not ready: ", l.first_failed_status().ToString());
     return;
   }
 
   scoped_refptr<TableInfo> table;
   Status s = master_->catalog_manager()->GetTableInfo(table_id, &table);
   if (!s.ok()) {
-    *output << "Master is not ready: " << s.ToString();
+    (*output)["error"] = Substitute("Master is not ready: ", s.ToString());
     return;
   }
 
   if (!table) {
-    *output << "Table not found";
+    (*output)["error"] = "Table not found";
     return;
   }
 
   Schema schema;
   PartitionSchema partition_schema;
-  string table_name;
-  vector<scoped_refptr<TabletInfo> > tablets;
+  vector<scoped_refptr<TabletInfo>> tablets;
   {
     TableMetadataLock l(table.get(), TableMetadataLock::READ);
-    table_name = l.data().name();
-    *output << "<h1>Table: " << EscapeForHtmlToString(table_name)
-            << " (" << EscapeForHtmlToString(table_id) << ")</h1>\n";
-
-    *output << "<table class='table table-striped'>\n";
-    *output << "  <tr><td>Version:</td><td>" << l.data().pb.version() << "</td></tr>\n";
+    (*output)["name"] = l.data().name();
+    (*output)["id"] = table_id;
+    (*output)["version"] = l.data().pb.version();
 
     string state = SysTablesEntryPB_State_Name(l.data().pb.state());
     Capitalize(&state);
-    *output << "  <tr><td>State:</td><td>"
-            << state
-            << EscapeForHtmlToString(l.data().pb.state_msg())
-            << "</td></tr>\n";
-    *output << "</table>\n";
+    (*output)["state"] = state;
+    string state_msg = l.data().pb.state_msg();
+    if (!state_msg.empty()) {
+      (*output)["state_msg"] = state_msg;
+    }
 
-    SchemaFromPB(l.data().pb.schema(), &schema);
+    s = SchemaFromPB(l.data().pb.schema(), &schema);
+    if (!s.ok()) {
+      (*output)["error"] = Substitute("Unable to decode schema: ", s.ToString());
+      return;
+    }
     s = PartitionSchema::FromPB(l.data().pb.partition_schema(), schema, &partition_schema);
     if (!s.ok()) {
-      *output << "Unable to decode partition schema: " << s.ToString();
+      (*output)["error"] =
+          Substitute("Unable to decode partition schema: ", s.ToString());
       return;
     }
     table->GetAllTablets(&tablets);
   }
 
-  *output << "<h3>Schema</h3>";
-  HtmlOutputSchemaTable(schema, output);
+  SchemaToJson(schema, output);
 
-  // Visit (& lock) each tablet once to build the partition schema, tablets summary,
-  // and tablets detail tables all at once.
+  // We have to collate partition schema and tablet information in order to set
+  // up the partition schema, tablet summary, and tablet detail tables.
   std::vector<string> range_partitions;
   map<string, int> summary_states;
-  std::ostringstream detail_output;
-
-  detail_output << "<h4>Detail</h4>\n";
-  detail_output << "<a href='#detail' data-toggle='collapse'>(toggle)</a>";
-  detail_output << "<div id='detail' class='collapse'>\n";
-  detail_output << "<table class='table table-striped table-hover'>\n";
-  detail_output << "  <thead><tr><th>Tablet ID</th>"
-                 << partition_schema.PartitionTableHeader(schema)
-                 << "<th>State</th><th>Message</th><th>Peers</th></tr></thead>\n";
-  detail_output << "<tbody>\n";
+  (*output)["detail_partition_schema_header"] = partition_schema.PartitionTableHeader(schema);
+  EasyJson tablets_detail_json = output->Set("tablets_detail", EasyJson::kArray);
   for (const scoped_refptr<TabletInfo>& tablet : tablets) {
-    vector<pair<string, RaftPeerPB::Role>> sorted_replicas;
+    vector<pair<TabletDetailPeerInfo, RaftPeerPB::Role>> sorted_replicas;
     TabletMetadataLock l(tablet.get(), TabletMetadataLock::READ);
 
+    // Count states for tablet summary.
     summary_states[SysTabletsEntryPB_State_Name(l.data().pb.state())]++;
+
+    // Collect details about each tablet replica.
     if (l.data().pb.has_consensus_state()) {
       const ConsensusStatePB& cstate = l.data().pb.consensus_state();
       for (const auto& peer : cstate.committed_config().peers()) {
-        RaftPeerPB::Role role = GetConsensusRole(peer.permanent_uuid(), cstate);
-        string html;
-        string location_html;
+        TabletDetailPeerInfo peer_info;
         shared_ptr<TSDescriptor> ts_desc;
         if (master_->ts_manager()->LookupTSByUUID(peer.permanent_uuid(), &ts_desc)) {
-          location_html = TSDescriptorToHtml(*ts_desc.get(), tablet->tablet_id());
-        } else {
-          location_html = EscapeForHtmlToString(peer.permanent_uuid());
-        }
-        if (role == RaftPeerPB::LEADER) {
-          html = Substitute("  <li><b>LEADER: $0</b></li>\n", location_html);
+          auto link_pair = TSDescToLinkPair(*ts_desc.get(), tablet->tablet_id());
+          peer_info.text = std::move(link_pair.first);
+          peer_info.target = std::move(link_pair.second);
         } else {
-          html = Substitute("  <li>$0: $1</li>\n",
-                            RaftPeerPB_Role_Name(role), location_html);
+          peer_info.text = peer.permanent_uuid();
         }
-        sorted_replicas.emplace_back(html, role);
+        RaftPeerPB::Role role = GetConsensusRole(peer.permanent_uuid(), cstate);
+        peer_info.role = RaftPeerPB_Role_Name(role);
+        peer_info.is_leader = role == RaftPeerPB::LEADER;
+        sorted_replicas.emplace_back(std::make_pair(peer_info, role));
       }
     }
     std::sort(sorted_replicas.begin(), sorted_replicas.end(), &CompareByRole);
 
-    // Generate the RaftConfig table cell.
-    ostringstream raft_config_html;
-    raft_config_html << "<ul>\n";
-    for (const auto& e : sorted_replicas) {
-      raft_config_html << e.first;
-    }
-    raft_config_html << "</ul>\n";
-
+    // Generate a readable description of the partition of each tablet, used
+    // both for each tablet's details and the readable range partition schema.
     Partition partition;
     Partition::FromPB(l.data().pb.partition(), &partition);
 
@@ -304,75 +300,52 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
     if (std::all_of(partition.hash_buckets().begin(),
                     partition.hash_buckets().end(),
                     [] (const int32_t& bucket) { return bucket == 0; })) {
-        range_partitions.emplace_back(
-            partition_schema.RangePartitionDebugString(partition.range_key_start(),
-                                                       partition.range_key_end(),
-                                                       schema));
+      range_partitions.emplace_back(
+          partition_schema.RangePartitionDebugString(partition.range_key_start(),
+                                                     partition.range_key_end(),
+                                                     schema));
     }
 
+    // Combine the tablet details and partition info for each tablet.
+    EasyJson tablet_detail_json = tablets_detail_json.PushBack(EasyJson::kObject);
+    tablet_detail_json["id"] = tablet->tablet_id();
+    tablet_detail_json["partition_cols"] = partition_schema.PartitionTableEntry(schema, partition);
     string state = SysTabletsEntryPB_State_Name(l.data().pb.state());
     Capitalize(&state);
-    detail_output << Substitute(
-        "<tr><th>$0</th>$1<td>$2</td><td>$3</td><td>$4</td></tr>\n",
-        tablet->tablet_id(),
-        partition_schema.PartitionTableEntry(schema, partition),
-        state,
-        EscapeForHtmlToString(l.data().pb.state_msg()),
-        raft_config_html.str());
+    tablet_detail_json["state"] = state;
+    tablet_detail_json["state_msg"] = l.data().pb.state_msg();
+    EasyJson peers_json = tablet_detail_json.Set("peers", EasyJson::kArray);
+    for (const auto& e : sorted_replicas) {
+      EasyJson peer_json = peers_json.PushBack(EasyJson::kObject);
+      peer_json["text"] = e.first.text;
+      if (!e.first.target.empty()) {
+        peer_json["target"] = e.first.target;
+      }
+      peer_json["role"] = e.first.role;
+      peer_json["is_leader"] = e.first.is_leader;
+    }
   }
-  detail_output << "</tbody></table></div>\n";
 
-  // Write out the partition schema and range bound information...
-  *output << "<h3>Partition Schema</h3>";
-  *output << "<pre>";
-  *output << EscapeForHtmlToString(partition_schema.DisplayString(schema, range_partitions));
-  *output << "</pre>";
+  (*output)["partition_schema"] = partition_schema.DisplayString(schema, range_partitions);
 
-  // ...then the summary table...
-  *output << "<h3>Tablets</h3>";
-  *output << "<h4>Summary</h4>\n";
-  *output << "<table class='table table-striped'>\n";
-  *output << "<thead><tr><th>State</th><th>Count</th><th>Percentage</th></tr></thead>";
-  *output << "<tbody>\n";
+  EasyJson summary_json = output->Set("tablets_summary", EasyJson::kArray);
   for (const auto& entry : summary_states) {
-    double percentage = tablets.size() == 0 ? 0.0 : (100.0 * entry.second) / tablets.size();
-    *output << Substitute("<tr><td>$0</td><td>$1</td><td>$2</td></tr>\n",
-                                 entry.first, entry.second, StringPrintf("%.2f", percentage));
+    EasyJson state_json = summary_json.PushBack(EasyJson::kObject);
+    state_json["state"] = entry.first;
+    state_json["count"] = entry.second;
+    double percentage = (100.0 * entry.second) / tablets.size();
+    state_json["percentage"] = tablets.empty() ? "0.0" : StringPrintf("%.2f", percentage);
   }
-  *output << "</tbody></table>\n";
 
-  // ...and finally the tablet detail table.
-  *output << detail_output.str();
+  // Used to make the Impala CREATE TABLE statement.
+  (*output)["master_addresses"] = MasterAddrsToCsv();
 
-  *output << "<h3>Impala CREATE TABLE statement</h3>\n";
-  string master_addresses;
-  if (master_->opts().IsDistributed()) {
-    vector<string> all_addresses;
-    all_addresses.reserve(master_->opts().master_addresses.size());
-    for (const HostPort& hp : master_->opts().master_addresses) {
-      all_addresses.push_back(hp.ToString());
-    }
-    master_addresses = JoinElements(all_addresses, ",");
-  } else {
-    Sockaddr addr = master_->first_rpc_address();
-    HostPort hp;
-    Status s = HostPortFromSockaddrReplaceWildcard(addr, &hp);
-    if (s.ok()) {
-      master_addresses = hp.ToString();
-    } else {
-      LOG(WARNING) << "Unable to determine proper local hostname: " << s.ToString();
-      master_addresses = addr.ToString();
-    }
-  }
-  HtmlOutputImpalaSchema(table_name, schema, master_addresses, output);
-
-  *output << "<h3>Tasks</h3>";
-  std::vector<scoped_refptr<MonitoredTask> > task_list;
+  std::vector<scoped_refptr<MonitoredTask>> task_list;
   table->GetTaskList(&task_list);
-  HtmlOutputTaskList(task_list, output);
+  TaskListToJson(task_list, output);
 }
 
-void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& req,
+void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& /*req*/,
                                        ostringstream* output) {
   vector<ServerEntryPB> masters;
   Status s = master_->ListMasters(&masters);
@@ -599,7 +572,7 @@ Status MasterPathHandlers::Register(Webserver* server) {
       "/tables", "Tables",
       boost::bind(&MasterPathHandlers::HandleCatalogManager, this, _1, _2),
       is_styled, is_on_nav_bar);
-  server->RegisterPrerenderedPathHandler(
+  server->RegisterPathHandler(
       "/table", "",
       boost::bind(&MasterPathHandlers::HandleTablePage, this, _1, _2),
       is_styled, false);
@@ -614,22 +587,20 @@ Status MasterPathHandlers::Register(Webserver* server) {
   return Status::OK();
 }
 
-string MasterPathHandlers::TSDescriptorToHtml(const TSDescriptor& desc,
-                                              const std::string& tablet_id) const {
+pair<string, string> MasterPathHandlers::TSDescToLinkPair(const TSDescriptor& desc,
+                                                          const string& tablet_id) const {
   ServerRegistrationPB reg;
   desc.GetRegistration(&reg);
-
-  if (reg.http_addresses().size() > 0) {
-    return Substitute("<a href=\"$0://$1:$2/tablet?id=$3\">$4:$5</a>",
-                      reg.https_enabled() ? "https" : "http",
-                      reg.http_addresses(0).host(),
-                      reg.http_addresses(0).port(),
-                      EscapeForHtmlToString(tablet_id),
-                      EscapeForHtmlToString(reg.http_addresses(0).host()),
-                      reg.http_addresses(0).port());
-  } else {
-    return EscapeForHtmlToString(desc.permanent_uuid());
+  if (reg.http_addresses().empty()) {
+    return std::make_pair(desc.permanent_uuid(), "");
   }
+  string text = Substitute("$0:$1", reg.http_addresses(0).host(), reg.http_addresses(0).port());
+  string target = Substitute("$0://$1:$2/tablet?id=$3",
+                             reg.https_enabled() ? "https" : "http",
+                             reg.http_addresses(0).host(),
+                             reg.http_addresses(0).port(),
+                             tablet_id);
+  return std::make_pair(std::move(text), std::move(target));
 }
 
 string MasterPathHandlers::RegistrationToHtml(
@@ -645,5 +616,24 @@ string MasterPathHandlers::RegistrationToHtml(
   return link_html;
 }
 
+string MasterPathHandlers::MasterAddrsToCsv() const {
+  if (master_->opts().IsDistributed()) {
+    vector<string> all_addresses;
+    all_addresses.reserve(master_->opts().master_addresses.size());
+    for (const HostPort& hp : master_->opts().master_addresses) {
+      all_addresses.push_back(hp.ToString());
+    }
+    return JoinElements(all_addresses, ",");
+  }
+  Sockaddr addr = master_->first_rpc_address();
+  HostPort hp;
+  Status s = HostPortFromSockaddrReplaceWildcard(addr, &hp);
+  if (s.ok()) {
+    return hp.ToString();
+  }
+  LOG(WARNING) << "Unable to determine proper local hostname: " << s.ToString();
+  return addr.ToString();
+}
+
 } // namespace master
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e66fb10d/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 57ac1b0..9bd05c7 100644
--- a/src/kudu/master/master-path-handlers.h
+++ b/src/kudu/master/master-path-handlers.h
@@ -19,6 +19,7 @@
 
 #include <iosfwd>
 #include <string>
+#include <utility>
 
 #include "kudu/gutil/macros.h"
 #include "kudu/server/webserver.h"
@@ -50,16 +51,18 @@ class MasterPathHandlers {
   void HandleCatalogManager(const Webserver::WebRequest& req,
                             EasyJson* output);
   void HandleTablePage(const Webserver::WebRequest& req,
-                       std::ostringstream *output);
+                       EasyJson* output);
   void HandleMasters(const Webserver::WebRequest& req,
                      std::ostringstream* output);
   void HandleDumpEntities(const Webserver::WebRequest& req,
                           std::ostringstream* output);
 
-  // Convert the specified TSDescriptor to HTML, adding a link to the
-  // tablet server's own webserver if specified in 'desc'.
-  std::string TSDescriptorToHtml(const TSDescriptor& desc,
-                                 const std::string& tablet_id) const;
+  // 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.
+  // - target is a url to to the tablet page for the tablet on the tserver's webui,
+  //   or an empty string if no http address is available for the tserver.
+  std::pair<std::string, std::string> TSDescToLinkPair(const TSDescriptor& desc,
+                                                       const std::string& tablet_id) const;
 
   // Convert the specified server registration to HTML, adding a link
   // to the server's own web server (if specified in 'reg') with
@@ -67,12 +70,13 @@ class MasterPathHandlers {
   std::string RegistrationToHtml(const ServerRegistrationPB& reg,
                                  const std::string& link_text) const;
 
+  // Return a CSV of master addresses suitable for display.
+  std::string MasterAddrsToCsv() const;
+
   Master* master_;
   DISALLOW_COPY_AND_ASSIGN(MasterPathHandlers);
 };
 
-void HandleTabletServersPage(const Webserver::WebRequest& req, std::ostringstream* output);
-
 } // namespace master
 } // namespace kudu
 #endif /* KUDU_MASTER_MASTER_PATH_HANDLERS_H */

http://git-wip-us.apache.org/repos/asf/kudu/blob/e66fb10d/src/kudu/server/webui_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webui_util.cc b/src/kudu/server/webui_util.cc
index 9b61b70..cd73042 100644
--- a/src/kudu/server/webui_util.cc
+++ b/src/kudu/server/webui_util.cc
@@ -27,6 +27,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/server/monitored_task.h"
 #include "kudu/util/compression/compression.pb.h"
+#include "kudu/util/easy_json.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/url-coding.h"
 
@@ -35,6 +36,25 @@ using strings::Substitute;
 
 namespace kudu {
 
+void SchemaToJson(const Schema& schema, EasyJson* output) {
+  EasyJson schema_json = output->Set("columns", EasyJson::kArray);
+  for (int i = 0; i < schema.num_columns(); i++) {
+    const ColumnSchema& col = schema.column(i);
+    EasyJson col_json = schema_json.PushBack(EasyJson::kObject);
+    col_json["name"] = col.name();
+    col_json["is_key"] = schema.is_key_column(i);
+    col_json["id"] = Substitute("$0", schema.column_id(i));
+    col_json["type"] = col.TypeToString();
+    const ColumnStorageAttributes& attrs = col.attributes();
+    col_json["encoding"] = EncodingType_Name(attrs.encoding);
+    col_json["compression"] = CompressionType_Name(attrs.compression);
+    col_json["read_default"] = col.has_read_default() ?
+                                   col.Stringify(col.read_default_value()) : "-";
+    col_json["write_default"] = col.has_write_default() ?
+                                    col.Stringify(col.write_default_value()) : "-";
+  }
+}
+
 void HtmlOutputSchemaTable(const Schema& schema,
                            std::ostringstream* output) {
   *output << "<table class='table table-striped'>\n";
@@ -47,7 +67,7 @@ void HtmlOutputSchemaTable(const Schema& schema,
   for (int i = 0; i < schema.num_columns(); i++) {
     const ColumnSchema& col = schema.column(i);
     const string& html_escaped_col_name = EscapeForHtmlToString(col.name());
-    const string& col_name = schema.is_key_column(col.name()) ?
+    const string& col_name = schema.is_key_column(i) ?
                                  Substitute("<u>$0</u>", html_escaped_col_name) :
                                  html_escaped_col_name;
     string read_default = "-";
@@ -74,48 +94,28 @@ void HtmlOutputSchemaTable(const Schema& schema,
   *output << "</tbody></table>\n";
 }
 
-void HtmlOutputImpalaSchema(const std::string& table_name,
-                            const Schema& schema,
-                            const string& master_addresses,
-                            std::ostringstream* output) {
-  *output << "<pre><code>\n";
-
-  // Escape table and column names with ` to avoid conflicts with Impala reserved words.
-  *output << "CREATE EXTERNAL TABLE " << EscapeForHtmlToString("`" + table_name + "`")
-          << " STORED AS KUDU\n";
-  *output << "TBLPROPERTIES(\n";
-  *output << "  'kudu.table_name' = '";
-  *output << EscapeForHtmlToString(table_name) << "',\n";
-  *output << "  'kudu.master_addresses' = '";
-  *output << EscapeForHtmlToString(master_addresses) << "'";
-  *output << ");\n";
-  *output << "</code></pre>\n";
-}
-
-void HtmlOutputTaskList(const std::vector<scoped_refptr<MonitoredTask> >& tasks,
-                        std::ostringstream* output) {
-  *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Task Name</th><th>State</th><th>Time</th><th>Description</th></tr>\n";
+void TaskListToJson(const std::vector<scoped_refptr<MonitoredTask> >& tasks, EasyJson* output) {
+  EasyJson tasks_json = output->Set("tasks", EasyJson::kArray);
   for (const scoped_refptr<MonitoredTask>& task : tasks) {
-    string state;
+    EasyJson task_json = tasks_json.PushBack(EasyJson::kObject);
+    task_json["name"] = task->type_name();
     switch (task->state()) {
       case MonitoredTask::kStatePreparing:
-        state = "Preparing";
+        task_json["state"] = "Preparing";
         break;
       case MonitoredTask::kStateRunning:
-        state = "Running";
+        task_json["state"] = "Running";
         break;
       case MonitoredTask::kStateComplete:
-        state = "Complete";
+        task_json["state"] = "Complete";
         break;
       case MonitoredTask::kStateFailed:
-        state = "Failed";
+        task_json["state"] = "Failed";
         break;
       case MonitoredTask::kStateAborted:
-        state = "Aborted";
+        task_json["state"] = "Aborted";
         break;
     }
-
     double running_secs = 0;
     if (task->completion_timestamp().Initialized()) {
       running_secs =
@@ -123,14 +123,8 @@ void HtmlOutputTaskList(const std::vector<scoped_refptr<MonitoredTask> >& tasks,
     } else if (task->start_timestamp().Initialized()) {
       running_secs = (MonoTime::Now() - task->start_timestamp()).ToSeconds();
     }
-
-    *output << Substitute(
-        "<tr><th>$0</th><td>$1</td><td>$2</td><td>$3</td></tr>\n",
-        EscapeForHtmlToString(task->type_name()),
-        EscapeForHtmlToString(state),
-        EscapeForHtmlToString(HumanReadableElapsedTime::ToShortString(running_secs)),
-        EscapeForHtmlToString(task->description()));
+    task_json["time"] = HumanReadableElapsedTime::ToShortString(running_secs);
+    task_json["description"] = task->description();
   }
-  *output << "</table>\n";
 }
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e66fb10d/src/kudu/server/webui_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/webui_util.h b/src/kudu/server/webui_util.h
index e8761b0..ae85148 100644
--- a/src/kudu/server/webui_util.h
+++ b/src/kudu/server/webui_util.h
@@ -18,7 +18,6 @@
 #define KUDU_SERVER_WEBUI_UTIL_H
 
 #include <iosfwd>
-#include <string>
 #include <vector>
 
 template <class T>
@@ -26,17 +25,20 @@ class scoped_refptr;
 
 namespace kudu {
 
+class EasyJson;
 class Schema;
 class MonitoredTask;
 
-void HtmlOutputSchemaTable(const Schema& schema,
-                           std::ostringstream* output);
-void HtmlOutputImpalaSchema(const std::string& table_name,
-                            const Schema& schema,
-                            const std::string& master_address,
-                            std::ostringstream* output);
-void HtmlOutputTaskList(const std::vector<scoped_refptr<MonitoredTask> >& tasks,
-                        std::ostringstream* output);
+// Appends a JSON array describing 'schema' to 'output', under the key "columns".
+void SchemaToJson(const Schema& schema, EasyJson* output);
+
+// Appends an HTML table describing 'schema' to 'output'.
+// TODO(wdberkeley) Remove this once /tablet is converted to a template.
+void HtmlOutputSchemaTable(const Schema& schema, std::ostringstream* output);
+
+// Appends a JSON array describing the tasks in 'tasks' to 'output', under the key "tasks".
+void TaskListToJson(const std::vector<scoped_refptr<MonitoredTask>>& tasks, EasyJson* output);
+
 } // namespace kudu
 
 #endif // KUDU_SERVER_WEBUI_UTIL_H

http://git-wip-us.apache.org/repos/asf/kudu/blob/e66fb10d/www/table.mustache
----------------------------------------------------------------------
diff --git a/www/table.mustache b/www/table.mustache
new file mode 100644
index 0000000..624edb3
--- /dev/null
+++ b/www/table.mustache
@@ -0,0 +1,143 @@
+{{!
+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.
+}}
+{{#error}}
+  <div class="text-error">{{.}}</div>
+{{/error}}
+{{^error}}
+  <h2>Table: {{name}} ({{id}})</h2>
+  <table class="table table-striped">
+    <tbody>
+    <tr><td>Version:</td><td>{{version}}</td></tr>
+    <tr><td>State:</td><td>{{state}} {{#state_msg}}({{.}}){{/state_msg}}</td></tr>
+    </tbody>
+  </table>
+  <h3>Schema</h3>
+  <table class='table table-striped'>
+    <thead><tr>
+      <th>Column</th>
+      <th>ID</th>
+      <th>Type</th>
+      <th>Encoding</th>
+      <th>Compression</th>
+      <th>Read default</th>
+      <th>Write default</th>
+    </tr></thead>
+    <tbody>
+    {{#columns}}
+      <tr>
+        <th>{{#is_key}}<u>{{/is_key}}
+          {{name}}
+          {{#is_key}}</u>{{/is_key}}</th>
+        <td>{{id}}</a></td>
+        <td>{{type}}</td>
+        <td>{{encoding}}</td>
+        <td>{{compression}}</td>
+        <td>{{read_default}}</td>
+        <td>{{write_default}}</td>
+      </tr>
+    {{/columns}}
+    </tbody>
+  </table>
+
+  <h3>Partition Schema</h3>
+  <pre>{{partition_schema}}</pre>
+
+  <h3>Tablets</h3>
+  <h4>Summary</h4>
+
+  <table class='table table-striped'>
+    <thead><tr>
+      <th>State</th>
+      <th>Count</th>
+      <th>Percentage</th>
+    </tr></thead>
+    <tbody>
+    {{#tablets_summary}}
+      <tr>
+        <td>{{state}}</td>
+        <td>{{count}}</td>
+        <td>{{percentage}}</td>
+      </tr>
+    {{/tablets_summary}}
+    </tbody>
+  </table>
+
+  <h4>Detail</h4>
+  <a href='#detail' data-toggle='collapse'>(toggle)</a>
+  <div id='detail' class='collapse'>
+    <table class='table table-striped table-hover'>
+      <thead><tr>
+        <th>Tablet ID</th>
+        {{{detail_partition_schema_header}}}
+        <th>State</th>
+        <th>Message</th>
+        <th>Peers</th>
+      </tr></thead>
+      <tbody>
+      {{#tablets_detail}}
+        <tr>
+          <td>{{id}}</td>
+          {{{partition_cols}}}
+          <td>{{state}}</td>
+          <td>{{state_msg}}</td>
+          <td>
+            <ul>
+              {{#peers}}
+                <li>{{#is_leader}}<b>LEADER: </b>{{/is_leader}}
+                  {{^is_leader}}{{role}}: {{/is_leader}}
+                  {{#target}}<a href="{{{.}}}">{{/target}}
+                  {{text}}
+                  {{#target}}</a>{{/target}}
+                </li>
+              {{/peers}}
+            </ul>
+          </td>
+        </tr>
+      {{/tablets_detail}}
+      </tbody>
+    </table>
+  </div>
+
+  <h3>Impala CREATE TABLE statement</h3>
+  {{! Unusual formatting below because <pre> preserves whitespace in the output. }}
+  <pre><code>CREATE EXTERNAL TABLE `{{name}}` STORED AS KUDU
+TBLPROPERTIES(
+    'kudu.table_name' = '{{name}}'
+    'kudu.master_addresses' = '{{master_addresses}}')</code></pre>
+
+  <h3>Tasks</h3>
+  <table class='table table-striped'>
+    <tr>
+      <th>Task Name</th>
+      <th>State</th>
+      <th>Time</th>
+      <th>Description</th>
+    </tr>
+    <tbody>
+    {{#tasks}}
+      <tr>
+        <td>{{name}}</td>
+        <td>{{state}}</td>
+        <td>{{time}}</td>
+        <td>{{description}}</td>
+      </tr>
+    {{/tasks}}
+    </tbody>
+  </table>
+{{/error}}

http://git-wip-us.apache.org/repos/asf/kudu/blob/e66fb10d/www/tables.mustache
----------------------------------------------------------------------
diff --git a/www/tables.mustache b/www/tables.mustache
index 6abe133..670b4b8 100644
--- a/www/tables.mustache
+++ b/www/tables.mustache
@@ -18,7 +18,7 @@ under the License.
 }}
 <h1>Tables</h1>
 {{#error}}
-  <div class="text-error">{{{.}}}</div>
+  <div class="text-error">{{.}}</div>
 {{/error}}
 {{^error}}
 There are {{num_tables}} tables


[3/3] kudu git commit: Add support for clang from XCode 9 in macOS

Posted by da...@apache.org.
Add support for clang from XCode 9 in macOS

Change-Id: Ib636deee77471e60c02e90028a0e5cc0f303b45a
Reviewed-on: http://gerrit.cloudera.org:8080/8119
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <da...@gmail.com>


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

Branch: refs/heads/master
Commit: 315a464c192d04e37f75a84aaceae22cc8f4425f
Parents: 4daa3ce
Author: Dan Burkert <da...@apache.org>
Authored: Thu Sep 21 12:24:31 2017 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Thu Sep 21 20:15:26 2017 +0000

----------------------------------------------------------------------
 cmake_modules/CompilerInfo.cmake | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/315a464c/cmake_modules/CompilerInfo.cmake
----------------------------------------------------------------------
diff --git a/cmake_modules/CompilerInfo.cmake b/cmake_modules/CompilerInfo.cmake
index 7e1812c..ff5f789 100644
--- a/cmake_modules/CompilerInfo.cmake
+++ b/cmake_modules/CompilerInfo.cmake
@@ -42,6 +42,19 @@ elseif("${COMPILER_VERSION_FULL}" MATCHES ".*clang-8")
   set(COMPILER_FAMILY "clang")
   set(COMPILER_VERSION "3.8.0svn")
 
+# clang on Mac OS X, XCode 9.
+#
+# TODO(dan): 4.0.0svn is just a guess, because Apple has not (as of the time of
+# writing) released the sources [1]. [2] and [3] are good resources for
+# determining the upstream Clang version as well.
+#
+# [1]: https://opensource.apple.com/
+# [2]: https://gist.github.com/yamaya/2924292
+# [3]: https://gist.github.com/epipping/ef8b02b0cfaec4a5ebf3a57092145a3f
+elseif("${COMPILER_VERSION_FULL}" MATCHES ".*clang-9")
+  set(COMPILER_FAMILY "clang")
+  set(COMPILER_VERSION "4.0.0svn")
+
 # gcc
 elseif("${COMPILER_VERSION_FULL}" MATCHES ".*gcc version.*")
   set(COMPILER_FAMILY "gcc")


[2/3] kudu git commit: KUDU-501 Redirect to leader master web UI

Posted by da...@apache.org.
KUDU-501 Redirect to leader master web UI

The /table and /tables page of the master web UI only work
on the leader master. Previously, we just showed a gross error when the
user tried to access these pages on a non-leader. This adds a nice
redirect instead (or an error about why a redirect wasn't possible).

Change-Id: If31543b7899976ec02704111e9e789035c44dfe1
Reviewed-on: http://gerrit.cloudera.org:8080/8068
Tested-by: Kudu Jenkins
Reviewed-by: Will Berkeley <wd...@gmail.com>


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

Branch: refs/heads/master
Commit: 4daa3ce8742280495c7b6f9a9b2a828e3572ea2c
Parents: e66fb10
Author: Will Berkeley <wd...@apache.org>
Authored: Wed Sep 13 23:59:36 2017 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Thu Sep 21 20:05:23 2017 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/CMakeLists.txt       |  2 +-
 .../integration-tests/webserver-stress-itest.cc |  6 +-
 src/kudu/master/master-path-handlers.cc         | 87 ++++++++++++++++++--
 src/kudu/master/master-path-handlers.h          |  8 ++
 www/table.mustache                              |  7 ++
 www/tables.mustache                             |  6 ++
 6 files changed, 107 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4daa3ce8/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 43582d1..8c36fdc 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -104,7 +104,7 @@ ADD_KUDU_TEST(token_signer-itest RESOURCE_LOCK "master-rpc-ports")
 ADD_KUDU_TEST(ts_recovery-itest)
 ADD_KUDU_TEST(ts_tablet_manager-itest)
 ADD_KUDU_TEST(update_scan_delta_compact-test RUN_SERIAL true)
-ADD_KUDU_TEST(webserver-stress-itest)
+ADD_KUDU_TEST(webserver-stress-itest RESOURCE_LOCK "master-rpc-ports")
 ADD_KUDU_TEST(write_throttling-itest)
 
 if (NOT APPLE)

http://git-wip-us.apache.org/repos/asf/kudu/blob/4daa3ce8/src/kudu/integration-tests/webserver-stress-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/webserver-stress-itest.cc b/src/kudu/integration-tests/webserver-stress-itest.cc
index 8c6cd92..b9503de 100644
--- a/src/kudu/integration-tests/webserver-stress-itest.cc
+++ b/src/kudu/integration-tests/webserver-stress-itest.cc
@@ -44,7 +44,11 @@ TEST_F(KuduTest, TestWebUIDoesNotCrashCluster) {
 #endif
   const int kNumTablets = 50;
 
-  ExternalMiniCluster cluster;
+  ExternalMiniClusterOptions opts;
+  opts.master_rpc_ports = { 11010, 11011, 11012 };
+  opts.num_masters = opts.master_rpc_ports.size();
+
+  ExternalMiniCluster cluster(opts);
   ASSERT_OK(cluster.Start());
 
   // Start pounding the master and tserver's web UIs.

http://git-wip-us.apache.org/repos/asf/kudu/blob/4daa3ce8/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 ebd42f2..bf3d12d 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -43,6 +43,7 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/catalog_manager.h"
 #include "kudu/master/master.h"
@@ -140,11 +141,33 @@ void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& req,
   generate_table(dead_tserver_rows, "Dead Tablet Servers", output);
 }
 
-void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req,
-                                              EasyJson* output) {
+namespace {
+
+// Extracts the value of the 'redirects' parameter from 'req'; returns 0 if the
+// parameter doesn't exist or couldn't be parsed.
+int ExtractRedirectsFromRequest(const Webserver::WebRequest& req) {
+  string redirects_str;
+  int redirects = 0;
+  if (FindCopy(req.parsed_args, "redirects", &redirects_str)) {
+    if (!safe_strto32(redirects_str, &redirects)) {
+      return 0;
+    }
+  }
+  return redirects;
+}
+
+} // anonymous namespace
+
+void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req, EasyJson* output) {
   CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
-  if (!l.first_failed_status().ok()) {
-    (*output)["error"] = Substitute("Master is not ready: $0",  l.first_failed_status().ToString());
+  if (!l.catalog_status().ok()) {
+    (*output)["error"] = Substitute("Master is not ready: $0",  l.catalog_status().ToString());
+    return;
+  }
+  if (!l.leader_status().ok()) {
+    // Track redirects to prevent a redirect loop.
+    int redirects = ExtractRedirectsFromRequest(req);
+    SetupLeaderMasterRedirect("tables?", redirects, output);
     return;
   }
 
@@ -206,15 +229,21 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
   }
 
   CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
-  if (!l.first_failed_status().ok()) {
-    (*output)["error"] = Substitute("Master is not ready: ", l.first_failed_status().ToString());
+  if (!l.catalog_status().ok()) {
+    (*output)["error"] = Substitute("Master is not ready: $0", l.catalog_status().ToString());
+    return;
+  }
+  if (!l.leader_status().ok()) {
+    // Track redirects to prevent a redirect loop.
+    int redirects = ExtractRedirectsFromRequest(req);
+    SetupLeaderMasterRedirect(Substitute("table?id=$0", table_id), redirects, output);
     return;
   }
 
   scoped_refptr<TableInfo> table;
   Status s = master_->catalog_manager()->GetTableInfo(table_id, &table);
   if (!s.ok()) {
-    (*output)["error"] = Substitute("Master is not ready: ", s.ToString());
+    (*output)["error"] = Substitute("Master is not ready: $0", s.ToString());
     return;
   }
 
@@ -635,5 +664,49 @@ string MasterPathHandlers::MasterAddrsToCsv() const {
   return addr.ToString();
 }
 
+Status MasterPathHandlers::GetLeaderMasterHttpAddr(string* leader_http_addr) const {
+  vector<ServerEntryPB> masters;
+  RETURN_NOT_OK_PREPEND(master_->ListMasters(&masters), "unable to list masters");
+  for (const auto& master : masters) {
+    if (master.has_error()) {
+      continue;
+    }
+    if (master.role() != RaftPeerPB::LEADER) {
+      continue;
+    }
+    const ServerRegistrationPB& reg = master.registration();
+    if (reg.http_addresses().empty()) {
+      return Status::NotFound("leader master has no http address");
+    }
+    *leader_http_addr = Substitute("$0://$1:$2",
+                                   reg.https_enabled() ? "https" : "http",
+                                   reg.http_addresses(0).host(),
+                                   reg.http_addresses(0).port());
+    return Status::OK();
+  }
+  return Status::NotFound("no leader master known to this master");
+}
+
+void MasterPathHandlers::SetupLeaderMasterRedirect(const string& path,
+                                                   int redirects,
+                                                   EasyJson* output) const {
+  // Allow 3 redirects.
+  const int max_redirects = 3;
+  (*output)["error"] = "Master is not the leader.";
+  if (redirects >= max_redirects) {
+    (*output)["redirect_error"] = "Too many redirects attempting to find the leader master.";
+    return;
+  }
+  string leader_http_addr;
+  Status s = GetLeaderMasterHttpAddr(&leader_http_addr);
+  if (!s.ok()) {
+    (*output)["redirect_error"] = Substitute("Unable to redirect to leader master: $0",
+                                             s.ToString());
+    return;
+  }
+  (*output)["leader_redirect"] = Substitute("$0/$1&redirects=$2",
+                                            leader_http_addr, path, redirects + 1);
+}
+
 } // namespace master
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/4daa3ce8/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 9bd05c7..6195a74 100644
--- a/src/kudu/master/master-path-handlers.h
+++ b/src/kudu/master/master-path-handlers.h
@@ -73,6 +73,14 @@ class MasterPathHandlers {
   // Return a CSV of master addresses suitable for display.
   std::string MasterAddrsToCsv() const;
 
+  // If a leader master is known and has an http address, place it in leader_http_addr.
+  Status GetLeaderMasterHttpAddr(std::string* leader_http_addr) const;
+
+  // Adds the necessary properties to 'output' to set up a redirect to the leader master, or
+  // provide an error message if no redirect is possible.
+  // The redirect will link to <master web UI url>/path&redirects=(redirects + 1).
+  void SetupLeaderMasterRedirect(const std::string& path, int redirects, EasyJson* output) const;
+
   Master* master_;
   DISALLOW_COPY_AND_ASSIGN(MasterPathHandlers);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/4daa3ce8/www/table.mustache
----------------------------------------------------------------------
diff --git a/www/table.mustache b/www/table.mustache
index 624edb3..575d3fc 100644
--- a/www/table.mustache
+++ b/www/table.mustache
@@ -19,6 +19,12 @@ under the License.
 {{#error}}
   <div class="text-error">{{.}}</div>
 {{/error}}
+{{#redirect_error}}
+  <div class="text-error">{{.}}</div>
+{{/redirect_error}}
+{{#leader_redirect}}
+  <div>You can find this page on the <a href="{{{.}}}">leader master's web UI</a>.</div>
+{{/leader_redirect}}
 {{^error}}
   <h2>Table: {{name}} ({{id}})</h2>
   <table class="table table-striped">
@@ -27,6 +33,7 @@ under the License.
     <tr><td>State:</td><td>{{state}} {{#state_msg}}({{.}}){{/state_msg}}</td></tr>
     </tbody>
   </table>
+
   <h3>Schema</h3>
   <table class='table table-striped'>
     <thead><tr>

http://git-wip-us.apache.org/repos/asf/kudu/blob/4daa3ce8/www/tables.mustache
----------------------------------------------------------------------
diff --git a/www/tables.mustache b/www/tables.mustache
index 670b4b8..e50fd8c 100644
--- a/www/tables.mustache
+++ b/www/tables.mustache
@@ -20,6 +20,12 @@ under the License.
 {{#error}}
   <div class="text-error">{{.}}</div>
 {{/error}}
+{{#redirect_error}}
+  <div class="text-error">{{.}}</div>
+{{/redirect_error}}
+{{#leader_redirect}}
+  <div>You can find this page on the <a href="{{{.}}}">leader master's web UI</a>.</div>
+{{/leader_redirect}}
 {{^error}}
 There are {{num_tables}} tables
 <table class="table table-striped">