You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by wd...@apache.org on 2019/02/15 23:37:50 UTC

[kudu] branch master updated (794a9ec -> df10529)

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

wdberkeley pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 794a9ec  docs: fix two broken links in the installation doc
     new c72a45c  [server] Add start_time in server description
     new df10529  [tools] Add useful flags to 'kudu remote_replica list'

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/common/wire_protocol.cc                |  19 ++++-
 src/kudu/common/wire_protocol.h                 |   5 ++
 src/kudu/common/wire_protocol.proto             |   3 +
 src/kudu/integration-tests/registration-test.cc |  20 +++++
 src/kudu/master/master-test.cc                  |  23 ++++++
 src/kudu/master/master.cc                       |   1 +
 src/kudu/master/master_path_handlers.cc         |  11 ++-
 src/kudu/master/ts_descriptor-test.cc           |   1 +
 src/kudu/server/server_base.cc                  |   3 +
 src/kudu/server/server_base.h                   |   6 ++
 src/kudu/tools/kudu-tool-test.cc                | 102 ++++++++++++++++++------
 src/kudu/tools/tool_action_master.cc            |  10 ++-
 src/kudu/tools/tool_action_remote_replica.cc    |  30 ++++++-
 src/kudu/tools/tool_action_tserver.cc           |   9 ++-
 src/kudu/tserver/heartbeater.cc                 |   1 +
 src/kudu/util/pb_util.cc                        |   1 -
 src/kudu/util/pb_util.h                         |   4 +-
 www/masters.mustache                            |   2 +
 www/tablet-servers.mustache                     |   2 +
 19 files changed, 217 insertions(+), 36 deletions(-)


[kudu] 01/02: [server] Add start_time in server description

Posted by wd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c72a45c49435b6800fa435aab6ed73adac288baf
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Wed Jan 30 14:22:52 2019 +0800

    [server] Add start_time in server description
    
    Add `start_time` to describe the start time of a server in
    localtime format, like `2019-01-01 00:00:00 UTC`. We can use
    this information to acknowledge the last start time of a server,
    both master and tserver, to check whether there is an unexpect
    restarting, the server joined time, etc. We can get `start_time`
    of servers from both web-ui and command line `kudu tserver/master
    list`.
    
    Change-Id: I041467014499ad00c07398ad8f61c6d6ca1ceeca
    Reviewed-on: http://gerrit.cloudera.org:8080/12304
    Tested-by: Kudu Jenkins
    Reviewed-by: Todd Lipcon <to...@apache.org>
---
 src/kudu/common/wire_protocol.cc                | 19 ++++++++++++++++++-
 src/kudu/common/wire_protocol.h                 |  5 +++++
 src/kudu/common/wire_protocol.proto             |  3 +++
 src/kudu/integration-tests/registration-test.cc | 20 ++++++++++++++++++++
 src/kudu/master/master-test.cc                  | 23 +++++++++++++++++++++++
 src/kudu/master/master.cc                       |  1 +
 src/kudu/master/master_path_handlers.cc         | 11 +++++++++--
 src/kudu/master/ts_descriptor-test.cc           |  1 +
 src/kudu/server/server_base.cc                  |  3 +++
 src/kudu/server/server_base.h                   |  6 ++++++
 src/kudu/tools/tool_action_master.cc            | 10 +++++++---
 src/kudu/tools/tool_action_tserver.cc           |  9 ++++++---
 src/kudu/tserver/heartbeater.cc                 |  1 +
 src/kudu/util/pb_util.cc                        |  1 -
 src/kudu/util/pb_util.h                         |  4 ++--
 www/masters.mustache                            |  2 ++
 www/tablet-servers.mustache                     |  2 ++
 17 files changed, 109 insertions(+), 12 deletions(-)

diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index 2aab6f0..dc7b1f2 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -17,6 +17,8 @@
 
 #include "kudu/common/wire_protocol.h"
 
+#include <time.h>
+
 #include <cstdint>
 #include <cstring>
 #include <ostream>
@@ -26,8 +28,8 @@
 #include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 
-#include "kudu/common/columnblock.h"
 #include "kudu/common/column_predicate.h"
