You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2017/10/19 21:48:08 UTC

[2/2] kudu git commit: tool: speed up test runtime a bit

tool: speed up test runtime a bit

We're good about setting --never_fsync in ExternalMiniClusters and in all
gtests, but test-based invocations of the Kudu CLI don't get it. This patch
changes that by centralizing all test-based CLI invocations and ensuring
they pass --never_fsync.

Change-Id: I27c67368b736f864f7cfa3482fdae72acafce851
Reviewed-on: http://gerrit.cloudera.org:8080/8337
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: a6f2322b241189d0a32d1d07165dee8268e4dcaa
Parents: 4d84eee
Author: Adar Dembo <ad...@cloudera.com>
Authored: Thu Oct 19 11:55:01 2017 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Thu Oct 19 21:47:22 2017 +0000

----------------------------------------------------------------------
 src/kudu/tools/kudu-admin-test.cc  | 80 ++++++++++++++++-----------------
 src/kudu/tools/kudu-tool-test.cc   | 40 +++++++----------
 src/kudu/tools/kudu-ts-cli-test.cc | 16 +++----
 src/kudu/tools/tool_test_util.cc   | 16 ++++++-
 src/kudu/tools/tool_test_util.h    | 13 +++++-
 5 files changed, 89 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a6f2322b/src/kudu/tools/kudu-admin-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index f49e9bd..696ba89 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -58,7 +58,6 @@
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
-#include "kudu/util/subprocess.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -85,6 +84,8 @@ using kudu::itest::WaitUntilCommittedOpIdIndexIs;
 using kudu::itest::WaitUntilTabletInState;
 using kudu::itest::WaitUntilTabletRunning;
 using kudu::pb_util::SecureDebugString;
+using std::back_inserter;
+using std::copy;
 using std::deque;
 using std::string;
 using std::vector;
