You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2017/05/16 20:22:27 UTC

[2/3] kudu git commit: [webui] Improvements for when there's many tablets & cleanup

[webui] Improvements for when there's many tablets & cleanup

This patch makes several improvements to the ui when there's
a lot of tablets:

1. /tablets displays a summary of the tablets' statuses.
2. /tablets has a toggle-collapse detailed table of tablets.
3. /table displays of summary of the tablets' statuses before the
tablet table.
4. /table has a toggle-collapse detailed table of tablets.
5. Detailed tablet tables use table-hover so it's easier to
use the table to look up information by, e.g. id.

All of these changes should help address KUDU-1974 and
KUDU-1959, by making it easier to see the overall health and
status of a table or tablet server's tablets.

Additionally, I found that many tables were not using the
<thead> and <tbody> elements, which caused some bootstrap
table styles not to be working as intended. All tables where
this was a problem have been fixed.

Also I added a memory usage column to the detailed tablets
table on /tablets.

Scrrenshots: http://imgur.com/a/Mmfbe

Change-Id: Ic3904a4b0fbb6446615cd46c8a6f30f81c832c53
Reviewed-on: http://gerrit.cloudera.org:8080/6870
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: b492d8ed3cc66e8fafe6e97f45ac9731de1e584f
Parents: 2f62443
Author: Will Berkeley <wd...@apache.org>
Authored: Thu May 11 15:46:06 2017 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Tue May 16 20:12:24 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/master-path-handlers.cc   | 68 ++++++++++++------
 src/kudu/server/default-path-handlers.cc  |  7 +-
 src/kudu/server/webserver.cc              |  2 +
 src/kudu/server/webui_util.cc             |  8 +--
 src/kudu/tserver/tserver-path-handlers.cc | 97 +++++++++++++++++++-------
 src/kudu/util/thread.cc                   |  7 +-
 6 files changed, 130 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b492d8ed/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 2f53676..70793a7 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -99,12 +99,13 @@ void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& req,
 
   *output << "<h3>Version Summary</h3>";
   *output << "<table class='table table-striped'>\n";
-  *output << "<tr><th>Version</th><th>Count (Live)</th><th>Count (Dead)</th></tr>\n";
+  *output << "<thead><tr><th>Version</th><th>Count (Live)</th><th>Count (Dead)</th></tr></thead>\n";
+  *output << "<tbody>\n";
   for (const auto& entry : version_counts) {
     *output << Substitute("<tr><td>$0</td><td>$1</td><td>$2</td></tr>\n",
                           entry.first, entry.second[0], entry.second[1]);
   }
-  *output << "</table>";
+  *output << "</tbody></table>\n";
 
   *output << "<h3>" << "Registrations" << "</h3>\n";
   auto generate_table = [](const vector<string>& rows,
@@ -113,9 +114,11 @@ void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& req,
     if (!rows.empty()) {
       *output << "<h4>" << header << "</h4>\n";
       *output << "<table class='table table-striped'>\n";
-      *output << "<tr><th>UUID</th><th>Time since heartbeat</th><th>Registration</th></tr>\n";
+      *output << "<thead><tr><th>UUID</th><th>Time since heartbeat</th>"
+          "<th>Registration</th></tr></thead>\n";
+      *output << "<tbody>\n";
       *output << JoinStrings(rows, "\n");
-      *output << "</table>\n";
+      *output << "</tbody></table>\n";
     }
   };
   generate_table(live_tserver_rows, "Live Tablet Servers", output);
@@ -135,9 +138,10 @@ void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req,
   std::vector<scoped_refptr<TableInfo>> tables;
   master_->catalog_manager()->GetAllTables(&tables);
 