+#include "kudu/common/columnblock.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/row.h"
 #include "kudu/common/rowblock.h"
@@ -39,9 +41,11 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/fastmem.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
 #include "kudu/util/bitmap.h"
 #include "kudu/util/compression/compression.pb.h"
 #include "kudu/util/faststring.h"
+#include "kudu/util/hash.pb.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
@@ -948,4 +952,17 @@ void SerializeRowBlock(const RowBlock& block,
   rowblock_pb->set_num_rows(rowblock_pb->num_rows() + num_rows);
 }
 
+std::string StartTimeToString(const ServerRegistrationPB& reg) {
+  string start_time;
+  if (reg.has_start_time()) {
+    // Convert epoch time to localtime.
+    StringAppendStrftime(&start_time, "%Y-%m-%d %H:%M:%S %Z",
+                         static_cast<time_t>(reg.start_time()), true);
+  } else {
+    start_time = "<unknown>";
+  }
+
+  return start_time;
+}
+
 } // namespace kudu
diff --git a/src/kudu/common/wire_protocol.h b/src/kudu/common/wire_protocol.h
index cedc7d9..cf6be92 100644
--- a/src/kudu/common/wire_protocol.h
+++ b/src/kudu/common/wire_protocol.h
@@ -20,6 +20,7 @@
 #define KUDU_COMMON_WIRE_PROTOCOL_H
 
 #include <cstdint>
+#include <string>
 #include <vector>
 
 #include "kudu/util/status.h"
@@ -56,6 +57,7 @@ class HostPortPB;
 class RowwiseRowBlockPB;
 class SchemaPB;
 class ServerEntryPB;
+class ServerRegistrationPB;
 
 // Convert the given C++ Status object into the equivalent Protobuf.
 void StatusToPB(const Status& status, AppStatusPB* pb);
@@ -195,5 +197,8 @@ Status ExtractRowsFromRowBlockPB(const Schema& schema,
 Status FindLeaderHostPort(const google::protobuf::RepeatedPtrField<ServerEntryPB>& entries,
                           HostPort* leader_hostport);
 
+// Extract 'start_time' in ServerRegistrationPB, and convert it to localtime as a string
+// or '<unknown>' if lack this field.
+std::string StartTimeToString(const ServerRegistrationPB& reg);
 } // namespace kudu
 #endif