@@ -152,8 +153,7 @@ TEST_F(AdminCliTest, TestChangeConfig) {
   ASSERT_EQ(leader->uuid(), master_observed_leader->uuid());
 
   LOG(INFO) << "Adding tserver with uuid " << new_node->uuid() << " as VOTER...";
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "tablet",
     "change_config",
     "add_replica",
@@ -190,8 +190,7 @@ TEST_F(AdminCliTest, TestChangeConfig) {
 
   // Now remove the server once again.
   LOG(INFO) << "Removing tserver with uuid " << new_node->uuid() << " from the config...";
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "tablet",
     "change_config",
     "remove_replica",
@@ -250,16 +249,15 @@ TEST_F(AdminCliTest, TestMoveTablet) {
   for (int i = 0; i < num_moves; i++) {
     const string remove = active_tservers.front();
     const string add = inactive_tservers.front();
-    ASSERT_OK(Subprocess::Call({
-                                   GetKuduCtlAbsolutePath(),
-                                   "tablet",
-                                   "change_config",
-                                   "move_replica",
-                                   cluster_->master()->bound_rpc_addr().ToString(),
-                                   tablet_id_,
-                                   remove,
-                                   add
-                               }));
+    ASSERT_OK(RunKuduTool({
+      "tablet",
+      "change_config",
+      "move_replica",
+      cluster_->master()->bound_rpc_addr().ToString(),
+      tablet_id_,
+      remove,
+      add
+    }));
     active_tservers.pop_front();
     active_tservers.push_back(add);
     inactive_tservers.pop_front();
@@ -283,12 +281,14 @@ TEST_F(AdminCliTest, TestMoveTablet) {
 Status RunUnsafeChangeConfig(const string& tablet_id,
                              const string& dst_host,
                              vector<string> peer_uuid_list) {
-  vector<string> command_args = { GetKuduCtlAbsolutePath(), "remote_replica",
-      "unsafe_change_config", dst_host, tablet_id };
-  for (const auto& peer_uuid : peer_uuid_list) {
-    command_args.push_back(peer_uuid);
-  }
-  return Subprocess::Call(command_args);
+  vector<string> command_args = {
+      "remote_replica",
+      "unsafe_change_config",
+      dst_host,
+      tablet_id
+  };
+  copy(peer_uuid_list.begin(), peer_uuid_list.end(), back_inserter(command_args));
+  return RunKuduTool(command_args);
 }
 
 // Test unsafe config change when there is one follower survivor in the cluster.
@@ -802,10 +802,12 @@ TEST_F(AdminCliTest, TestUnsafeChangeConfigFollowerWithPendingConfig) {
   // Force leader to step down, best effort command since the leadership
   // could change anytime during cluster lifetime.
   string stderr;
-  s = Subprocess::Call({GetKuduCtlAbsolutePath(), "tablet", "leader_step_down",
-                        cluster_->master()->bound_rpc_addr().ToString(),
-                        tablet_id_},
-                       "", nullptr, &stderr);
+  s = RunKuduTool({
+    "tablet",
+    "leader_step_down",
+    cluster_->master()->bound_rpc_addr().ToString(),
+    tablet_id_
+  }, nullptr, &stderr);
   bool not_currently_leader = stderr.find(
       Status::IllegalState("").CodeAsString()) != string::npos;
   ASSERT_TRUE(s.ok() || not_currently_leader);
@@ -1106,10 +1108,12 @@ TEST_F(AdminCliTest, TestLeaderStepDown) {
   // the term advancement without honoring status of the command since
   // there may not have been another election in the meanwhile.
   string stderr;
-  Status s = Subprocess::Call({GetKuduCtlAbsolutePath(),
-                               "tablet", "leader_step_down",
-                               cluster_->master()->bound_rpc_addr().ToString(),
-                               tablet_id_}, "", nullptr, &stderr);
+  Status s = RunKuduTool({
+    "tablet",
+    "leader_step_down",
+    cluster_->master()->bound_rpc_addr().ToString(),
+    tablet_id_
+  }, nullptr, &stderr);
   bool not_currently_leader = stderr.find(
       Status::IllegalState("").CodeAsString()) != string::npos;
   ASSERT_TRUE(s.ok() || not_currently_leader);
@@ -1142,13 +1146,12 @@ TEST_F(AdminCliTest, TestLeaderStepDownWhenNotPresent) {
   ASSERT_TRUE(GetTermFromConsensus(tservers, tablet_id_,
                                    &current_term).IsNotFound());
   string stdout;
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "tablet",
     "leader_step_down",
     cluster_->master()->bound_rpc_addr().ToString(),
     tablet_id_
-  }, "", &stdout));
+  }, &stdout));
   ASSERT_STR_CONTAINS(stdout,
                       Substitute("No leader replica found for tablet $0",
                                  tablet_id_));
@@ -1165,8 +1168,7 @@ TEST_F(AdminCliTest, TestDeleteTable) {
             .add_master_server_addr(master_address)
             .Build(&client));
 
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "table",
     "delete",
     master_address,
@@ -1185,12 +1187,11 @@ TEST_F(AdminCliTest, TestListTables) {
   NO_FATALS(BuildAndStart());
 
   string stdout;
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "table",
     "list",
     cluster_->master()->bound_rpc_addr().ToString()
-  }, "", &stdout, nullptr));
+  }, &stdout));
 
   vector<string> stdout_lines = Split(stdout, ",", strings::SkipEmpty());
   ASSERT_EQ(1, stdout_lines.size());
@@ -1221,13 +1222,12 @@ TEST_F(AdminCliTest, TestListTablesDetail) {
                        MonoDelta::FromSeconds(30), &tablet_ids);
 
   string stdout;
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "table",
     "list",
     "--list_tablets",
     cluster_->master()->bound_rpc_addr().ToString()
