You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2020/02/08 17:44:26 UTC

[impala] 04/04: IMPALA-9243: Add info about blacklisting decisions to the webui

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 014f973e927a74b4d6020f756fe548e6a7b84ff5
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Wed Feb 5 14:58:41 2020 -0800

    IMPALA-9243: Add info about blacklisting decisions to the webui
    
    This patch adds information about blacklisting decisions to the
    /backends webui endpoint.
    
    For the JSON, it adds an 'is_blacklisted' field to all backends, and
    for and backends where 'is_blacklisted' is true it adds a
    'blacklist_cause' field indicating the error status that led to the
    backend getting blacklisted and an 'blacklist_time_remaining' field
    indiciating how much longer the backend will remain on the blacklist.
    It also adds counts for the number of blacklisted and quiescing
    backends, if any, and the number of active (i.e. all other) backends.
    
    For display, in order to prevent the table of backend information from
    having too many columns (prior to this patch it already had 12), it
    separates blacklisted, quiescing, and active backends into three
    separate table, with the blacklisted and quiescing tables only getting
    displayed if there are any such backends.
    
    Additionally, tooltips are added next to the headers for the
    blacklisted and quiescing tables that provide a brief explanation of
    what it means for a backend to appear on there lists.
    
    Using separate tables also facilitates having state-specific columns -
    the blacklisted table displays columns for the blacklist cause and
    time remaining. Future work could consider adding columns to the
    quiescing table, such as time until the grace period and deadline
    expires.
    
    Testing:
    - Manually ran various quiescing/blacklisting scenarios and confirmed
      the /backends page displays as expected.
    - Added cases to test_web_pages (to verify the new fields when nothing
      is blacklisted) and test_blacklist.
    
    Change-Id: Ia0c309315b142a50be102dcb516b36ec6cb3cf47
    Reviewed-on: http://gerrit.cloudera.org:8080/15178
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/coordinator.cc                    |  11 +--
 be/src/scheduling/cluster-membership-mgr-test.cc |  12 +--
 be/src/scheduling/cluster-membership-mgr.cc      |   9 +-
 be/src/scheduling/cluster-membership-mgr.h       |   3 +-
 be/src/scheduling/executor-blacklist.cc          |  14 ++-
 be/src/scheduling/executor-blacklist.h           |  21 +++--
 be/src/service/impala-http-handler.cc            |  40 +++++++++
 bin/rat_exclude_files.txt                        |   2 +
 tests/custom_cluster/test_blacklist.py           |  14 +++
 tests/webserver/test_web_pages.py                |   4 +
 www/backends.tmpl                                | 104 ++++++++++++++++++++++-
 www/blacklisted_tooltip.txt                      |   1 +
 www/quiescing_tooltip.txt                        |   1 +
 13 files changed, 208 insertions(+), 28 deletions(-)

diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 94a85e5..486a810 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -487,11 +487,9 @@ Status Coordinator::FinishBackendStartup() {
     }
     if (!backend_state->exec_rpc_status().ok()) {
       // The Exec() rpc failed, so blacklist the executor.
-      LOG(INFO) << "Blacklisting "
-                << TNetworkAddressToString(backend_state->impalad_address())
-                << " because an Exec() rpc to it failed.";
       const TBackendDescriptor& be_desc = backend_state->exec_params()->be_desc;
-      ExecEnv::GetInstance()->cluster_membership_mgr()->BlacklistExecutor(be_desc);
+      ExecEnv::GetInstance()->cluster_membership_mgr()->BlacklistExecutor(
+          be_desc, backend_state->exec_rpc_status());
     }
     if (backend_state->rpc_latency() > max_latency) {
       // Find the backend that takes the most time to acknowledge to
@@ -942,10 +940,9 @@ void Coordinator::UpdateBlacklistWithAuxErrorInfo(
 
       if (blacklistable_rpc_error_codes.find(rpc_error_info.posix_error_code())
           != blacklistable_rpc_error_codes.end()) {
-        LOG(INFO) << "Blacklisting " << NetworkAddressPBToString(dest_node)
-                  << " because a RPC to it failed.";
         ExecEnv::GetInstance()->cluster_membership_mgr()->BlacklistExecutor(
-            dest_node_exec_params->be_desc);
+            dest_node_exec_params->be_desc,
+            Status(Substitute("RPC to $0 failed", NetworkAddressPBToString(dest_node))));
         break;
       }
     }