diff --git a/src/kudu/common/wire_protocol.proto b/src/kudu/common/wire_protocol.proto
index f592734..225ce47 100644
--- a/src/kudu/common/wire_protocol.proto
+++ b/src/kudu/common/wire_protocol.proto
@@ -93,6 +93,9 @@ message ServerRegistrationPB {
   // In this case, https:// URLs should be generated for the above
   // 'http_addresses' field.
   optional bool https_enabled = 4;
+
+  // Seconds since the epoch.
+  optional int64 start_time = 5;
 }
 
 message ServerEntryPB {
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index bbd0834..6414127 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <stdint.h>
+
 #include <algorithm>
 #include <memory>
 #include <ostream>
@@ -32,6 +34,7 @@
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
 #include "kudu/master/catalog_manager.h"
 #include "kudu/master/master-test-util.h"
 #include "kudu/master/master.h"
@@ -143,6 +146,7 @@ class RegistrationTest : public KuduTest {
   void SetUp() override {
     // Make heartbeats faster to speed test runtime.
     FLAGS_heartbeat_interval_ms = 10;
+    setup_time_ = WallTime_Now();
 
     KuduTest::SetUp();
 
@@ -209,6 +213,7 @@ class RegistrationTest : public KuduTest {
  protected:
   gscoped_ptr<InternalMiniCluster> cluster_;
   Schema schema_;
+  int64_t setup_time_;
 };
 
 TEST_F(RegistrationTest, TestTSRegisters) {
@@ -250,6 +255,21 @@ TEST_F(RegistrationTest, TestMasterSoftwareVersion) {
     ASSERT_TRUE(reg.has_software_version());
     ASSERT_STR_CONTAINS(reg.software_version(),
                         VersionInfo::GetVersionInfo());
+    ASSERT_LE(setup_time_, reg.start_time());
+    ASSERT_LE(reg.start_time(), WallTime_Now());
+  }
+}
+
+TEST_F(RegistrationTest, TestServerStartTime) {
+  ServerRegistrationPB reg;
+  cluster_->mini_master()->master()->GetMasterRegistration(&reg);
+  ASSERT_LE(setup_time_, reg.start_time());
+  ASSERT_LE(reg.start_time(), WallTime_Now());
+
+  for (int i = 0; i < cluster_->num_tablet_servers(); ++i) {
+    auto start_time = cluster_->mini_tablet_server(i)->server()->start_time();
+    ASSERT_LE(setup_time_, start_time);
+    ASSERT_LE(start_time, WallTime_Now());
   }
 }
 
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 24db50d..552d1ed 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -17,6 +17,8 @@
 
 #include "kudu/master/master.h"
 
+#include <time.h>
+
 #include <algorithm>
 #include <cstdint>
 #include <map>
@@ -54,6 +56,7 @@
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
+#include "kudu/gutil/walltime.h"
 #include "kudu/master/catalog_manager.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
@@ -210,6 +213,7 @@ TEST_F(MasterTest, TestRegisterAndHeartbeat) {
   MakeHostPortPB("localhost", 1000, fake_reg.add_rpc_addresses());
   MakeHostPortPB("localhost", 2000, fake_reg.add_http_addresses());
   fake_reg.set_software_version(VersionInfo::GetVersionInfo());
+  fake_reg.set_start_time(10000);
 
   // Information on replica management scheme.
   ReplicaManagementInfoPB rmi;
@@ -378,6 +382,9 @@ TEST_F(MasterTest, TestRegisterAndHeartbeat) {
     ASSERT_EQ(true, tablet_server["live"].GetBool());
     ASSERT_STREQ(VersionInfo::GetVersionInfo().c_str(),
         tablet_server["version"].GetString());
+    string start_time;
+    StringAppendStrftime(&start_time, "%Y-%m-%d %H:%M:%S %Z", static_cast<time_t>(10000), true);
+    ASSERT_STREQ(start_time.c_str(), tablet_server["start_time"].GetString());
   }
 
   // Ensure that trying to re-register with a different version is OK.
@@ -392,6 +399,22 @@ TEST_F(MasterTest, TestRegisterAndHeartbeat) {
     // the numeric portion will match.
     req.mutable_registration()->set_software_version(Substitute("kudu $0 (rev SOME_NON_GIT_HASH)",
                                                                 KUDU_VERSION_STRING));
+
+    ASSERT_OK(proxy_->TSHeartbeat(req, &resp, &rpc));
+    ASSERT_FALSE(resp.has_error());
+  }
+
+  // Ensure that trying to re-register with a different start_time is OK.
+  {
+    TSHeartbeatRequestPB req;
+    TSHeartbeatResponsePB resp;
+    RpcController rpc;
+    req.mutable_common()->CopyFrom(common);
+    req.mutable_registration()->CopyFrom(fake_reg);
+    req.mutable_replica_management_info()->CopyFrom(rmi);
+    // 10 minutes later.
+    req.mutable_registration()->set_start_time(10600);
+
     ASSERT_OK(proxy_->TSHeartbeat(req, &resp, &rpc));
     ASSERT_FALSE(resp.has_error());
   }
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index ae33cd2..bc7f9ae 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -293,6 +293,7 @@ Status Master::InitMasterRegistration() {
     reg.set_https_enabled(web_server()->IsSecure());
   }
   reg.set_software_version(VersionInfo::GetVersionInfo());
+  reg.set_start_time(start_time_);
 
   registration_.Swap(&reg);
   registration_initialized_.store(true);
diff --git a/src/kudu/master/master_path_handlers.cc b/src/kudu/master/master_path_handlers.cc
index 8566966..02e6474 100644
--- a/src/kudu/master/master_path_handlers.cc
+++ b/src/kudu/master/master_path_handlers.cc
@@ -129,6 +129,8 @@ void MasterPathHandlers::HandleTabletServers(const Webserver::WebRequest& /*req*
                                      reg.http_addresses(0).port());
     }
     ts_json["time_since_hb"] = StringPrintf("%.1fs", desc->TimeSinceHeartbeat().ToSeconds());
+    ts_json["start_time"] = StartTimeToString(reg);
+    reg.clear_start_time();  // Clear 'start_time' before dumping to string.
     ts_json["registration"] = pb_util::SecureShortDebugString(reg);
     ts_json["location"] = desc->location().get_value_or("<none>");
     version_counts[reg.software_version()][desc->PresumedDead() ? 1 : 0]++;
@@ -458,7 +460,7 @@ void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& /*req*/,
       continue;
     }
     EasyJson master_json = (*output)["masters"].PushBack(EasyJson::kObject);
-    const ServerRegistrationPB& reg = master.registration();
+    ServerRegistrationPB reg = master.registration();
     master_json["uuid"] = master.instance_id().permanent_uuid();
     if (!reg.http_addresses().empty()) {
       master_json["target"] = Substitute("$0://$1:$2/",
@@ -467,7 +469,9 @@ void MasterPathHandlers::HandleMasters(const Webserver::WebRequest& /*req*/,
                                          reg.http_addresses(0).port());
     }
     master_json["role"] = master.has_role() ? RaftPeerPB_Role_Name(master.role()) : "N/A";
-    master_json["registration"] = pb_util::SecureShortDebugString(master.registration());
+    master_json["start_time"] = StartTimeToString(reg);
+    reg.clear_start_time();  // Clear 'start_time' before dumping to string.
+    master_json["registration"] = pb_util::SecureShortDebugString(reg);
   }
 }
 
@@ -645,6 +649,9 @@ void MasterPathHandlers::HandleDumpEntities(const Webserver::WebRequest& /*req*/
     jw.String("version");
     jw.String(reg.software_version());
 
+    jw.String("start_time");
+    jw.String(StartTimeToString(reg));
+
     jw.EndObject();
   }
   jw.EndArray();
diff --git a/src/kudu/master/ts_descriptor-test.cc b/src/kudu/master/ts_descriptor-test.cc
index a7ec824..92e3fec 100644
--- a/src/kudu/master/ts_descriptor-test.cc
+++ b/src/kudu/master/ts_descriptor-test.cc
@@ -67,6 +67,7 @@ void SetupBasicRegistrationInfo(const string& uuid,
   http_hostport->set_port(54321);
   registration->set_software_version("1.0.0");
   registration->set_https_enabled(false);
+  registration->set_start_time(10000);
 }
 
 TEST(TSDescriptorTest, TestRegistration) {
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 1a742d1..8894373 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -43,6 +43,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/strcat.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/remote_user.h"
 #include "kudu/rpc/result_tracker.h"
@@ -679,6 +680,8 @@ Status ServerBase::Start() {
                           "Failed to dump server info to " + options_.dump_info_path);
   }
 
+  start_time_ = WallTime_Now();
+
   return Status::OK();
 }
 
diff --git a/src/kudu/server/server_base.h b/src/kudu/server/server_base.h
index 8c0f940..203dd99 100644
--- a/src/kudu/server/server_base.h
+++ b/src/kudu/server/server_base.h
@@ -107,6 +107,10 @@ class ServerBase {
   // Return a PB describing the status of the server (version info, bound ports, etc)
   Status GetStatusPB(ServerStatusPB* status) const;
 
+  int64_t start_time() const {
+    return start_time_;
+  }
+
   enum {
     SUPER_USER = 1,
     USER = 1 << 1,
@@ -161,6 +165,8 @@ class ServerBase {
   void LogUnauthorizedAccess(rpc::RpcContext* rpc) const;
 
   const std::string name_;
+  // Seconds since the epoch.
+  int64_t start_time_;
 
   std::unique_ptr<MinidumpExceptionHandler> minidump_handler_;
   std::shared_ptr<MemTracker> mem_tracker_;
diff --git a/src/kudu/tools/tool_action_master.cc b/src/kudu/tools/tool_action_master.cc
index 98acfb9..e74c1b7 100644
--- a/src/kudu/tools/tool_action_master.cc
+++ b/src/kudu/tools/tool_action_master.cc
@@ -15,8 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "kudu/tools/tool_action.h"
-
 #include <algorithm>
 #include <iostream>
 #include <iterator>
@@ -40,6 +38,7 @@
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
+#include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/util/status.h"
 
@@ -144,6 +143,10 @@ Status ListMasters(const RunnerContext& context) {
       for (const auto& master : masters) {
         values.push_back(master.registration().software_version());
       }
+    } else if (boost::iequals(column, "start_time")) {
+      for (const auto& master : masters) {
+        values.emplace_back(StartTimeToString(master.registration()));
+      }
     } else {
       return Status::InvalidArgument("unknown column (--columns)", column);
     }
@@ -207,7 +210,8 @@ unique_ptr<Mode> BuildMasterMode() {
       .AddOptionalParameter("columns", string("uuid,rpc-addresses"),
                             string("Comma-separated list of master info fields to "
                                    "include in output.\nPossible values: uuid, "
-                                   "rpc-addresses, http-addresses, version, and seqno"))
+                                   "rpc-addresses, http-addresses, version, seqno "
+                                   "and start_time"))
       .AddOptionalParameter("format")
       .AddOptionalParameter("timeout_ms")
       .Build();
diff --git a/src/kudu/tools/tool_action_tserver.cc b/src/kudu/tools/tool_action_tserver.cc
index a0527b8..95fa07e 100644
--- a/src/kudu/tools/tool_action_tserver.cc
+++ b/src/kudu/tools/tool_action_tserver.cc
@@ -15,8 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "kudu/tools/tool_action.h"
-
 #include <iostream>
 #include <memory>
 #include <string>
@@ -37,6 +35,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
+#include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/tserver/tablet_server.h"
 #include "kudu/util/status.h"
@@ -143,6 +142,10 @@ Status ListTServers(const RunnerContext& context) {
         string loc = server.location();
         values.emplace_back(loc.empty() ? "<none>" : std::move(loc));
       }
+    } else if (boost::iequals(column, "start_time")) {
+      for (const auto& server : servers) {
+        values.emplace_back(StartTimeToString(server.registration()));
+      }
     } else {
       return Status::InvalidArgument("unknown column (--columns)", column);
     }
@@ -207,7 +210,7 @@ unique_ptr<Mode> BuildTServerMode() {
                             string("Comma-separated list of tserver info fields to "
                                    "include in output.\nPossible values: uuid, "
                                    "rpc-addresses, http-addresses, version, seqno, "
-                                   "and heartbeat"))
+                                   "heartbeat and start_time"))
       .AddOptionalParameter("format")
       .AddOptionalParameter("timeout_ms")
       .Build();
diff --git a/src/kudu/tserver/heartbeater.cc b/src/kudu/tserver/heartbeater.cc
index b9cb0a8..0b1587f 100644
--- a/src/kudu/tserver/heartbeater.cc
+++ b/src/kudu/tserver/heartbeater.cc
@@ -368,6 +368,7 @@ Status Heartbeater::Thread::SetupRegistration(ServerRegistrationPB* reg) {
     reg->set_https_enabled(server_->web_server()->IsSecure());
   }
   reg->set_software_version(VersionInfo::GetVersionInfo());
+  reg->set_start_time(server_->start_time());
 
   return Status::OK();
 }
