You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2018/05/23 20:39:19 UTC

[1/2] kudu git commit: KUDU-1867 Improve the "Could not lock .../block_manager_instance" error log

Repository: kudu
Updated Branches:
  refs/heads/master 14080bb0f -> 0e19cf772


KUDU-1867 Improve the "Could not lock .../block_manager_instance" error log

This improvement removes the double "Could not lock.." and the
double file paths and adds a descriptive message for the cause
of the error.

Before:
Check failed: _s.ok() Bad status: IO error: Failed to load FS
layout: Could not lock /private/tmp/kudu/data/master/0/1/data/
block_manager_instance: Could not lock /private/tmp/kudu/data/
master/0/1/data/block_manager_instance: lock /private/tmp/kudu/
data/master/0/1/data/block_manager_instance: Resource temporarily
unavailable (error 35)

After:
Check failed: _s.ok() Bad status: IO error: Failed to load FS
layout: Could not lock block_manager_instance file. Make sure
that Kudu is not already running and you are not trying to run
Kudu with a different user than before: lock /private/tmp/kudu/
data/master/0/1/data/block_manager_instance: Resource temporarily
unavailable (error 35)

Change-Id: I1c6f078f95b3295e31047b9fe9a5dd643a402ba0
Reviewed-on: http://gerrit.cloudera.org:8080/10419
Reviewed-by: Andrew Wong <aw...@cloudera.com>
Tested-by: Andrew Wong <aw...@cloudera.com>


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

Branch: refs/heads/master
Commit: 83f3a7e9347462c11bb6c3c6b58d91cf6dfc52ed
Parents: 14080bb
Author: fwang29 <fw...@cloudera.com>
Authored: Tue May 15 14:48:06 2018 -0700
Committer: Andrew Wong <aw...@cloudera.com>
Committed: Mon May 21 23:14:24 2018 +0000

----------------------------------------------------------------------
 src/kudu/fs/block_manager_util.cc | 4 +++-
 src/kudu/fs/data_dirs.cc          | 6 ++----
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/83f3a7e9/src/kudu/fs/block_manager_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager_util.cc b/src/kudu/fs/block_manager_util.cc
index 1d2e372..6c7b5bd 100644
--- a/src/kudu/fs/block_manager_util.cc
+++ b/src/kudu/fs/block_manager_util.cc
@@ -157,7 +157,9 @@ Status PathInstanceMetadataFile::Lock() {
 
   FileLock* lock;
   RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(env_->LockFile(filename_, &lock),
-                                      Substitute("Could not lock $0", filename_));
+                                      "Could not lock block_manager_instance file. Make sure that "
+                                      "Kudu is not already running and you are not trying to run "
+                                      "Kudu with a different user than before");
   lock_.reset(lock);
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/83f3a7e9/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index c35cbfc..8e1acd8 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -609,14 +609,12 @@ Status DataDirManager::LoadInstances(
       // This may return OK and mark 'instance' as failed.
       Status s = instance->Lock();
       if (!s.ok()) {
-        Status new_status = s.CloneAndPrepend(Substitute(
-            "Could not lock $0", instance_filename));
         if (lock_mode == LockMode::OPTIONAL) {
-          LOG(WARNING) << new_status.ToString();
+          LOG(WARNING) << s.ToString();
           LOG(WARNING) << "Proceeding without lock";
         } else {
           DCHECK(LockMode::MANDATORY == lock_mode);
-          return new_status;
+          return s;
         }
       }
     }


[2/2] kudu git commit: [webui] Convert /tablets page to mustache

Posted by aw...@apache.org.
[webui] Convert /tablets page to mustache

Originally I wanted to add a message to the /tablets page if there were
no tablet replicas on the server, just so it wouldn't be blank, but here
we are.

Also, since our pre-commit test machines don't have templates, I had to
adjust some tests to search for substrings of the JSON rather than the
template-generated HTML.

Change-Id: Ifddd74b23add52ae181b98a226b39e39df9b51ce
Reviewed-on: http://gerrit.cloudera.org:8080/10422
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Tested-by: Will Berkeley <wd...@gmail.com>
Reviewed-by: Andrew Wong <aw...@cloudera.com>


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

