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/08/10 18:23:43 UTC

kudu git commit: Web UI: switch /maintenance-manager endpoint to mustache

Repository: kudu
Updated Branches:
  refs/heads/master 7eb58663f -> ecd67486b


Web UI: switch /maintenance-manager endpoint to mustache

This cherry-picks just the changes to switch the /maintenance-manager
endpoint to use mustache by Sam Okrent in
https://gerrit.cloudera.org/#/c/7570/, with the aim of making it easier
to review the swimlane-specific changes.

Change-Id: Icf331f5d789893171608bac4d970db7680e0fcc4
Reviewed-on: http://gerrit.cloudera.org:8080/7607
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>


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

Branch: refs/heads/master
Commit: ecd67486b8ee50afc15438c8a0c4ba484f0f7362
Parents: 7eb5866
Author: Dan Burkert <da...@apache.org>
Authored: Mon Aug 7 12:15:31 2017 -0700
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Thu Aug 10 18:09:02 2017 +0000

----------------------------------------------------------------------
 src/kudu/tserver/tserver-path-handlers.cc | 71 ++++++++--------------
 src/kudu/tserver/tserver-path-handlers.h  |  3 +-
 src/kudu/util/easy_json.cc                |  8 +++
 www/maintenance-manager.mustache          | 84 ++++++++++++++++++++++++++
 4 files changed, 119 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/ecd67486/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 073d209..c53ab4c 100644
--- a/src/kudu/tserver/tserver-path-handlers.cc
+++ b/src/kudu/tserver/tserver-path-handlers.cc
@@ -40,6 +40,7 @@
 #include "kudu/tserver/scanners.h"
 #include "kudu/tserver/tablet_server.h"
 #include "kudu/tserver/ts_tablet_manager.h"
+#include "kudu/util/easy_json.h"
 #include "kudu/util/maintenance_manager.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/url-coding.h"
@@ -104,7 +105,7 @@ Status TabletServerPathHandlers::Register(Webserver* server) {
     "/dashboards", "Dashboards",
     boost::bind(&TabletServerPathHandlers::HandleDashboardsPage, this, _1, _2),
     true /* styled */, true /* is_on_nav_bar */);