+  *output << Substitute("There are $0 tables\n", tables.size());
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Table Name</th><th>Table Id</th>" <<
-      "<th>State</th><th>State Message</th></tr>\n";
+  *output << "  <thead><tr><th>Table Name</th><th>Table Id</th>" <<
+      "<th>State</th><th>State Message</th></tr></thead>\n";
   typedef std::map<string, string> StringMap;
   StringMap ordered_tables;
   for (const scoped_refptr<TableInfo>& table : tables) {
@@ -155,10 +159,11 @@ void MasterPathHandlers::HandleCatalogManager(const Webserver::WebRequest& req,
         state,
         EscapeForHtmlToString(l.data().pb.state_msg()));
   }
+  *output << "<tbody>\n";
   for (const StringMap::value_type& table : ordered_tables) {
     *output << table.second;
   }
-  *output << "</table>\n";
+  *output << "</tbody></table>\n";
 }
 
 namespace {
@@ -183,7 +188,7 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
   // Parse argument.
   string table_id;
   if (!FindCopy(req.parsed_args, "id", &table_id)) {
-    // TODO: webserver should give a way to return a non-200 response code
+    // TODO(wdb): webserver should give a way to return a non-200 response code
     *output << "Missing 'id' argument";
     return;
   }
@@ -239,18 +244,24 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
   *output << "<h3>Schema</h3>";
   HtmlOutputSchemaTable(schema, output);
 
-  // Prepare the tablets table first because the tablet partition information is
-  // also used to make the range bounds.
+  // Visit (& lock) each tablet once to build the partition schema, tablets summary,
+  // and tablets detail tables all at once.
   std::vector<string> range_partitions;
-  std::ostringstream tablets_output;
-  tablets_output << "<h3>Tablets</h3>";
-  tablets_output << "<table class='table table-striped'>\n";
-  tablets_output << "  <tr><th>Tablet ID</th>"
+  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>\n";
+                 << "<th>State</th><th>Message</th><th>Peers</th></tr></thead>\n";
+  detail_output << "<tbody>\n";
   for (const scoped_refptr<TabletInfo>& tablet : tablets) {
     vector<pair<string, RaftPeerPB::Role>> sorted_replicas;
     TabletMetadataLock l(tablet.get(), TabletMetadataLock::READ);
+    summary_states[SysTabletsEntryPB_State_Name(l.data().pb.state())]++;
     if (l.data().pb.has_committed_consensus_state()) {
       const ConsensusStatePB& cstate = l.data().pb.committed_consensus_state();
       for (const auto& peer : cstate.config().peers()) {
@@ -298,7 +309,7 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
 
     string state = SysTabletsEntryPB_State_Name(l.data().pb.state());
     Capitalize(&state);
-    tablets_output << Substitute(
+    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),
@@ -306,7 +317,7 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
         EscapeForHtmlToString(l.data().pb.state_msg()),
         raft_config_html.str());
   }
-  tablets_output << "</table>\n";
+  detail_output << "</tbody></table></div>\n";
 
   // Write out the partition schema and range bound information...
   *output << "<h3>Partition Schema</h3>";
@@ -314,8 +325,21 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
   *output << EscapeForHtmlToString(partition_schema.DisplayString(schema, range_partitions));
   *output << "</pre>";
 
-  // ...then the tablets table.
-  *output << tablets_output.str();
+  // ...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";
+  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));
+  }
+  *output << "</tbody></table>\n";
+
+  // ...and finally the tablet detail table.
+  *output << detail_output.str();
 
   *output << "<h3>Impala CREATE TABLE statement</h3>\n";
   string master_addresses;
@@ -357,8 +381,8 @@ void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& req,
   }
   *output << "<h1> Masters </h1>\n";
   *output <<  "<table class='table table-striped'>\n";
-  *output <<  "  <tr><th>UUID</th><th>Role</th><th>Registration</th></tr>\n";
-
+  *output <<  "  <thead><tr><th>UUID</th><th>Role</th><th>Registration</th></tr></thead>\n";
+  *output << "<tbody>\n";
   for (const ServerEntryPB& master : masters) {
     if (master.has_error()) {
       Status error = StatusFromPB(master.error());
@@ -378,7 +402,7 @@ void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& req,
         reg_str);
   }
 