diff --git a/be/src/scheduling/cluster-membership-mgr-test.cc b/be/src/scheduling/cluster-membership-mgr-test.cc
index e3e24d1..acebe10 100644
--- a/be/src/scheduling/cluster-membership-mgr-test.cc
+++ b/be/src/scheduling/cluster-membership-mgr-test.cc
@@ -376,17 +376,17 @@ TEST_F(ClusterMembershipMgrTest, ExecutorBlacklist) {
   }
 
   // Tell a BE to blacklist itself, should have no effect.
-  backends_[0]->cmm->BlacklistExecutor(*backends_[0]->desc);
+  backends_[0]->cmm->BlacklistExecutor(*backends_[0]->desc, Status("error"));
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS, GetDefaultGroupSize(*backends_[0]->cmm));
 
   // Tell a BE to blacklist another BE, should remove it from executor_groups but not
   // current_backends.
-  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc);
+  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc, Status("error"));
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS - 1, GetDefaultGroupSize(*backends_[0]->cmm));
   // Blacklist a BE that is already blacklisted. Should have no effect.
-  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc);
+  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc, Status("error"));
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS - 1, GetDefaultGroupSize(*backends_[0]->cmm));
 
@@ -397,7 +397,7 @@ TEST_F(ClusterMembershipMgrTest, ExecutorBlacklist) {
   EXPECT_EQ(NUM_BACKENDS, GetDefaultGroupSize(*backends_[0]->cmm));
 
   // Blacklist the BE and sleep again.
-  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc);
+  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc, Status("error"));
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS - 1, GetDefaultGroupSize(*backends_[0]->cmm));
   usleep(BLACKLIST_TIMEOUT_SLEEP_US);
@@ -407,12 +407,12 @@ TEST_F(ClusterMembershipMgrTest, ExecutorBlacklist) {
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS - 1, GetDefaultGroupSize(*backends_[0]->cmm));
   // Try blacklisting the quiesced BE, should have no effect.
-  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc);
+  backends_[0]->cmm->BlacklistExecutor(*backends_[1]->desc, Status("error"));
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS - 1, GetDefaultGroupSize(*backends_[0]->cmm));
 
   // Blacklist another BE and sleep.
-  backends_[0]->cmm->BlacklistExecutor(*backends_[2]->desc);
+  backends_[0]->cmm->BlacklistExecutor(*backends_[2]->desc, Status("error"));
   EXPECT_EQ(NUM_BACKENDS, backends_[0]->cmm->GetSnapshot()->current_backends.size());
   EXPECT_EQ(NUM_BACKENDS - 2, GetDefaultGroupSize(*backends_[0]->cmm));
   usleep(BLACKLIST_TIMEOUT_SLEEP_US);
diff --git a/be/src/scheduling/cluster-membership-mgr.cc b/be/src/scheduling/cluster-membership-mgr.cc
index aed7d7d..90c57df 100644
--- a/be/src/scheduling/cluster-membership-mgr.cc
+++ b/be/src/scheduling/cluster-membership-mgr.cc
@@ -346,7 +346,9 @@ void ClusterMembershipMgr::UpdateMembership(
   recovering_membership_.reset();
 }
 