-  }, "", &stdout, nullptr));
+  }, &stdout));
 
   vector<string> stdout_lines = Split(stdout, "\n", strings::SkipEmpty());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6f2322b/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 28054cb..0016b28 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -168,10 +168,6 @@ using tserver::TabletServerServiceProxy;
 
 class ToolTest : public KuduTest {
  public:
-  ToolTest()
-      : tool_path_(GetKuduCtlAbsolutePath()) {
-  }
-
   ~ToolTest() {
     STLDeleteValues(&ts_map_);
   }
@@ -187,14 +183,10 @@ class ToolTest : public KuduTest {
                  string* stderr,
                  vector<string>* stdout_lines,
                  vector<string>* stderr_lines) const {
-    vector<string> args = { tool_path_ };
-    vector<string> more_args = strings::Split(arg_str, " ",
-                                              strings::SkipEmpty());
-    args.insert(args.end(), more_args.begin(), more_args.end());
-
     string out;
     string err;
-    Status s = Subprocess::Call(args, "", &out, &err);
+    Status s = RunKuduTool(strings::Split(arg_str, " ", strings::SkipEmpty()),
+                           &out, &err);
     if (stdout) {
       *stdout = out;
       StripTrailingNewline(stdout);
@@ -259,11 +251,11 @@ class ToolTest : public KuduTest {
     ASSERT_TRUE(stdout.empty());
     ASSERT_FALSE(stderr.empty());
 
-    // If it was an invalid command, the usage string is on the second line.
-    int usage_idx = 0;
+    // If it was an invalid command, the usage string is on the third line.
+    int usage_idx = 1;
     if (!expected_status.ok()) {
-      ASSERT_EQ(expected_status.ToString(), stderr[0]);
-      usage_idx = 1;
+      ASSERT_EQ(expected_status.ToString(), stderr[1]);
+      usage_idx = 2;
     }
     ASSERT_EQ(0, stderr[usage_idx].find("Usage: "));
 
@@ -286,8 +278,8 @@ class ToolTest : public KuduTest {
     vector<string> err_lines;
     RunTool(arg_str, nullptr, nullptr, nullptr, /* stderr_lines = */ &err_lines);
     ASSERT_GE(err_lines.size(), 3) << err_lines;
-    ASSERT_EQ(expected_status.ToString(), err_lines[0]);
-    ASSERT_STR_MATCHES(err_lines[2], "Usage: kudu.*");
+    ASSERT_EQ(expected_status.ToString(), err_lines[1]);
+    ASSERT_STR_MATCHES(err_lines[3], "Usage: kudu.*");
   }
 
   void RunFsCheck(const string& arg_str,
@@ -337,7 +329,6 @@ class ToolTest : public KuduTest {
   unordered_map<string, TServerDetails*> ts_map_;
   unique_ptr<InternalMiniCluster> mini_cluster_;
   ExternalMiniClusterOptions cluster_opts_;
-  string tool_path_;
 };
 
 void ToolTest::StartExternalMiniCluster(const vector<string>& extra_master_flags,
@@ -798,7 +789,7 @@ TEST_F(ToolTest, TestPbcTools) {
     Status s = DoEdit(path, &stdout, &stderr);
     ASSERT_FALSE(s.ok());
     ASSERT_EQ("", stdout);
-    ASSERT_EQ("Aborted: editor returned non-zero exit code", stderr);
+    ASSERT_STR_CONTAINS(stderr, "Aborted: editor returned non-zero exit code");
   }
 
   // Test 'edit' with an edit which tries to write some invalid JSON (missing required fields).
@@ -817,9 +808,11 @@ TEST_F(ToolTest, TestPbcTools) {
     string stdout, stderr;
     Status s = DoEdit("echo not-a-json-string > $@\n", &stdout, &stderr);
     ASSERT_EQ("", stdout);
-    ASSERT_EQ("Invalid argument: Unable to parse JSON line: not-a-json-string: Unexpected token.\n"
-              "not-a-json-string\n"
-              "^", stderr);
+    ASSERT_STR_CONTAINS(
+        stderr,
+        "Invalid argument: Unable to parse JSON line: not-a-json-string: Unexpected token.\n"
+        "not-a-json-string\n"
+        "^");
   }
 
   // The file should be unchanged by the unsuccessful edits above.
@@ -1297,7 +1290,6 @@ void ToolTest::RunLoadgen(int num_tservers,
               .Create());
   }
   vector<string> args = {
-    GetKuduCtlAbsolutePath(),
     "perf",
     "loadgen",
     cluster_->master()->bound_rpc_addr().ToString(),
@@ -1306,7 +1298,7 @@ void ToolTest::RunLoadgen(int num_tservers,
     args.push_back(Substitute("-table_name=$0", table_name));
   }
   copy(tool_args.begin(), tool_args.end(), back_inserter(args));
-  ASSERT_OK(Subprocess::Call(args));
+  ASSERT_OK(RunKuduTool(args));
 }
 
 // Run the loadgen benchmark with all optional parameters set to defaults.
@@ -1772,7 +1764,7 @@ class ControlShellToolTest :
       default: LOG(FATAL) << "Unknown serialization mode";
     }
     shell_.reset(new Subprocess({
-      tool_path_,
+      GetKuduToolAbsolutePath(),
       "test",
       "mini_cluster",
       Substitute("--serialization=$0", mode)

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6f2322b/src/kudu/tools/kudu-ts-cli-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-ts-cli-test.cc b/src/kudu/tools/kudu-ts-cli-test.cc
index 1143185..3a52fdf 100644
--- a/src/kudu/tools/kudu-ts-cli-test.cc
+++ b/src/kudu/tools/kudu-ts-cli-test.cc
@@ -36,7 +36,6 @@
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
-#include "kudu/util/subprocess.h"
 #include "kudu/util/test_macros.h"
 
 using kudu::itest::TabletServerMap;
@@ -74,14 +73,13 @@ TEST_F(KuduTsCliTest, TestDeleteTablet) {
                                             tablet_id, timeout));
   }
   string out;
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "remote_replica",
     "delete",
     cluster_->tablet_server(0)->bound_rpc_addr().ToString(),
     tablet_id,
     "Deleting for kudu-ts-cli-test"
-  }, "", &out));
+  }, &out));
   ASSERT_EQ("", out);
 
   ASSERT_OK(inspect_->WaitForTabletDataStateOnTS(0, tablet_id, { tablet::TABLET_DATA_TOMBSTONED }));