-  *output << "</table>";
+  *output << "</tbody></table>";
 }
 
 namespace {

http://git-wip-us.apache.org/repos/asf/kudu/blob/b492d8ed/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 94acbbe..f4b3f27 100644
--- a/src/kudu/server/default-path-handlers.cc
+++ b/src/kudu/server/default-path-handlers.cc
@@ -170,8 +170,9 @@ static void MemTrackersHandler(const Webserver::WebRequest& /*req*/, std::ostrin
 
   *output << "<h1>Memory usage by subsystem</h1>\n";
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Id</th><th>Parent</th><th>Limit</th><th>Current Consumption</th>"
-      "<th>Peak consumption</th></tr>\n";
+  *output << "  <thead><tr><th>Id</th><th>Parent</th><th>Limit</th><th>Current Consumption</th>"
+      "<th>Peak consumption</th></tr></thead>\n";
+  *output << "<tbody>\n";
 
   vector<shared_ptr<MemTracker> > trackers;
   MemTracker::ListTrackers(&trackers);
@@ -186,7 +187,7 @@ static void MemTrackersHandler(const Webserver::WebRequest& /*req*/, std::ostrin
                             tracker->id(), parent, limit_str, current_consumption_str,
                             peak_consumption_str);
   }
-  *output << "</table>\n";
+  *output << "</tbody></table>\n";
 }
 
 void AddDefaultPathHandlers(Webserver* webserver) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/b492d8ed/src/kudu/server/webserver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webserver.cc b/src/kudu/server/webserver.cc
index f7a3757..7d7619c 100644
--- a/src/kudu/server/webserver.cc
+++ b/src/kudu/server/webserver.cc
@@ -440,6 +440,8 @@ const char* const PAGE_HEADER = "<!DOCTYPE html>"
 "   <head><title>Kudu</title>"
 " <meta charset='utf-8'/>"
 " <link href='/bootstrap/css/bootstrap.min.css' rel='stylesheet' media='screen' />"
+" <script src='/jquery-1.11.1.min.js' defer></script>"
+" <script src='/bootstrap/js/bootstrap.min.js' defer></script>"
 " <link href='/kudu.css' rel='stylesheet' />"
 " </head>"
 " <body>";