-void ClusterMembershipMgr::BlacklistExecutor(const TBackendDescriptor& be_desc) {
+void ClusterMembershipMgr::BlacklistExecutor(
+    const TBackendDescriptor& be_desc, const Status& cause) {
+  DCHECK(!cause.ok());
   if (!ExecutorBlacklist::BlacklistingEnabled()) return;
   lock_guard<mutex> l(update_membership_lock_);
   // Don't blacklist the local executor. Some queries may have root fragments that must be
@@ -382,6 +384,9 @@ void ClusterMembershipMgr::BlacklistExecutor(const TBackendDescriptor& be_desc)
     return;
   }
 
+  LOG(INFO) << "Blacklisting " << TNetworkAddressToString(be_desc.address) << ": "
+            << cause;
+
   std::shared_ptr<Snapshot> new_state;
   if (recovering) {
     // If the statestore is currently recovering, we can apply the blacklisting to
@@ -399,7 +404,7 @@ void ClusterMembershipMgr::BlacklistExecutor(const TBackendDescriptor& be_desc)
   }
 
   ExecutorBlacklist* new_blacklist = &(new_state->executor_blacklist);
-  new_blacklist->Blacklist(be_desc);
+  new_blacklist->Blacklist(be_desc, cause);
 
   // We'll call SetState() with 'recovering_membership_' once the statestore is no longer
   // in recovery.
diff --git a/be/src/scheduling/cluster-membership-mgr.h b/be/src/scheduling/cluster-membership-mgr.h
index 7ee249d..8305b13 100644
--- a/be/src/scheduling/cluster-membership-mgr.h
+++ b/be/src/scheduling/cluster-membership-mgr.h
@@ -164,7 +164,8 @@ class ClusterMembershipMgr {
 
   /// Adds the given backend to the local blacklist. Updates 'current_membership_' to
   /// remove the backend from 'executor_groups' so that it will not be scheduled on.
-  void BlacklistExecutor(const TBackendDescriptor& be_desc);
+  /// 'cause' is an error status representing the reason the node was blacklisted.
+  void BlacklistExecutor(const TBackendDescriptor& be_desc, const Status& cause);
 
  private:
   /// Serializes and adds the local backend descriptor to 'subscriber_topic_updates'.
diff --git a/be/src/scheduling/executor-blacklist.cc b/be/src/scheduling/executor-blacklist.cc
index 58035d5..206fb2b 100644
--- a/be/src/scheduling/executor-blacklist.cc
+++ b/be/src/scheduling/executor-blacklist.cc
@@ -36,7 +36,8 @@ bool ExecutorBlacklist::BlacklistingEnabled() {
   return FLAGS_blacklisting_enabled;
 }
 
-void ExecutorBlacklist::Blacklist(const TBackendDescriptor& be_desc) {
+void ExecutorBlacklist::Blacklist(
+    const TBackendDescriptor& be_desc, const Status& cause) {
   DCHECK(BlacklistingEnabled());
   DCHECK(!be_desc.ip_address.empty());
   vector<Entry>& be_descs = executor_list_[be_desc.ip_address];
@@ -49,6 +50,7 @@ void ExecutorBlacklist::Blacklist(const TBackendDescriptor& be_desc) {
       // This executor was on probation, so re-blacklist it.
       it->state = State::BLACKLISTED;
       it->blacklist_time_ms = MonotonicMillis();
+      it->cause = cause;
 
       int64_t probation_timeout = GetBlacklistTimeoutMs() * PROBATION_TIMEOUT_MULTIPLIER;
       int64_t elapsed = MonotonicMillis() - it->blacklist_time_ms;
@@ -64,7 +66,7 @@ void ExecutorBlacklist::Blacklist(const TBackendDescriptor& be_desc) {
     }
   } else {
     // This executor was not already on the list, create a new Entry for it.
-    be_descs.emplace_back(be_desc, MonotonicMillis());
+    be_descs.emplace_back(be_desc, MonotonicMillis(), cause);
   }
   VLOG(2) << "Blacklisted " << TNetworkAddressToString(be_desc.address)
           << ", current blacklist: " << DebugString();
@@ -145,12 +147,18 @@ void ExecutorBlacklist::Maintenance(std::list<TBackendDescriptor>* probation_lis
   VLOG(2) << "Completed blacklist maintenance. Current blacklist: " << DebugString();
 }
 
-bool ExecutorBlacklist::IsBlacklisted(const TBackendDescriptor& be_desc) const {
+bool ExecutorBlacklist::IsBlacklisted(
+    const TBackendDescriptor& be_desc, Status* cause, int64_t* time_remaining_ms) const {
   for (auto executor_it : executor_list_) {
     if (executor_it.first == be_desc.ip_address) {
       for (auto entry_it : executor_it.second) {
         if (entry_it.be_desc.address.port == be_desc.address.port
             && entry_it.state == State::BLACKLISTED) {
+          if (cause != nullptr) *cause = entry_it.cause;
+          int64_t elapsed_ms = MonotonicMillis() - entry_it.blacklist_time_ms;
+          int64_t total_timeout_ms =
+              GetBlacklistTimeoutMs() * entry_it.num_consecutive_blacklistings;
+          *time_remaining_ms = total_timeout_ms - elapsed_ms;
           return true;
         }
       }
diff --git a/be/src/scheduling/executor-blacklist.h b/be/src/scheduling/executor-blacklist.h
index a8adcc7..4d14b69 100644
--- a/be/src/scheduling/executor-blacklist.h
+++ b/be/src/scheduling/executor-blacklist.h
@@ -64,8 +64,9 @@ class ExecutorBlacklist {
 
   /// Adds an executor to the blacklist, if it is not already blacklisted. If the executor
   /// was on probation, updates its entry in 'executor_list_' accordingly. Should only be
-  /// called if BlacklistingEnabled() is true.
-  void Blacklist(const TBackendDescriptor& be_desc);
+  /// called if BlacklistingEnabled() is true. 'cause' is an error status indicating why
+  /// the executor is being blacklisted.
+  void Blacklist(const TBackendDescriptor& be_desc, const Status& cause);
 
   /// Removes an executor from the blacklist or probation, if it is in 'executor_list_'.
   /// Does not put blacklisted executors on probation. Returns the executor's state prior
@@ -87,8 +88,11 @@ class ExecutorBlacklist {
   /// probation.
   void Maintenance(std::list<TBackendDescriptor>* probation_list);
 
-  /// Returns true if 'be_desc' is blacklisted.
-  bool IsBlacklisted(const TBackendDescriptor& be_desc) const;
+  /// If 'be_desc' is blacklisted, sets 'cause' to the error Status that caused this
+  /// executor to be blacklisted, sets 'time_remaining_ms' to the amount of time the
+  /// executor has left on the blacklist, and returns true.
+  bool IsBlacklisted(const TBackendDescriptor& be_desc, Status* cause = nullptr,
+      int64_t* time_remaining_ms = nullptr) const;
 
   /// Returns a space-separated string of the addresses of executors that are currently
   /// blacklisted.
@@ -100,11 +104,13 @@ class ExecutorBlacklist {
  private:
   /// Info about an executor that is either blacklisted or on probabtion.
   struct Entry {
-    Entry(const TBackendDescriptor& be_desc, int64_t blacklist_time_ms)
+    Entry(
+        const TBackendDescriptor& be_desc, int64_t blacklist_time_ms, const Status& cause)
       : be_desc(be_desc),
         blacklist_time_ms(blacklist_time_ms),
         state(State::BLACKLISTED),
-        num_consecutive_blacklistings(1) {}
+        num_consecutive_blacklistings(1),
+        cause(cause) {}
 
     TBackendDescriptor be_desc;
 
@@ -117,6 +123,9 @@ class ExecutorBlacklist {
     /// Number of times that this executor has been blacklisted since the last time it was
     /// off probation.
     int32_t num_consecutive_blacklistings;
+
+    /// Error status representing the reason the executor was blacklisted.
+    Status cause;
   };
 
   /// Returns the base blacklist timeout in ms. This should be multiplied by
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index c3bac80..865a73b 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -919,6 +919,9 @@ void ImpalaHttpHandler::BackendsHandler(const Webserver::WebRequest& req,
   ExecEnv::GetInstance()->admission_controller()->PopulatePerHostMemReservedAndAdmitted(
       &host_stats);
   Value backends_list(kArrayType);
+  int num_active_backends = 0;
+  int num_quiescing_backends = 0;
+  int num_blacklisted_backends = 0;
   ClusterMembershipMgr* cluster_membership_mgr =
       ExecEnv::GetInstance()->cluster_membership_mgr();
   DCHECK(cluster_membership_mgr != nullptr);
@@ -943,6 +946,31 @@ void ImpalaHttpHandler::BackendsHandler(const Webserver::WebRequest& req,
         document->GetAllocator());
     backend_obj.AddMember("is_executor", backend.is_executor, document->GetAllocator());
     backend_obj.AddMember("is_quiescing", backend.is_quiescing, document->GetAllocator());
+    Status blacklist_cause;
+    int64_t blacklist_time_remaining_ms;
+    bool is_blacklisted = membership_snapshot->executor_blacklist.IsBlacklisted(
+        backend, &blacklist_cause, &blacklist_time_remaining_ms);
+    backend_obj.AddMember("is_blacklisted", is_blacklisted, document->GetAllocator());
+    backend_obj.AddMember(
+        "is_active", !is_blacklisted && !backend.is_quiescing, document->GetAllocator());
+    if (backend.is_quiescing) {
+      // Backends cannot be both blacklisted and quiescing.
+      DCHECK(!is_blacklisted);
+      ++num_quiescing_backends;
+    } else if (is_blacklisted) {
+      Value blacklist_cause_value(
+          blacklist_cause.GetDetail().c_str(), document->GetAllocator());
+      backend_obj.AddMember(
+          "blacklist_cause", blacklist_cause_value, document->GetAllocator());
+      Value blacklist_time_remaining_str(
+          Substitute("$0 s", (blacklist_time_remaining_ms / 1000)).c_str(),
+          document->GetAllocator());
+      backend_obj.AddMember("blacklist_time_remaining", blacklist_time_remaining_str,
+          document->GetAllocator());
+      ++num_blacklisted_backends;
+    } else {
+      ++num_active_backends;
+    }
     Value admit_mem_limit(PrettyPrinter::PrintBytes(backend.admit_mem_limit).c_str(),
         document->GetAllocator());
     backend_obj.AddMember("admit_mem_limit", admit_mem_limit, document->GetAllocator());
@@ -968,6 +996,18 @@ void ImpalaHttpHandler::BackendsHandler(const Webserver::WebRequest& req,
     backends_list.PushBack(backend_obj, document->GetAllocator());
   }
   document->AddMember("backends", backends_list, document->GetAllocator());
+  document->AddMember(
+      "num_active_backends", num_active_backends, document->GetAllocator());
+  // Don't add the following fields if they're 0 so that we won't display the
+  // corresponding tables if they would be empty.
+  if (num_quiescing_backends > 0) {
+    document->AddMember(
+        "num_quiescing_backends", num_quiescing_backends, document->GetAllocator());
+  }
+  if (num_blacklisted_backends > 0) {
+    document->AddMember(
+        "num_blacklisted_backends", num_blacklisted_backends, document->GetAllocator());
+  }
 }
 
 void ImpalaHttpHandler::AdmissionStateHandler(
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index a129959..a07d128 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -187,8 +187,10 @@ be/src/thirdparty/pcg-cpp-0.98/LICENSE.txt
 # http://www.apache.org/legal/src-headers.html: 'Snippet' files that are combined as form
 # a larger file where the larger file would have duplicate licensing headers.
 www/all_child_groups.tmpl
+www/blacklisted_tooltip.txt
 www/common-footer.tmpl
 www/form-hidden-inputs.tmpl
+www/quiescing_tooltip.txt
 
 # GNU tar artifact
 pax_global_header
diff --git a/tests/custom_cluster/test_blacklist.py b/tests/custom_cluster/test_blacklist.py
index f9d13cc..78ec93e 100644
--- a/tests/custom_cluster/test_blacklist.py
+++ b/tests/custom_cluster/test_blacklist.py
@@ -39,6 +39,8 @@ class TestBlacklist(CustomClusterTestSuite):
     super(TestBlacklist, cls).setup_class()
 
   @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+      statestored_args="-statestore_heartbeat_frequency_ms=1000")
   def test_kill_impalad(self, cursor):
     """Test that verifies that when an impalad is killed, it is properly blacklisted."""
     # Run a query and verify that no impalads are blacklisted yet.
@@ -59,9 +61,21 @@ class TestBlacklist(CustomClusterTestSuite):
 
     # Run another query which should succeed and verify the impalad was blacklisted.
     result = self.execute_query("select count(*) from tpch.lineitem")
+    backends_json = self.cluster.impalads[0].service.get_debug_webpage_json("/backends")
     match = re.search("Blacklisted Executors: (.*)", result.runtime_profile)
     assert match.group(1) == "%s:%s" % \
         (killed_impalad.hostname, killed_impalad.service.be_port), result.runtime_profile
+    assert backends_json["num_blacklisted_backends"] == 1, backends_json
+    assert backends_json["num_active_backends"] == 2, backends_json
+    assert len(backends_json["backends"]) == 3, backends_json
+    num_blacklisted = 0
+    for backend_json in backends_json["backends"]:
+      if str(killed_impalad.service.krpc_port) in backend_json["krpc_address"]:
+        assert backend_json["is_blacklisted"], backend_json
+        num_blacklisted += 1
+      else:
+        assert not backend_json["is_blacklisted"], backend_json
+    assert num_blacklisted == 1, backends_json
 
     # Sleep for long enough for the statestore to remove the impalad from the cluster
     # membership, i.e. Statestore::FailedExecutorDetectionTime() + some padding
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index b264fbe..d94fcfd 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -651,6 +651,9 @@ class TestWebPage(ImpalaTestSuite):
     assert 'backends' in response_json
     # When this test runs, all impalads would have already started.
     assert len(response_json['backends']) == 3
+    assert response_json['num_active_backends'] == 3
+    assert 'num_quiescing_backends' not in response_json
+    assert 'num_blacklisted_backends' not in response_json
 
     # Look at results for a single backend - they are not sorted.
     backend_row = response_json['backends'][0]
@@ -672,6 +675,7 @@ class TestWebPage(ImpalaTestSuite):
     assert backend_row['is_coordinator']
     assert backend_row['is_executor']
     assert not backend_row['is_quiescing']
+    assert not backend_row['is_blacklisted']
     assert len(backend_row['admit_mem_limit']) > 0
 
   def test_download_profile(self):
diff --git a/www/backends.tmpl b/www/backends.tmpl
index 1070993..cabadc9 100644
--- a/www/backends.tmpl
+++ b/www/backends.tmpl
@@ -18,7 +18,7 @@ under the License.
 -->
 {{> www/common-header.tmpl }}
 
-<h2>Known backend(s): {{%backends}}</h2>
+<h2>Active backend(s): {{num_active_backends}}</h2>
 
 <table id="backends" class='table table-hover table-bordered'>
   <thead>
@@ -28,7 +28,6 @@ under the License.
       <th>Krpc Address</th>
       <th>Coordinator</th>
       <th>Executor</th>
-      <th>Quiescing</th>
       <th>Memory Limit for Admission</th>
       <th>Memory Reserved</th>
       <th>Memory Admitted by Queries Submitted to this Coordinator</th>
@@ -39,13 +38,13 @@ under the License.
   </thead>
   <tbody>
     {{#backends}}
+    {{?is_active}}
     <tr>
       <td><a href='{{webserver_url}}'>Web UI</a></td>
       <td>{{address}}</td>
       <td>{{krpc_address}}</td>
       <td>{{is_coordinator}}</td>
       <td>{{is_executor}}</td>
-      <td>{{is_quiescing}}</td>
       <td>{{admit_mem_limit}}</td>
       <td>{{mem_reserved}}</td>
       <td>{{mem_admitted}}</td>
@@ -53,10 +52,97 @@ under the License.
       <td>{{num_admitted}}</td>
       <td>{{executor_groups}}</td>
     </tr>
+    {{/is_active}}
     {{/backends}}
   </tbody>
 </table>
 
+{{#num_quiescing_backends}}
+<h2>Quiescing backend(s)<sup><a href='#' data-toggle="tooltip" title="{{> www/quiescing_tooltip.txt }}">[?]</a></sup>: {{num_quiescing_backends}}</h2>
+
+<table id="quiescing_backends" class='table table-hover table-bordered'>
+  <thead>
+    <tr>
+      <th></th>
+      <th>Address</th>
+      <th>Krpc Address</th>
+      <th>Coordinator</th>
+      <th>Executor</th>
+      <th>Memory Limit for Admission</th>
+      <th>Memory Reserved</th>
+      <th>Memory Admitted by Queries Submitted to this Coordinator</th>
+      <th>Admission Control Slots In Use</th>
+      <th>Num. Queries Admitted by this Coordinator</th>
+      <th>Executor Groups</th>
+    </tr>
+  </thead>
+  <tbody>
+    {{#backends}}
+    {{?is_quiescing}}
+    <tr>
+      <td><a href='{{webserver_url}}'>Web UI</a></td>
+      <td>{{address}}</td>
+      <td>{{krpc_address}}</td>
+      <td>{{is_coordinator}}</td>
+      <td>{{is_executor}}</td>
+      <td>{{admit_mem_limit}}</td>
+      <td>{{mem_reserved}}</td>
+      <td>{{mem_admitted}}</td>
+      <td>{{admission_slots_in_use}}/{{admission_slots}}</td>
+      <td>{{num_admitted}}</td>
+      <td>{{executor_groups}}</td>
+    </tr>
+    {{/is_quiescing}}
+    {{/backends}}
+  </tbody>
+</table>
+{{/num_quiescing_backends}}
+
+{{#num_blacklisted_backends}}
+<h2>Blacklisted backend(s)<sup><a href='#' data-toggle="tooltip" title="{{> www/blacklisted_tooltip.txt }}">[?]</a></sup>: {{num_blacklisted_backends}}</h2>
+
+<table id="blacklisted_backends" class='table table-hover table-bordered'>
+  <thead>
+    <tr>
+      <th></th>
+      <th>Address</th>
+      <th>Krpc Address</th>
+      <th>Blacklisting Cause</th>
+      <th>Time remaining on blacklist</th>
+      <th>Coordinator</th>
+      <th>Executor</th>
+      <th>Memory Limit for Admission</th>
+      <th>Memory Reserved</th>
+      <th>Memory Admitted by Queries Submitted to this Coordinator</th>
+      <th>Admission Control Slots In Use</th>
+      <th>Num. Queries Admitted by this Coordinator</th>
+      <th>Executor Groups</th>
+    </tr>
+  </thead>
+  <tbody>
+    {{#backends}}
+    {{?is_blacklisted}}
+    <tr>
+      <td><a href='{{webserver_url}}'>Web UI</a></td>
+      <td>{{address}}</td>
+      <td>{{krpc_address}}</td>
+      <td>{{blacklist_cause}}</td>
+      <td>{{blacklist_time_remaining}}</td>
+      <td>{{is_coordinator}}</td>
+      <td>{{is_executor}}</td>
+      <td>{{admit_mem_limit}}</td>
+      <td>{{mem_reserved}}</td>
+      <td>{{mem_admitted}}</td>
+      <td>{{admission_slots_in_use}}/{{admission_slots}}</td>
+      <td>{{num_admitted}}</td>
+      <td>{{executor_groups}}</td>
+    </tr>
+    {{/is_blacklisted}}
+    {{/backends}}
+  </tbody>
+</table>
+{{/num_blacklisted_backends}}
+
 <script>
     $(document).ready(function() {
         $('#backends').DataTable({
@@ -64,6 +150,18 @@ under the License.
             "pageLength": 100
         });
     });
+    $(document).ready(function() {
+        $('#quiescing_backends').DataTable({
+            "order": [[ 0, "desc" ]],
+            "pageLength": 100
+        });
+    });
+    $(document).ready(function() {
+        $('#blacklisted_backends').DataTable({
+            "order": [[ 0, "desc" ]],
+            "pageLength": 100
+        });
+    });
 </script>
 
 {{> www/common-footer.tmpl}}
diff --git a/www/blacklisted_tooltip.txt b/www/blacklisted_tooltip.txt
new file mode 100644
index 0000000..c6ef983
--- /dev/null
+++ b/www/blacklisted_tooltip.txt
@@ -0,0 +1 @@
+Backends are temporarily blacklisted by a coordinator when it sees an error that may indicate the backend is non-functioning. Blacklisted backends are not considered by that coordinator when scheduling new queries.
diff --git a/www/quiescing_tooltip.txt b/www/quiescing_tooltip.txt
new file mode 100644
index 0000000..a55f324
--- /dev/null
+++ b/www/quiescing_tooltip.txt
@@ -0,0 +1 @@
+Quiescing backends are in the process of gracefully shutting down. No more queries will be scheduled on them, and they will shut down once the grace period has passed and either all running queries have completed or the shutdown deadline is reached.