-  server->RegisterPrerenderedPathHandler(
+  server->RegisterPathHandler(
     "/maintenance-manager", "",
     boost::bind(&TabletServerPathHandlers::HandleMaintenanceManagerPage, this, _1, _2),
     true /* styled */, false /* is_on_nav_bar */);
@@ -594,65 +595,43 @@ string TabletServerPathHandlers::GetDashboardLine(const std::string& link,
 }
 
 void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::WebRequest& req,
-                                                            std::ostringstream* output) {
+                                                            EasyJson* output) {
   MaintenanceManager* manager = tserver_->maintenance_manager();
   MaintenanceManagerStatusPB pb;
   manager->GetMaintenanceManagerStatusDump(&pb);
   if (ContainsKey(req.parsed_args, "raw")) {
-    *output << SecureDebugString(pb);
+    (*output)["raw"] = SecureDebugString(pb);
     return;
   }
 
-  int ops_count = pb.registered_operations_size();
-
-  *output << "<h1>Maintenance Manager state</h1>\n";
-  *output << "<h3>Running operations</h3>\n";
-  *output << "<table class='table table-striped'>\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++) {
-    const MaintenanceManagerStatusPB_MaintenanceOpPB& op_pb = pb.registered_operations(i);
+  EasyJson running_ops = output->Set("running_operations", EasyJson::kArray);
+  for (const auto& op_pb : pb.registered_operations()) {
     if (op_pb.running() > 0) {
-      *output <<  Substitute("<tr><td>$0</td><td>$1</td></tr>\n",
-                             EscapeForHtmlToString(op_pb.name()),
-                             op_pb.running());
+      EasyJson running_op = running_ops.PushBack(EasyJson::kObject);
+      running_op["name"] = op_pb.name();
+      running_op["instances_running"] = op_pb.running();
     }
   }
-  *output << "</tbody></table>\n";
 
-  *output << "<h3>Recent completed operations</h3>\n";
-  *output << "<table class='table table-striped'>\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++) {
-    const MaintenanceManagerStatusPB_OpInstancePB& op_pb = pb.completed_operations(i);
-    *output <<  Substitute("<tr><td>$0</td><td>$1</td><td>$2</td></tr>\n",
-                           EscapeForHtmlToString(op_pb.name()),
-                           HumanReadableElapsedTime::ToShortString(
-                               op_pb.duration_millis() / 1000.0),
-                           HumanReadableElapsedTime::ToShortString(
-                               op_pb.millis_since_start() / 1000.0));
+  EasyJson completed_ops = output->Set("completed_operations", EasyJson::kArray);
+  for (const auto& op_pb : pb.completed_operations()) {
+    EasyJson completed_op = completed_ops.PushBack(EasyJson::kObject);
+    completed_op["name"] = op_pb.name();
+    completed_op["duration"] =
+      HumanReadableElapsedTime::ToShortString(op_pb.duration_millis() / 1000.0);
+    completed_op["time_since_start"] =
+      HumanReadableElapsedTime::ToShortString(op_pb.millis_since_start() / 1000.0);
   }
-  *output << "</tbody></table>\n";
 
-  *output << "<h3>Non-running operations</h3>\n";
-  *output << "<table class='table table-striped'>\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++) {
-    const MaintenanceManagerStatusPB_MaintenanceOpPB& op_pb = pb.registered_operations(i);
-    if (op_pb.running() == 0) {
-      *output << Substitute("<tr><td>$0</td><td>$1</td><td>$2</td><td>$3</td><td>$4</td></tr>\n",
-                            EscapeForHtmlToString(op_pb.name()),
-                            op_pb.runnable(),
-                            HumanReadableNumBytes::ToString(op_pb.ram_anchored_bytes()),
-                            HumanReadableNumBytes::ToString(op_pb.logs_retained_bytes()),
-                            op_pb.perf_improvement());
-    }
+  EasyJson registered_ops = output->Set("registered_operations", EasyJson::kArray);
+  for (const auto& op_pb : pb.registered_operations()) {
+    EasyJson registered_op = registered_ops.PushBack(EasyJson::kObject);
+    registered_op["name"] = op_pb.name();
+    registered_op["runnable"] = op_pb.runnable();
+    registered_op["ram_anchored"] = HumanReadableNumBytes::ToString(op_pb.ram_anchored_bytes());
+    registered_op["logs_retained"] = HumanReadableNumBytes::ToString(op_pb.logs_retained_bytes());
+    registered_op["perf"] = op_pb.perf_improvement();
   }
-  *output << "</tbody></table>\n";
 }
 
 } // namespace tserver