Branch: refs/heads/master
Commit: 0e19cf77295224d001553b4070ccbe6929a4cd83
Parents: 83f3a7e
Author: Will Berkeley <wd...@apache.org>
Authored: Tue May 15 17:28:46 2018 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Tue May 22 16:32:06 2018 +0000

----------------------------------------------------------------------
 src/kudu/tserver/tablet_server-test.cc    |  28 ++-
 src/kudu/tserver/tserver_path_handlers.cc | 287 +++++++++++--------------
 src/kudu/tserver/tserver_path_handlers.h  |   8 +-
 www/tablets.mustache                      | 113 ++++++++++
 4 files changed, 256 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0e19cf77/src/kudu/tserver/tablet_server-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index 6dba667..436c80f 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -400,13 +400,13 @@ TEST_F(TabletServerTest, TestWebPages) {
   ASSERT_OK(c.FetchURL(Substitute("http://$0/tablets", addr),
                               &buf));
   ASSERT_STR_CONTAINS(buf.ToString(), kTabletId);
-  ASSERT_STR_CONTAINS(buf.ToString(), "RANGE (key) PARTITION UNBOUNDED</td>");
+  ASSERT_STR_CONTAINS(buf.ToString(), "RANGE (key) PARTITION UNBOUNDED");
 
   // Tablet page should include the schema.
   ASSERT_OK(c.FetchURL(Substitute("http://$0/tablet?id=$1", addr, kTabletId),
                        &buf));
-  ASSERT_STR_CONTAINS(buf.ToString(), "<th><u>key</u></th>");
-  ASSERT_STR_CONTAINS(buf.ToString(), "<td>string NULLABLE</td>");
+  ASSERT_STR_CONTAINS(buf.ToString(), "key");
+  ASSERT_STR_CONTAINS(buf.ToString(), "string NULLABLE");
 
   // Test fetching metrics.
   // Fetching metrics has the side effect of retiring metrics, but not in a single pass.
@@ -505,8 +505,8 @@ TEST_F(TabletServerTest, TestFailedTabletsRejectConsensusState) {
   ASSERT_STR_CONTAINS(s.ToString(), "Tablet replica is shutdown");
 }
 
-// Test that tablets that get failed and deleted will eventually show up as
-// failed tombstones on the web UI.
+// Test that tablet replicas that get failed and deleted will eventually show
+// up as failed tombstones on the web UI.
 TEST_F(TabletServerTest, TestFailedTabletsOnWebUI) {
   scoped_refptr<TabletReplica> replica;
   TSTabletManager* tablet_manager = mini_server_->server()->tablet_manager();
@@ -515,7 +515,7 @@ TEST_F(TabletServerTest, TestFailedTabletsOnWebUI) {
   replica->Shutdown();
   ASSERT_EQ(tablet::FAILED, replica->state());
 
-  // Now delete the tablet and leave it tombstoned, e.g. as if the failed
+  // Now delete the replica and leave it tombstoned, e.g. as if the failed
   // replica were deleted.
   TabletServerErrorPB::Code error_code;
   ASSERT_OK(tablet_manager->DeleteTablet(kTabletId,
@@ -526,9 +526,8 @@ TEST_F(TabletServerTest, TestFailedTabletsOnWebUI) {
   const string addr = mini_server_->bound_http_addr().ToString();
   ASSERT_OK(c.FetchURL(Substitute("http://$0/tablets", addr), &buf));
 
-  // Ensure the html contains the "Tombstoned Tablets" header, indicating the
-  // failed tombstone is correctly displayed as a tombstone.
-  ASSERT_STR_CONTAINS(buf.ToString(), "Tombstoned Tablets");
+  // The webui should have a record of a FAILED and tombstoned tablet replica.
+  ASSERT_STR_CONTAINS(buf.ToString(), "FAILED (TABLET_DATA_TOMBSTONED)");
 }
 
 // Test that tombstoned tablets are displayed correctly in the web ui:
@@ -552,14 +551,13 @@ TEST_F(TabletServerTest, TestTombstonedTabletOnWebUI) {
   const string addr = mini_server_->bound_http_addr().ToString();
   ASSERT_OK(c.FetchURL(Substitute("http://$0/tablets", addr), &buf));
 
-  // Ensure the html contains the "Tombstoned Tablets" header and
-  // a table entry with the proper status message.
+  // Check the page contains a tombstoned tablet, and its state is not
+  // "Tablet initializing...".
   string s = buf.ToString();
-  ASSERT_STR_CONTAINS(s, "Tombstoned Tablets");
-  ASSERT_STR_CONTAINS(s, "<td>Tombstoned</td>");
-  ASSERT_STR_NOT_CONTAINS(s, "<td>Tablet initializing...</td>");
+  ASSERT_STR_CONTAINS(s, "TABLET_DATA_TOMBSTONED");
+  ASSERT_STR_NOT_CONTAINS(s, "Tablet initializing...");
 
-  // Since the consensus config shouldn't be displayed, the html should not
+  // Since the consensus config shouldn't be displayed, the page should not
   // contain the server's RPC address.
   ASSERT_STR_NOT_CONTAINS(s, mini_server_->bound_rpc_addr().ToString());
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/0e19cf77/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 9723b4a..b8071ed 100644
--- a/src/kudu/tserver/tserver_path_handlers.cc
+++ b/src/kudu/tserver/tserver_path_handlers.cc
@@ -93,6 +93,88 @@ class Schema;
 
 namespace tserver {
 
+namespace {
+
+bool CompareByMemberType(const RaftPeerPB& a, const RaftPeerPB& b) {
+  if (!a.has_member_type()) return false;
+  if (!b.has_member_type()) return true;
+  return a.member_type() < b.member_type();
+}
+
+string ConsensusStatePBToHtml(const ConsensusStatePB& cstate,
+                              const string& local_uuid) {
+  ostringstream html;
+
+  html << "<ul>\n";
+  std::vector<RaftPeerPB> sorted_peers;
+  sorted_peers.assign(cstate.committed_config().peers().begin(),
+                      cstate.committed_config().peers().end());
+  std::sort(sorted_peers.begin(), sorted_peers.end(), &CompareByMemberType);
+  for (const RaftPeerPB& peer : sorted_peers) {
+    string peer_addr_or_uuid =
+        peer.has_last_known_addr() ? Substitute("$0:$1",
+                                                peer.last_known_addr().host(),
+                                                peer.last_known_addr().port())
+                                   : peer.permanent_uuid();
+    peer_addr_or_uuid = EscapeForHtmlToString(peer_addr_or_uuid);
+    string role_name = RaftPeerPB::Role_Name(GetConsensusRole(peer.permanent_uuid(), cstate));
+    string formatted = Substitute("$0: $1", role_name, peer_addr_or_uuid);
+    // Make the local peer bold.
+    if (peer.permanent_uuid() == local_uuid) {
+      formatted = Substitute("<b>$0</b>", formatted);
+    }
+
+    html << Substitute(" <li>$0</li>\n", formatted);
+  }
+  html << "</ul>\n";
+  return html.str();
+}
+
+bool GetTabletID(const Webserver::WebRequest& req,
+                 string* id,
+                 Webserver::PrerenderedWebResponse* resp) {
+  if (!FindCopy(req.parsed_args, "id", id)) {
+    resp->status_code = HttpStatusCode::BadRequest;
+    *resp->output << "Tablet missing 'id' argument";
+    return false;
+  }
+  return true;
+}
+
+bool GetTabletReplica(TabletServer* tserver, const Webserver::WebRequest& /*req*/,
+                      scoped_refptr<TabletReplica>* replica, const string& tablet_id,
+                      Webserver::PrerenderedWebResponse* resp) {
+  if (!tserver->tablet_manager()->LookupTablet(tablet_id, replica)) {
+    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,
+                         Webserver::PrerenderedWebResponse* resp) {
+  if (replica->state() == tablet::BOOTSTRAPPING) {
+    resp->status_code = HttpStatusCode::ServiceUnavailable;
+    *resp->output << "Tablet " << EscapeForHtmlToString(tablet_id) << " is still bootstrapping";
+    return true;
+  }
+  return false;
+}
+
+// Returns true if the tablet_id was properly specified, the
+// tablet is found, and is in a non-bootstrapping state.
+bool LoadTablet(TabletServer* tserver,
+                const Webserver::WebRequest& req,
+                string* tablet_id, scoped_refptr<TabletReplica>* replica,
+                Webserver::PrerenderedWebResponse* resp) {
+  return GetTabletID(req, tablet_id, resp) &&
+      GetTabletReplica(tserver, req, replica, *tablet_id, resp) &&
+      !TabletBootstrapping(*replica, *tablet_id, resp);
+}
+
+} // anonymous namespace
+
 TabletServerPathHandlers::~TabletServerPathHandlers() {
 }
 
@@ -101,7 +183,7 @@ Status TabletServerPathHandlers::Register(Webserver* server) {
     "/scans", "Scans",
     boost::bind(&TabletServerPathHandlers::HandleScansPage, this, _1, _2),
     true /* styled */, false /* is_on_nav_bar */);
-  server->RegisterPrerenderedPathHandler(
+  server->RegisterPathHandler(
     "/tablets", "Tablets",
     boost::bind(&TabletServerPathHandlers::HandleTabletsPage, this, _1, _2),
     true /* styled */, true /* is_on_nav_bar */);
@@ -214,11 +296,16 @@ bool IsTombstoned(const scoped_refptr<TabletReplica>& replica) {
 } // anonymous namespace
 
 void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*req*/,
-                                                 Webserver::PrerenderedWebResponse* resp) {
-  ostringstream* output = resp->output;
+                                                 Webserver::WebResponse* resp) {
+  EasyJson* output = resp->output;
   vector<scoped_refptr<TabletReplica>> replicas;
   tserver_->tablet_manager()->GetTabletReplicas(&replicas);
 
+  if (replicas.empty()) {
+    (*output)["no_replicas"] = true;
+    return;
+  }
+
   // Sort by (table_name, tablet_id) tuples.
   std::sort(replicas.begin(), replicas.end(),
             [](const scoped_refptr<TabletReplica>& rep_a,
@@ -227,88 +314,61 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
                      std::make_pair(rep_b->tablet_metadata()->table_name(), rep_b->tablet_id());
             });
 
-  // For assigning ids to table divs;
-  int i = 0;
-  auto generate_table = [this, &i](const vector<scoped_refptr<TabletReplica>>& replicas,
-                                   std::ostream* output) {
-    i++;
-
-    *output << "<h4>Summary</h4>\n";
-    map<string, int> tablet_statuses;
+  // Populate the JSON object 'replicas_json' with information about the replicas
+  // in 'replicas'
+  const auto& local_uuid = tserver_->instance_pb().permanent_uuid();
+  auto make_replicas_json =
+      [&local_uuid](const vector<scoped_refptr<TabletReplica>>& replicas,
+                    EasyJson* replicas_json) {
+    map<string, int> statuses;
     for (const scoped_refptr<TabletReplica>& replica : replicas) {
-      tablet_statuses[TabletStatePB_Name(replica->state())]++;
+      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) {
+    EasyJson statuses_json = replicas_json->Set("statuses", EasyJson::kArray);
+    for (const auto& entry : statuses) {
+      EasyJson status_json = statuses_json.PushBack(EasyJson::kObject);
       double percent = replicas.empty() ? 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));
+      status_json["status"] = entry.first;
+      status_json["count"] = entry.second;
+      status_json["percentage"] = 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";
+    (*replicas_json)["total_count"] = std::to_string(replicas.size());
+
+    EasyJson details_json = replicas_json->Set("replicas", EasyJson::kArray);
     for (const scoped_refptr<TabletReplica>& replica : replicas) {
+      EasyJson replica_json = details_json.PushBack(EasyJson::kObject);
+      const auto* tablet = replica->tablet();
+      const auto& tmeta = replica->tablet_metadata();
       TabletStatusPB status;
       replica->GetTabletStatusPB(&status);
-      string id = status.tablet_id();
-      string table_name = status.table_name();
-      string tablet_id_or_link;
-      if (replica->tablet() != nullptr) {
-        tablet_id_or_link = TabletLink(id);
+      replica_json["table_name"] = status.table_name();
+      if (tablet != nullptr) {
+        replica_json["id_or_link"] = TabletLink(status.tablet_id());
+        replica_json["mem_bytes"] = HumanReadableNumBytes::ToString(
+            tablet->mem_tracker()->consumption());
       } else {
-        tablet_id_or_link = EscapeForHtmlToString(id);
-      }
-      string mem_bytes = "";
-      if (replica->tablet() != nullptr) {
-        mem_bytes = HumanReadableNumBytes::ToString(
-            replica->tablet()->mem_tracker()->consumption());
+        replica_json["id_or_link"] = status.tablet_id();
       }
-      string n_bytes = "";
+      replica_json["partition"] =
+          tmeta->partition_schema().PartitionDebugString(tmeta->partition(),
+                                                         tmeta->schema());
+      replica_json["state"] = replica->HumanReadableState();
       if (status.has_estimated_on_disk_size()) {
-        n_bytes = HumanReadableNumBytes::ToString(status.estimated_on_disk_size());
+        replica_json["n_bytes"] =
+            HumanReadableNumBytes::ToString(status.estimated_on_disk_size());
       }
-      string partition = replica->tablet_metadata()
-                                ->partition_schema()
-                                 .PartitionDebugString(replica->tablet_metadata()->partition(),
-                                                       replica->tablet_metadata()->schema());
-
       // We don't show the config if it's a tombstone because it's misleading.
       string consensus_state_html;
       shared_ptr<consensus::RaftConsensus> consensus = replica->shared_consensus();
       if (!IsTombstoned(replica) && consensus) {
         ConsensusStatePB cstate;
         if (consensus->ConsensusState(&cstate).ok()) {
-          consensus_state_html = ConsensusStatePBToHtml(cstate);
+          replica_json["consensus_state_html"] = ConsensusStatePBToHtml(cstate,
+                                                                        local_uuid);
         }
       }
-
-      *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><td>$7</td></tr>\n",
-          EscapeForHtmlToString(table_name), // $0
-          tablet_id_or_link, // $1
-          EscapeForHtmlToString(partition), // $2
-          EscapeForHtmlToString(replica->HumanReadableState()), mem_bytes, n_bytes, // $3, $4, $5
-          consensus_state_html, // $6
-          EscapeForHtmlToString(status.last_status())); // $7
+      replica_json["last_status"] = status.last_status();
     }
-    *output << "<tbody></table>\n</div>\n";
   };
 
   vector<scoped_refptr<TabletReplica>> live_replicas;
@@ -322,104 +382,15 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
   }
 
   if (!live_replicas.empty()) {
-    *output << "<h3>Live Tablets</h3>\n";
-    generate_table(live_replicas, output);
+    EasyJson live_replicas_json = output->Set("live_replicas", EasyJson::kObject);
+    make_replicas_json(live_replicas, &live_replicas_json);
   }
   if (!tombstoned_replicas.empty()) {
-    *output << "<h3>Tombstoned Tablets</h3>\n";
-    *output << "<p><small>Tombstone tablets are necessary for correct operation "
-               "of Kudu. These tablets have had all of their data removed from "
-               "disk and do not consume significant resources, and must not be "
-               "deleted.</small></p>";
-    generate_table(tombstoned_replicas, output);
-  }
-}
-
-namespace {
-
-bool CompareByMemberType(const RaftPeerPB& a, const RaftPeerPB& b) {
-  if (!a.has_member_type()) return false;
-  if (!b.has_member_type()) return true;
-  return a.member_type() < b.member_type();
-}
-
-} // anonymous namespace
-
-string TabletServerPathHandlers::ConsensusStatePBToHtml(const ConsensusStatePB& cstate) const {
-  ostringstream html;
-
-  html << "<ul>\n";
-  std::vector<RaftPeerPB> sorted_peers;
-  sorted_peers.assign(cstate.committed_config().peers().begin(),
-                      cstate.committed_config().peers().end());
-  std::sort(sorted_peers.begin(), sorted_peers.end(), &CompareByMemberType);
-  for (const RaftPeerPB& peer : sorted_peers) {
-    string peer_addr_or_uuid =
-        peer.has_last_known_addr() ? Substitute("$0:$1",
-                                                peer.last_known_addr().host(),
-                                                peer.last_known_addr().port())
-                                   : peer.permanent_uuid();
-    peer_addr_or_uuid = EscapeForHtmlToString(peer_addr_or_uuid);
-    string role_name = RaftPeerPB::Role_Name(GetConsensusRole(peer.permanent_uuid(), cstate));
-    string formatted = Substitute("$0: $1", role_name, peer_addr_or_uuid);
-    // Make the local peer bold.
-    if (peer.permanent_uuid() == tserver_->instance_pb().permanent_uuid()) {
-      formatted = Substitute("<b>$0</b>", formatted);
-    }
-
-    html << Substitute(" <li>$0</li>\n", formatted);
-  }
-  html << "</ul>\n";
-  return html.str();
-}
-
-namespace {
-
-bool GetTabletID(const Webserver::WebRequest& req,
-                 string* id,
-                 Webserver::PrerenderedWebResponse* resp) {
-  if (!FindCopy(req.parsed_args, "id", id)) {
-    resp->status_code = HttpStatusCode::BadRequest;
-    *resp->output << "Tablet missing 'id' argument";
-    return false;
-  }
-  return true;
-}
-
-bool GetTabletReplica(TabletServer* tserver, const Webserver::WebRequest& /*req*/,
-                      scoped_refptr<TabletReplica>* replica, const string& tablet_id,
-                      Webserver::PrerenderedWebResponse* resp) {
-  if (!tserver->tablet_manager()->LookupTablet(tablet_id, replica)) {
-    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,
-                         Webserver::PrerenderedWebResponse* resp) {
-  if (replica->state() == tablet::BOOTSTRAPPING) {
-    resp->status_code = HttpStatusCode::ServiceUnavailable;
-    *resp->output << "Tablet " << EscapeForHtmlToString(tablet_id) << " is still bootstrapping";
-    return true;
+    EasyJson tombstoned_replicas_json = output->Set("tombstoned_replicas", EasyJson::kObject);
+    make_replicas_json(tombstoned_replicas, &tombstoned_replicas_json);
   }
-  return false;
-}
-
-// Returns true if the tablet_id was properly specified, the
-// tablet is found, and is in a non-bootstrapping state.
-bool LoadTablet(TabletServer* tserver,
-                const Webserver::WebRequest& req,
-                string* tablet_id, scoped_refptr<TabletReplica>* replica,
-                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,
                                                 Webserver::PrerenderedWebResponse* resp) {
   ostringstream* output = resp->output;

http://git-wip-us.apache.org/repos/asf/kudu/blob/0e19cf77/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 cb7c88f..bd9124e 100644
--- a/src/kudu/tserver/tserver_path_handlers.h
+++ b/src/kudu/tserver/tserver_path_handlers.h
@@ -24,11 +24,6 @@
 #include "kudu/util/status.h"
 
 namespace kudu {
-
-namespace consensus {
-class ConsensusStatePB;
-} // namespace consensus
-
 namespace tserver {
 
 class TabletServer;
@@ -47,7 +42,7 @@ class TabletServerPathHandlers {
   void HandleScansPage(const Webserver::WebRequest& req,
                        Webserver::WebResponse* resp);
   void HandleTabletsPage(const Webserver::WebRequest& req,
-                         Webserver::PrerenderedWebResponse* resp);
+                         Webserver::WebResponse* resp);
   void HandleTabletPage(const Webserver::WebRequest& req,
                         Webserver::PrerenderedWebResponse* resp);
   void HandleTransactionsPage(const Webserver::WebRequest& req,
@@ -62,7 +57,6 @@ class TabletServerPathHandlers {
                             Webserver::PrerenderedWebResponse* resp);
   void HandleMaintenanceManagerPage(const Webserver::WebRequest& req,
                                     Webserver::WebResponse* resp);
-  std::string ConsensusStatePBToHtml(const consensus::ConsensusStatePB& cstate) const;
   std::string GetDashboardLine(const std::string& link,
                                const std::string& text, const std::string& desc);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/0e19cf77/www/tablets.mustache
----------------------------------------------------------------------
diff --git a/www/tablets.mustache b/www/tablets.mustache
new file mode 100644
index 0000000..c66e12f
--- /dev/null
+++ b/www/tablets.mustache
@@ -0,0 +1,113 @@
+{{!
+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.
+}}
+{{#no_replicas}}
+There are no tablet replicas.
+{{/no_replicas}}
+{{#live_replicas}}
+  <h3>Live Tablet Replicas</h3>
+  <h4>Summary</h4>
+  <table class='table table-striped table-hover'>
+    <thead><tr><th>Status</th><th>Count</th><th>Percentage</th></tr></thead>
+    <tbody>
+    {{#statuses}}
+      <td>{{status}}</td><td>{{count}}</td><td>{{percentage}}</td>
+    {{/statuses}}
+    </tbody>
+    <tfoot><tr><td>Total</td><td>{{total_count}}</td><td></td></tfoot>
+  </table>
+  <h4>Detail</h4>
+  <a href='#detail_live' data-toggle='collapse'>(toggle)</a>
+  <div id='detail_live' class='collapse'>
+    <table class='table table-striped table-hover'>
+      <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>
+      <tbody>
+      {{#replicas}}
+        <tr>
+          <td>{{table_name}}</td>
+          <td>{{{id_or_link}}}</td>
+          <td>{{partition}}</td>
+          <td>{{state}}</td>
+          <td>{{mem_bytes}}</td>
+          <td>{{n_bytes}}</td>
+          <td>{{{consensus_state_html}}}</td>
+          <td>{{last_status}}</td>
+        </tr>
+      {{/replicas}}
+      </tbody>
+    </table>
+  </div>
+{{/live_replicas}}
+
+{{#tombstoned_replicas}}
+  <h3>Tombstoned Tablet Replicas</h3>
+  <p><small>
+    Tombstoned tablet replicas are necessary for correct operation of Kudu.
+    These tablet replicas have had all of their data removed from disk and do
+    not consume significant resources, and must not be deleted.
+  </small></p>
+  <h4>Summary</h4>
+  <table class='table table-striped table-hover'>
+    <thead><tr><th>Status</th><th>Count</th><th>Percentage</th></tr></thead>
+    <tbody>
+    {{#replica_statuses}}
+      <td>{{status}}</td><td>{{count}}</td><td>{{percentage}}</td>
+    {{/replica_statuses}}
+    </tbody>
+    <tfoot><tr><td>Total</td><td>{{total_count}}</td><td></td></tfoot>
+  </table>
+  <h4>Detail</h4>
+  <a href='#detail_tombstoned' data-toggle='collapse'>(toggle)</a>
+  <div id='detail_tombstoned' class='collapse'>
+    <table class='table table-striped table-hover'>
+      <thead>
+        <tr>
+          <th>Table name</th>
+          <th>Tablet ID</th>
+          <th>Partition</th>
+          <th>State</th>
+          <th>On-disk size</th>
+          <th>Last status</th>
+        </tr>
+      </thead>
+      <tbody>
+      {{#replicas}}
+        <tr>
+          <td>{{table_name}}</td>
+          <td>{{{id_or_link}}}</td>
+          <td>{{partition}}</td>
+          <td>{{state}}</td>
+          <td>{{n_bytes}}</td>
+          <td>{{last_status}}</td>
+        </tr>
+      {{/replicas}}
+      </tbody>
+    </table>
+  </div>
+{{#tombstoned_replicas}}