diff --git a/src/kudu/util/pb_util.cc b/src/kudu/util/pb_util.cc
index 35e4628..6237c80 100644
--- a/src/kudu/util/pb_util.cc
+++ b/src/kudu/util/pb_util.cc
@@ -649,7 +649,6 @@ string SecureShortDebugString(const Message& msg) {
   return debug_string;
 }
 
-
 WritablePBContainerFile::WritablePBContainerFile(shared_ptr<RWFile> writer)
   : state_(FileState::NOT_INITIALIZED),
     offset_(0),
diff --git a/src/kudu/util/pb_util.h b/src/kudu/util/pb_util.h
index 6c132a6..13e496f 100644
--- a/src/kudu/util/pb_util.h
+++ b/src/kudu/util/pb_util.h
@@ -31,8 +31,8 @@
 #include <gtest/gtest_prod.h>
 
 #include "kudu/gutil/ref_counted.h"
-#include "kudu/util/mutex.h"
 #include "kudu/util/debug/trace_event_impl.h"
+#include "kudu/util/mutex.h"
 
 namespace google {
 namespace protobuf {
@@ -47,11 +47,11 @@ class SimpleDescriptorDatabase;
 namespace kudu {
 
 class Env;
+class RWFile;
 class RandomAccessFile;
 class SequentialFile;
 class Slice;
 class Status;
-class RWFile;
 class faststring;
 
 namespace pb_util {
diff --git a/www/masters.mustache b/www/masters.mustache
index 0ce7649..268533e 100644
--- a/www/masters.mustache
+++ b/www/masters.mustache
@@ -32,6 +32,7 @@ under the License.
     <thead><tr>
       <th>UUID</th>
       <th>Role</th>
+      <th>Start time</th>
       <th>Registration</th>
     </tr></thead>
     <tbody>
@@ -39,6 +40,7 @@ under the License.
       <tr>
           <td>{{#target}}<a href="{{.}}">{{/target}}{{uuid}}{{#target}}</a>{{/target}}</td>
           <td>{{role}}</td>
+          <td>{{start_time}}</td>
           <td><pre><code>{{registration}}</code></pre></td>
       </tr>
     {{/masters}}
diff --git a/www/tablet-servers.mustache b/www/tablet-servers.mustache
index a778f72..7370232 100644
--- a/www/tablet-servers.mustache
+++ b/www/tablet-servers.mustache
@@ -47,6 +47,7 @@ under the License.
       <th>UUID</th>
       <th>Location</th>
       <th>Time since heartbeat</th>
+      <th>Start time</th>
       <th>Registration</th>
     </tr></thead>
     <tbody>
@@ -55,6 +56,7 @@ under the License.
         <td>{{#target}}<a href="{{.}}">{{/target}}{{uuid}}{{#target}}</a>{{/target}}</td>
         <td>{{location}}</td>
         <td>{{time_since_hb}}</td>
+        <td>{{start_time}}</td>
         <td><pre><code>{{registration}}</code></pre></td>
       </tr>
     {{/live_tservers}}


[kudu] 02/02: [tools] Add useful flags to 'kudu remote_replica list'

Posted by wd...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit df105295b3da8515992815c6cddabb031fc7b089
Author: Will Berkeley <wd...@gmail.org>
AuthorDate: Mon Feb 11 14:28:40 2019 -0800

    [tools] Add useful flags to 'kudu remote_replica list'
    
    1. --include_schema: The ability to exclude schema.
    2. --table_name and --tablets: The ability to filter on tablet id, table name.
    
    Change-Id: I462515f1bc3e8487185aebb6cb99d1c5c00cea40
    Reviewed-on: http://gerrit.cloudera.org:8080/12450
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/tools/kudu-tool-test.cc             | 102 +++++++++++++++++++++------
 src/kudu/tools/tool_action_remote_replica.cc |  30 +++++++-
 2 files changed, 108 insertions(+), 24 deletions(-)

diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index f6542c0..6e3909e 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1913,31 +1913,87 @@ TEST_F(ToolTest, TestRemoteReplicaList) {
   vector<ListTabletsResponsePB::StatusAndSchemaPB> tablets;
   ASSERT_OK(WaitForNumTabletsOnTS(ts, kNumTablets, kTimeout, &tablets));
   const string& ts_addr = cluster_->tablet_server(0)->bound_rpc_addr().ToString();
-  string stdout;
-  NO_FATALS(RunActionStdoutString(
-        Substitute("remote_replica list $0", ts_addr), &stdout));
-  const auto& tablet_status = tablets[0].tablet_status();
 
-  // Some fields like state or estimated on disk size may vary. Just check a
-  // few whose values we should know exactly.
-  ASSERT_STR_CONTAINS(stdout,
-                      Substitute("Tablet id: $0", tablet_status.tablet_id()));
-  ASSERT_STR_CONTAINS(stdout,
-                      Substitute("Table name: $0", workload.table_name()));
-  ASSERT_STR_CONTAINS(stdout,
-      Substitute("Data dirs: $0", JoinStrings(tablet_status.data_dirs(), ", ")));
-
-  // Tombstone the replica and try again.
-  ASSERT_OK(DeleteTablet(ts, tablet_status.tablet_id(),
-                         TabletDataState::TABLET_DATA_TOMBSTONED, kTimeout));
-  NO_FATALS(RunActionStdoutString(
+  const auto& tablet_status = tablets[0].tablet_status();
+  {
+    // Test the basic case.
+    string stdout;
+    NO_FATALS(RunActionStdoutString(
         Substitute("remote_replica list $0", ts_addr), &stdout));
-  ASSERT_STR_CONTAINS(stdout,
-                      Substitute("Tablet id: $0", tablet_status.tablet_id()));
-  ASSERT_STR_CONTAINS(stdout,
-                      Substitute("Table name: $0", workload.table_name()));
-  ASSERT_STR_CONTAINS(stdout,
-      Substitute("Data dirs: $0", JoinStrings(tablet_status.data_dirs(), ", ")));
+
+    // Some fields like state or estimated on disk size may vary. Just check a
+    // few whose values we should know exactly.
+    ASSERT_STR_CONTAINS(stdout,
+                        Substitute("Tablet id: $0", tablet_status.tablet_id()));
+    ASSERT_STR_CONTAINS(stdout,
+                        Substitute("Table name: $0", workload.table_name()));
+    ASSERT_STR_CONTAINS(stdout, "key INT32 NOT NULL");
+    ASSERT_STR_CONTAINS(stdout,
+        Substitute("Data dirs: $0", JoinStrings(tablet_status.data_dirs(), ", ")));
+  }
+
+  {
+    // Test we lose the schema with --include_schema=false.
+    string stdout;
+    NO_FATALS(RunActionStdoutString(
+          Substitute("remote_replica list $0 --include_schema=false", ts_addr),
+          &stdout));
+    ASSERT_STR_NOT_CONTAINS(stdout, "key INT32 NOT NULL");
+  }
+
+  {
+    // Test we see the tablet when matching on wrong tablet id or wrong table
+    // name.
+    string stdout;
+    NO_FATALS(RunActionStdoutString(
+          Substitute("remote_replica list $0 --table_name=$1",
+                     ts_addr, workload.table_name()),
+          &stdout));
+    ASSERT_STR_CONTAINS(stdout,
+                        Substitute("Tablet id: $0",
+                                   tablet_status.tablet_id()));
+    stdout.clear();
+    NO_FATALS(RunActionStdoutString(
+          Substitute("remote_replica list $0 --tablets=$1",
+                     ts_addr, tablet_status.tablet_id()),
+          &stdout));
+    ASSERT_STR_CONTAINS(stdout,
+                        Substitute("Tablet id: $0",
+                                   tablet_status.tablet_id()));
+  }
+
+  {
+    // Test we lose the tablet when matching on the wrong tablet id or the wrong
+    // table name.
+    string stdout;
+    NO_FATALS(RunActionStdoutString(
+          Substitute("remote_replica list $0 --table_name=foo", ts_addr),
+          &stdout));
+    ASSERT_STR_NOT_CONTAINS(stdout,
+                            Substitute("Tablet id: $0",
+                                       tablet_status.tablet_id()));
+    stdout.clear();
+    NO_FATALS(RunActionStdoutString(
+          Substitute("remote_replica list $0 --tablets=foo", ts_addr),
+          &stdout));
+    ASSERT_STR_NOT_CONTAINS(stdout,
+                            Substitute("Tablet id: $0",
+                                       tablet_status.tablet_id()));
+  }
+
+  {
+    // Finally, tombstone the replica and try again.
+    string stdout;
+    ASSERT_OK(DeleteTablet(ts, tablet_status.tablet_id(),
+                           TabletDataState::TABLET_DATA_TOMBSTONED, kTimeout));
+    NO_FATALS(RunActionStdoutString(
+          Substitute("remote_replica list $0", ts_addr), &stdout));
+    ASSERT_STR_CONTAINS(stdout,
+                        Substitute("Tablet id: $0", tablet_status.tablet_id()));
+    ASSERT_STR_CONTAINS(stdout,
+                        Substitute("Table name: $0", workload.table_name()));
+    ASSERT_STR_CONTAINS(stdout, "Data dirs: <not available>");
+  }
 }
 
 // Test 'kudu local_replica delete' tool with --clean_unsafe flag for
diff --git a/src/kudu/tools/tool_action_remote_replica.cc b/src/kudu/tools/tool_action_remote_replica.cc
index 6323f9b..e3b12b1 100644
--- a/src/kudu/tools/tool_action_remote_replica.cc
+++ b/src/kudu/tools/tool_action_remote_replica.cc
@@ -21,6 +21,7 @@
 #include <memory>
 #include <string>
 #include <unordered_map>
+#include <unordered_set>
 #include <utility>
 #include <vector>
 
@@ -44,6 +45,7 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/server/server_base.pb.h"
@@ -62,6 +64,10 @@
 
 DEFINE_bool(force_copy, false,
             "Force the copy when the destination tablet server has this replica");
+DEFINE_bool(include_schema, true,
+            "Whether to include the schema of each replica");
+DECLARE_string(table_name);
+DECLARE_string(tablets);
 DECLARE_int64(timeout_ms); // defined in ksck
 
 namespace kudu {
@@ -82,6 +88,7 @@ using std::cout;
 using std::endl;
 using std::string;
 using std::unique_ptr;
+using std::unordered_set;
 using std::vector;
 using tablet::TabletStatusPB;
 using tserver::DeleteTabletRequestPB;
@@ -178,6 +185,10 @@ Status GetReplicas(TabletServerServiceProxy* proxy,
   RpcController rpc;
   rpc.set_timeout(MonoDelta::FromMilliseconds(FLAGS_timeout_ms));
 
+  // Even with FLAGS_include_schema=false, don't set need_schema_info=false
+  // in the request. The reason is that the schema is still needed to decode
+  // the partition of each replica, and the partition information is pretty
+  // much always nice to have.
   RETURN_NOT_OK(proxy->ListTablets(req, &resp, &rpc));
   if (resp.has_error()) {
     return StatusFromPB(resp.error().status());
@@ -281,7 +292,16 @@ Status ListReplicas(const RunnerContext& context) {
   vector<ListTabletsResponsePB::StatusAndSchemaPB> replicas;
   RETURN_NOT_OK(GetReplicas(proxy.get(), &replicas));
 
+  unordered_set<string> tablet_ids = strings::Split(FLAGS_tablets, ",");
   for (const auto& r : replicas) {
+    if (!FLAGS_table_name.empty() &&
+        r.tablet_status().table_name() != FLAGS_table_name) {
+      continue;
+    }
+    if (!FLAGS_tablets.empty() &&
+        !ContainsKey(tablet_ids, r.tablet_status().tablet_id())) {
+      continue;
+    }
     Schema schema;
     RETURN_NOT_OK_PREPEND(
         SchemaFromPB(r.schema(), &schema),
@@ -311,7 +331,9 @@ Status ListReplicas(const RunnerContext& context) {
     } else {
       cout << "Data dirs: <not available>" << endl;
     }
-    cout << "Schema: " << schema.ToString() << endl;
+    if (FLAGS_include_schema) {
+      cout << "Schema: " << schema.ToString() << endl;
+    }
   }
 
   return Status::OK();
@@ -437,6 +459,12 @@ unique_ptr<Mode> BuildRemoteReplicaMode() {
       ActionBuilder("list", &ListReplicas)
       .Description("List all tablet replicas on a Kudu Tablet Server")
       .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
+      .AddOptionalParameter("include_schema")
+      .AddOptionalParameter("table_name")
+      .AddOptionalParameter("tablets",
+                            string(""),
+                            string("Comma-separated list of tablet IDs used to "
+                                   "filter the list of replicas"))
       .Build();
 
   unique_ptr<Action> unsafe_change_config =