http://git-wip-us.apache.org/repos/asf/kudu/blob/ecd67486/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 9b2b038..b37ca1c 100644
--- a/src/kudu/tserver/tserver-path-handlers.h
+++ b/src/kudu/tserver/tserver-path-handlers.h
@@ -26,6 +26,7 @@
 
 namespace kudu {
 
+class EasyJson;
 class Schema;
 struct IteratorStats;
 
@@ -66,7 +67,7 @@ class TabletServerPathHandlers {
   void HandleDashboardsPage(const Webserver::WebRequest& req,
                             std::ostringstream* output);
   void HandleMaintenanceManagerPage(const Webserver::WebRequest& req,
-                                    std::ostringstream* output);
+                                    EasyJson* output);
   std::string ConsensusStatePBToHtml(const consensus::ConsensusStatePB& cstate) const;
   std::string ScannerToHtml(const Scanner& scanner) const;
   std::string IteratorStatsToHtml(const Schema& projection,

http://git-wip-us.apache.org/repos/asf/kudu/blob/ecd67486/src/kudu/util/easy_json.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/easy_json.cc b/src/kudu/util/easy_json.cc
index 6fb2507..4b26dad 100644
--- a/src/kudu/util/easy_json.cc
+++ b/src/kudu/util/easy_json.cc
@@ -85,6 +85,8 @@ EasyJson& EasyJson::operator=(T val) {
 template EasyJson& EasyJson::operator=<bool>(bool val);
 template EasyJson& EasyJson::operator=<int32_t>(int32_t val);
 template EasyJson& EasyJson::operator=<int64_t>(int64_t val);
+template EasyJson& EasyJson::operator=<uint32_t>(uint32_t val);
+template EasyJson& EasyJson::operator=<uint64_t>(uint64_t val);
 template EasyJson& EasyJson::operator=<double>(double val);
 template<> EasyJson& EasyJson::operator=<const char*>(const char* val) {
   value_->SetString(val, alloc_->allocator());
@@ -124,6 +126,8 @@ EasyJson EasyJson::Set(const string& key, T val) {
 template EasyJson EasyJson::Set<bool>(const string& key, bool val);
 template EasyJson EasyJson::Set<int32_t>(const string& key, int32_t val);
 template EasyJson EasyJson::Set<int64_t>(const string& key, int64_t val);
+template EasyJson EasyJson::Set<uint32_t>(const string& key, uint32_t val);
+template EasyJson EasyJson::Set<uint64_t>(const string& key, uint64_t val);
 template EasyJson EasyJson::Set<double>(const string& key, double val);
 template EasyJson EasyJson::Set<const char*>(const string& key, const char* val);
 template EasyJson EasyJson::Set<EasyJson::ComplexTypeInitializer>(
@@ -140,6 +144,8 @@ EasyJson EasyJson::Set(int index, T val) {
 template EasyJson EasyJson::Set<bool>(int index, bool val);
 template EasyJson EasyJson::Set<int32_t>(int index, int32_t val);
 template EasyJson EasyJson::Set<int64_t>(int index, int64_t val);
+template EasyJson EasyJson::Set<uint32_t>(int index, uint32_t val);
+template EasyJson EasyJson::Set<uint64_t>(int index, uint64_t val);
 template EasyJson EasyJson::Set<double>(int index, double val);
 template EasyJson EasyJson::Set<const char*>(int index, const char* val);
 template EasyJson EasyJson::Set<EasyJson::ComplexTypeInitializer>(
@@ -165,6 +171,8 @@ EasyJson EasyJson::PushBack(T val) {
 template EasyJson EasyJson::PushBack<bool>(bool val);
 template EasyJson EasyJson::PushBack<int32_t>(int32_t val);
 template EasyJson EasyJson::PushBack<int64_t>(int64_t val);
+template EasyJson EasyJson::PushBack<uint32_t>(uint32_t val);
+template EasyJson EasyJson::PushBack<uint64_t>(uint64_t val);
 template EasyJson EasyJson::PushBack<double>(double val);
 template<> EasyJson EasyJson::PushBack<const char*>(const char* val) {
   if (!value_->IsArray()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/ecd67486/www/maintenance-manager.mustache
----------------------------------------------------------------------
diff --git a/www/maintenance-manager.mustache b/www/maintenance-manager.mustache
new file mode 100644
index 0000000..a906f52
--- /dev/null
+++ b/www/maintenance-manager.mustache
@@ -0,0 +1,84 @@
+{{!
+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.
+}}{{#raw}}{{{.}}}{{/raw}}{{^raw}}
+
+<h1>Maintenance Manager state</h1>
+
+<h3>Running operations</h3>
+<table class="table table-striped">
+  <thead>
+    <tr>
+      <th>Name</th>
+      <th>Instances running</th>
+    </tr>
+  </thead>
+  <tbody>
+   {{#running_operations}}
+    <tr>
+      <td>{{name}}</td>
+      <td>{{instances_running}}</td>
+    </tr>
+   {{/running_operations}}
+  </tbody>
+</table>
+
+<h3>Recent completed operations</h3>
+<table class="table table-striped">
+  <thead>
+    <tr>
+      <th>Name</th>
+      <th>Duration</th>
+      <th>Time since op started</th>
+    </tr>
+  </thead>
+  <tbody>
+   {{#completed_operations}}
+    <tr>
+      <td>{{name}}</td>
+      <td>{{duration}}</td>
+      <td>{{time_since_start}}</td>
+    </tr>
+   {{/completed_operations}}
+  </tbody>
+</table>
+
+<h3>Non-running operations</h3>
+<table class="table table-striped">
+  <thead>
+    <tr>
+      <th>Name</th>
+      <th>Runnable</th>
+      <th>RAM anchored</th>
+      <th>Logs retained</th>
+      <th>Perf</th>
+    </tr>
+  </thead>
+  <tbody>
+   {{#registered_operations}}
+    <tr>
+      <td>{{name}}</td>
+      <td>{{runnable}}</td>
+      <td>{{ram_anchored}}</td>
+      <td>{{logs_retained}}</td>
+      <td>{{perf}}</td>
+    </tr>
+   {{/registered_operations}}
+  </tbody>
+</table>
+
+{{/raw}}