http://git-wip-us.apache.org/repos/asf/kudu/blob/b492d8ed/src/kudu/server/webui_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/webui_util.cc b/src/kudu/server/webui_util.cc
index 01b7f92..2a7307f 100644
--- a/src/kudu/server/webui_util.cc
+++ b/src/kudu/server/webui_util.cc
@@ -34,12 +34,12 @@ namespace kudu {
 void HtmlOutputSchemaTable(const Schema& schema,
                            std::ostringstream* output) {
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr>"
+  *output << "  <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>\n";
-
+          << "</tr></thead>\n";
+  *output << "<tbody>";
   for (int i = 0; i < schema.num_columns(); i++) {
     const ColumnSchema& col = schema.column(i);
     string read_default = "-";
@@ -63,7 +63,7 @@ void HtmlOutputSchemaTable(const Schema& schema,
                           EscapeForHtmlToString(read_default),
                           EscapeForHtmlToString(write_default));
   }
-  *output << "</table>\n";
+  *output << "</tbody></table>\n";
 }
 
 void HtmlOutputImpalaSchema(const std::string& table_name,

http://git-wip-us.apache.org/repos/asf/kudu/blob/b492d8ed/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 f12e4b9..e89016c 100644
--- a/src/kudu/tserver/tserver-path-handlers.cc
+++ b/src/kudu/tserver/tserver-path-handlers.cc
@@ -18,6 +18,7 @@
 #include "kudu/tserver/tserver-path-handlers.h"
 
 #include <algorithm>
+#include <map>
 #include <memory>
 #include <sstream>
 #include <string>
@@ -32,6 +33,7 @@
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/server/webui_util.h"
+#include "kudu/tablet/metadata.pb.h"
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/tablet/tablet_bootstrap.h"
 #include "kudu/tablet/tablet_replica.h"
@@ -52,6 +54,7 @@ using kudu::MaintenanceManagerStatusPB_CompletedOpPB;
 using kudu::MaintenanceManagerStatusPB_MaintenanceOpPB;
 using kudu::tablet::Tablet;
 using kudu::tablet::TabletReplica;
+using kudu::tablet::TabletStatePB;
 using kudu::tablet::TabletStatusPB;
 using kudu::tablet::Transaction;
 using std::endl;
@@ -120,9 +123,10 @@ void TabletServerPathHandlers::HandleTransactionsPage(const Webserver::WebReques
   if (!as_text) {
     *output << "<h1>Transactions</h1>\n";
     *output << "<table class='table table-striped'>\n";
-    *output << "   <tr><th>Tablet id</th><th>Op Id</th>"
+    *output << "   <thead><tr><th>Tablet id</th><th>Op Id</th>"
       "<th>Transaction Type</th><th>"
-      "Total time in-flight</th><th>Description</th></tr>\n";
+      "Total time in-flight</th><th>Description</th></tr></thead>\n";
+    *output << "<tbody>\n";
   }
 
   for (const scoped_refptr<TabletReplica>& replica : replicas) {
@@ -163,7 +167,7 @@ void TabletServerPathHandlers::HandleTransactionsPage(const Webserver::WebReques
   }
 
   if (!as_text) {
-    *output << "</table>\n";
+    *output << "</tbody></table>\n";
   }
 }
 
@@ -189,14 +193,42 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& re
                      std::make_pair(rep_b->tablet_metadata()->table_name(), rep_b->tablet_id());
             });
 
-  auto generate_table = [this](const string& header,
-                               const vector<scoped_refptr<TabletReplica>>& replicas,
-                               ostream* output) {
+  // For assigning ids to table divs;
+  int i = 0;
+  auto generate_table = [this, &i](const string& header,
+                                   const vector<scoped_refptr<TabletReplica>>& replicas,
+                                   ostream* output) {
+    i++;
     *output << "<h3>" << header << "</h3>\n";
-    *output << "<table class='table table-striped'>\n";
-    *output << "  <tr><th>Table name</th><th>Tablet ID</th>"
-         "<th>Partition</th>"
-         "<th>State</th><th>On-disk size</th><th>RaftConfig</th><th>Last status</th></tr>\n";
+
+    *output << "<h4>Summary</h4>\n";
+    map<string, int> tablet_statuses;
+    for (const scoped_refptr<TabletReplica>& replica : replicas) {
+      tablet_statuses[TabletStatePB_Name(replica->state())]++;
+    }
+    *output << "<table class='table table-striped table-hover'>\n";
+    *output << "<thead><tr><th>Status</th><th>Count</th><th>Percentage</th></tr></thead>\n";
+    *output << "<tbody>\n";
+    for (const auto& entry : tablet_statuses) {
+      double percent = replicas.size() == 0 ? 0 : (100.0 * entry.second) / replicas.size();
+      *output << Substitute("<tr><td>$0</td><td>$1</td><td>$2</td></tr>\n",
+                            entry.first,
+                            entry.second,
+                            StringPrintf("%.2f", percent));
+    }
+    *output << "</tbody>\n";
+    *output << Substitute("<tfoot><tr><td>Total</td><td>$0</td><td></td></tr></tfoot>\n",
+                          replicas.size());
+    *output << "</table>\n";
+
+    *output << "<h4>Detail</h4>";
+    *output << Substitute("<a href='#detail$0' data-toggle='collapse'>(toggle)</a>\n", i);
+    *output << Substitute("<div id='detail$0' class='collapse'>\n", i);
+    *output << "<table class='table table-striped table-hover'>\n";
+    *output << "<thead><tr><th>Table name</th><th>Tablet ID</th>"
+        "<th>Partition</th><th>State</th><th>Write buffer memory usage</th>"
+        "<th>On-disk size</th><th>RaftConfig</th><th>Last status</th></tr></thead>\n";
+    *output << "<tbody>\n";
     for (const scoped_refptr<TabletReplica>& replica : replicas) {
       TabletStatusPB status;
       replica->GetTabletStatusPB(&status);
@@ -208,6 +240,11 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& re
       } else {
         tablet_id_or_link = EscapeForHtmlToString(id);
       }
+      string mem_bytes = "";
+      if (replica->tablet() != nullptr) {
+        mem_bytes = HumanReadableNumBytes::ToString(
+            replica->tablet()->mem_tracker()->consumption());
+      }
       string n_bytes = "";
       if (status.has_estimated_on_disk_size()) {
         n_bytes = HumanReadableNumBytes::ToString(status.estimated_on_disk_size());
@@ -217,23 +254,22 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& re
                                  .PartitionDebugString(replica->tablet_metadata()->partition(),
                                                        replica->tablet_metadata()->schema());
 
-      // TODO(unknown): would be nice to include some other stuff like memory usage
       scoped_refptr<consensus::Consensus> consensus = replica->shared_consensus();
       (*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
-          "<td>$3</td><td>$4</td><td>$5</td><td>$6</td></tr>\n",
+          "<td>$3</td><td>$4</td><td>$5</td><td>$6</td><td>$7</td></tr>\n",
           EscapeForHtmlToString(table_name), // $0
           tablet_id_or_link, // $1
           EscapeForHtmlToString(partition), // $2
-          EscapeForHtmlToString(replica->HumanReadableState()), n_bytes, // $3, $4
+          EscapeForHtmlToString(replica->HumanReadableState()), mem_bytes, n_bytes, // $3, $4, $5
           consensus ? ConsensusStatePBToHtml(consensus->
               ConsensusState(CONSENSUS_CONFIG_COMMITTED))
-                    : "", // $5
-          EscapeForHtmlToString(status.last_status())); // $6
+                    : "", // $6
+          EscapeForHtmlToString(status.last_status())); // $7
     }
-    *output << "</table>\n";
+    *output << "<tbody></table>\n</div>\n";
   };
 
   vector<scoped_refptr<TabletReplica>> live_replicas;
@@ -434,16 +470,17 @@ void TabletServerPathHandlers::HandleScansPage(const Webserver::WebRequest& req,
                                                std::ostringstream* output) {
   *output << "<h1>Scans</h1>\n";
   *output << "<table class='table table-striped'>\n";
-  *output << "<tr><th>Tablet id</th><th>Scanner id</th><th>Total time in-flight</th>"
+  *output << "<thead><tr><th>Tablet id</th><th>Scanner id</th><th>Total time in-flight</th>"
       "<th>Time since last update</th><th>Requestor</th><th>Iterator Stats</th>"
-      "<th>Pushed down key predicates</th><th>Other predicates</th></tr>\n";
+      "<th>Pushed down key predicates</th><th>Other predicates</th></tr></thead>\n";
+  *output << "<tbody>\n";
 
   vector<SharedScanner> scanners;
   tserver_->scanner_manager()->ListScanners(&scanners);
   for (const SharedScanner& scanner : scanners) {
     *output << ScannerToHtml(*scanner);
   }
-  *output << "</table>";
+  *output << "</tbody></table>";
 }
 
 string TabletServerPathHandlers::ScannerToHtml(const Scanner& scanner) const {
@@ -534,13 +571,15 @@ void TabletServerPathHandlers::HandleDashboardsPage(const Webserver::WebRequest&
 
   *output << "<h3>Dashboards</h3>\n";
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Dashboard</th><th>Description</th></tr>\n";
+  *output << "  <thead><tr><th>Dashboard</th><th>Description</th></tr></thead>\n";
+  *output << "  <tbody\n";
   *output << GetDashboardLine("scans", "Scans", "List of scanners that are currently running.");
   *output << GetDashboardLine("transactions", "Transactions", "List of transactions that are "
                                                               "currently running.");
   *output << GetDashboardLine("maintenance-manager", "Maintenance Manager",
                               "List of operations that are currently running and those "
                               "that are registered.");
+  *output << "</tbody></table>\n";
 }
 
 string TabletServerPathHandlers::GetDashboardLine(const std::string& link,
@@ -567,7 +606,8 @@ void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::Web
   *output << "<h1>Maintenance Manager state</h1>\n";
   *output << "<h3>Running operations</h3>\n";
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Name</th><th>Instances running</th></tr>\n";
+  *output << "  <thead><tr><th>Name</th><th>Instances running</th></tr></thead>\n";
+  *output << "<tbody>\n";
   for (int i = 0; i < ops_count; i++) {
     MaintenanceManagerStatusPB_MaintenanceOpPB op_pb = pb.registered_operations(i);
     if (op_pb.running() > 0) {
@@ -576,11 +616,13 @@ void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::Web
                              op_pb.running());
     }
   }
-  *output << "</table>\n";
+  *output << "</tbody></table>\n";
 
   *output << "<h3>Recent completed operations</h3>\n";
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Name</th><th>Duration</th><th>Time since op started</th></tr>\n";
+  *output << "  <thead><tr><th>Name</th><th>Duration</th>"
+      "<th>Time since op started</th></tr></thead>\n";
+  *output << "<tbody>\n";
   for (int i = 0; i < pb.completed_operations_size(); i++) {
     MaintenanceManagerStatusPB_CompletedOpPB op_pb = pb.completed_operations(i);
     *output <<  Substitute("<tr><td>$0</td><td>$1</td><td>$2</td></tr>\n",
@@ -590,12 +632,13 @@ void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::Web
                            HumanReadableElapsedTime::ToShortString(
                                op_pb.secs_since_start()));
   }
-  *output << "</table>\n";
+  *output << "</tbody></table>\n";
 
   *output << "<h3>Non-running operations</h3>\n";
   *output << "<table class='table table-striped'>\n";
-  *output << "  <tr><th>Name</th><th>Runnable</th><th>RAM anchored</th>\n"
-          << "       <th>Logs retained</th><th>Perf</th></tr>\n";
+  *output << "  <thead><tr><th>Name</th><th>Runnable</th><th>RAM anchored</th>\n"
+          << "       <th>Logs retained</th><th>Perf</th></tr></thead>\n";
+  *output << "<tbody>\n";
   for (int i = 0; i < ops_count; i++) {
     MaintenanceManagerStatusPB_MaintenanceOpPB op_pb = pb.registered_operations(i);
     if (op_pb.running() == 0) {
@@ -607,7 +650,7 @@ void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::Web
                             op_pb.perf_improvement());
     }
   }
-  *output << "</table>\n";
+  *output << "</tbody></table>\n";
 }
 
 } // namespace tserver

http://git-wip-us.apache.org/repos/asf/kudu/blob/b492d8ed/src/kudu/util/thread.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/thread.cc b/src/kudu/util/thread.cc
index 71aa400..471b87d 100644
--- a/src/kudu/util/thread.cc
+++ b/src/kudu/util/thread.cc
@@ -371,14 +371,15 @@ void ThreadMgr::ThreadPathHandler(const WebCallbackRegistry::WebRequest& req,
     }
 
     (*output) << "<table class='table table-hover table-border'>";
-    (*output) << "<tr><th>Thread name</th><th>Cumulative User CPU(s)</th>"
+    (*output) << "<thead><tr><th>Thread name</th><th>Cumulative User CPU(s)</th>"
               << "<th>Cumulative Kernel CPU(s)</th>"
-              << "<th>Cumulative IO-wait(s)</th></tr>";
+              << "<th>Cumulative IO-wait(s)</th></tr></thead>";
+    (*output) << "<tbody>\n";
 
     for (const ThreadCategory* category : categories_to_print) {
       PrintThreadCategoryRows(*category, output);
     }
-    (*output) << "</table>";
+    (*output) << "</tbody></table>";
   } else {
     (*output) << "<h2>Thread Groups</h2>";
     if (metrics_enabled_) {