@@ -113,13 +111,12 @@ TEST_F(KuduTsCliTest, TestDumpTablet) {
 
   string out;
   // Test for dump_tablet when there is no data in tablet.
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "remote_replica",
     "dump",
     cluster_->tablet_server(0)->bound_rpc_addr().ToString(),
     tablet_id
-  }, "", &out));
+  }, &out));
   ASSERT_EQ("", out);
 
   // Insert very little data and dump_tablet again.
@@ -129,13 +126,12 @@ TEST_F(KuduTsCliTest, TestDumpTablet) {
   }
   workload.StopAndJoin();
   ASSERT_OK(WaitForServersToAgree(timeout, ts_map_, tablet_id, workload.batches_completed()));
-  ASSERT_OK(Subprocess::Call({
-    GetKuduCtlAbsolutePath(),
+  ASSERT_OK(RunKuduTool({
     "remote_replica",
     "dump",
     cluster_->tablet_server(0)->bound_rpc_addr().ToString(),
     tablet_id
-  }, "", &out));
+  }, &out));
 
   // Split the output into multiple rows and check format of each row,
   // and also check total number of rows are at least kNumRows.

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6f2322b/src/kudu/tools/tool_test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_test_util.cc b/src/kudu/tools/tool_test_util.cc
index a0ac540..7f80732 100644
--- a/src/kudu/tools/tool_test_util.cc
+++ b/src/kudu/tools/tool_test_util.cc
@@ -20,19 +20,22 @@
 #include "kudu/tools/tool_test_util.h"
 
 #include <ostream>
+#include <vector>
 
 #include <glog/logging.h>
 
 #include "kudu/util/env.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
 
 using std::string;
+using std::vector;
 
 namespace kudu {
 namespace tools {
 
-string GetKuduCtlAbsolutePath() {
+string GetKuduToolAbsolutePath() {
   static const string kKuduCtlFileName = "kudu";
   string exe;
   CHECK_OK(Env::Default()->GetExecutablePath(&exe));
@@ -43,5 +46,16 @@ string GetKuduCtlAbsolutePath() {
   return tool_abs_path;
 }
 
+Status RunKuduTool(const vector<string>& args, string* out, string* err) {
+  vector<string> total_args = { GetKuduToolAbsolutePath() };
+
+  // Speed up filesystem-based operations.
+  total_args.emplace_back("--unlock_unsafe_flags");
+  total_args.emplace_back("--never_fsync");
+
+  total_args.insert(total_args.end(), args.begin(), args.end());
+  return Subprocess::Call(total_args, "", out, err);
+}
+
 } // namespace tools
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6f2322b/src/kudu/tools/tool_test_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_test_util.h b/src/kudu/tools/tool_test_util.h
index 6a89281..f5d6012 100644
--- a/src/kudu/tools/tool_test_util.h
+++ b/src/kudu/tools/tool_test_util.h
@@ -20,12 +20,23 @@
 #pragma once
 
 #include <string>
+#include <vector>
+
+#include "kudu/util/status.h"
 
 namespace kudu {
 namespace tools {
 
 // Get full path to the top-level 'kudu' tool binary.
-std::string GetKuduCtlAbsolutePath();
+std::string GetKuduToolAbsolutePath();
+
+// Runs the 'kudu' tool binary with the given arguments.
+//
+// If 'out' or 'err' is set, the tool's stdout or stderr output will be
+// written to each respectively.
+Status RunKuduTool(const std::vector<std::string>& args,
+                   std::string* out = nullptr,
+                   std::string* err = nullptr);
 
 } // namespace tools
 } // namespace kudu