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 2021/08/07 05:02:12 UTC

[kudu] 01/02: [tests] clean-up on cluster_itest_util/ts_itest-base

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

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

commit adb8fbd7b724b0a5c04ac967a917f610383a8a80
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Fri Aug 6 19:40:37 2021 -0700

    [tests] clean-up on cluster_itest_util/ts_itest-base
    
    This patch is makes updates the signature of GetLastOpIdForEachReplica()
    and removes unused utility test functions:
      * TabletServerIntegrationTestBase::GetFurthestAheadReplicaIdx()
      * itest::WaitUntilAllReplicasHaveOp()
    
    This changelist doesn't contain any functional changes.
    
    Change-Id: I516d929424bb53dfc15f9a61030771602e2429eb
    Reviewed-on: http://gerrit.cloudera.org:8080/17761
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/integration-tests/cluster_itest_util.cc | 47 ++----------------------
 src/kudu/integration-tests/cluster_itest_util.h  | 21 ++++-------
 src/kudu/integration-tests/ts_itest-base.cc      | 21 -----------
 src/kudu/integration-tests/ts_itest-base.h       |  5 ---
 4 files changed, 11 insertions(+), 83 deletions(-)

diff --git a/src/kudu/integration-tests/cluster_itest_util.cc b/src/kudu/integration-tests/cluster_itest_util.cc
index ab9eab2..d06bbdb 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -140,7 +140,7 @@ client::KuduSchema SimpleIntKeyKuduSchema() {
 }
 
 Status GetLastOpIdForEachReplica(const string& tablet_id,
-                                 const vector<TServerDetails*>& replicas,
+                                 const vector<const TServerDetails*>& replicas,
                                  OpIdType opid_type,
                                  const MonoDelta& timeout,
                                  vector<OpId>* op_ids) {
@@ -150,7 +150,7 @@ Status GetLastOpIdForEachReplica(const string& tablet_id,
   RpcController controller;
 
   op_ids->clear();
-  for (TServerDetails* ts : replicas) {
+  for (const auto* ts : replicas) {
     controller.Reset();
     controller.set_timeout(timeout);
     opid_resp.Clear();
@@ -168,7 +168,7 @@ Status GetLastOpIdForEachReplica(const string& tablet_id,
 }
 
 Status GetLastOpIdForReplica(const std::string& tablet_id,
-                             TServerDetails* replica,
+                             const TServerDetails* replica,
                              OpIdType opid_type,
                              const MonoDelta& timeout,
                              consensus::OpId* op_id) {
@@ -222,7 +222,7 @@ Status WaitForServersToAgree(const MonoDelta& timeout,
                              consensus::OpIdType op_id_type) {
   const MonoTime deadline = MonoTime::Now() + timeout;
 
-  vector<TServerDetails*> servers;
+  vector<const TServerDetails*> servers;
   AppendValuesFromMap(tablet_servers, &servers);
   for (int i = 1; MonoTime::Now() < deadline; i++) {
     vector<OpId> ids;
@@ -260,45 +260,6 @@ Status WaitForServersToAgree(const MonoDelta& timeout,
                  minimum_index, timeout.ToString()));
 }
 
-// Wait until all specified replicas have logged the given index.
-Status WaitUntilAllReplicasHaveOp(const int64_t log_index,
-                                  const string& tablet_id,
-                                  const vector<TServerDetails*>& replicas,
-                                  const MonoDelta& timeout) {
-  MonoTime start = MonoTime::Now();
-  MonoDelta passed = MonoDelta::FromMilliseconds(0);
-  while (true) {
-    vector<OpId> op_ids;
-    Status s = GetLastOpIdForEachReplica(tablet_id, replicas, consensus::RECEIVED_OPID, timeout,
-                                         &op_ids);
-    if (s.ok()) {
-      bool any_behind = false;
-      for (const OpId& op_id : op_ids) {
-        if (op_id.index() < log_index) {
-          any_behind = true;
-          break;
-        }
-      }
-      if (!any_behind) return Status::OK();
-    } else {
-      LOG(WARNING) << "Got error getting last opid for each replica: " << s.ToString();
-    }
-    passed = MonoTime::Now() - start;
-    if (passed > timeout) {
-      break;
-    }
-    SleepFor(MonoDelta::FromMilliseconds(50));
-  }
-  string replicas_str;
-  for (const TServerDetails* replica : replicas) {
-    if (!replicas_str.empty()) replicas_str += ", ";
-    replicas_str += "{ " + replica->ToString() + " }";
-  }
-  return Status::TimedOut(Substitute("Index $0 not available on all replicas after $1. "
-                                     "Replicas: [ $2 ]",
-                                     log_index, passed.ToString(), replicas_str));
-}
-
 Status CreateTabletServerMap(const shared_ptr<MasterServiceProxy>& master_proxy,
                              const shared_ptr<Messenger>& messenger,
                              unordered_map<string, TServerDetails*>* ts_map) {
diff --git a/src/kudu/integration-tests/cluster_itest_util.h b/src/kudu/integration-tests/cluster_itest_util.h
index a8394fa..2d5b02c 100644
--- a/src/kudu/integration-tests/cluster_itest_util.h
+++ b/src/kudu/integration-tests/cluster_itest_util.h
@@ -110,15 +110,16 @@ Status CreateTabletServerMap(const std::shared_ptr<master::MasterServiceProxy>&
 
 // Gets a vector containing the latest OpId for each of the given replicas.
 // Returns a bad Status if any replica cannot be reached.
-Status GetLastOpIdForEachReplica(const std::string& tablet_id,
-                                 const std::vector<TServerDetails*>& replicas,
-                                 consensus::OpIdType opid_type,
-                                 const MonoDelta& timeout,
-                                 std::vector<consensus::OpId>* op_ids);
+Status GetLastOpIdForEachReplica(
+    const std::string& tablet_id,
+    const std::vector<const TServerDetails*>& replicas,
+    consensus::OpIdType opid_type,
+    const MonoDelta& timeout,
+    std::vector<consensus::OpId>* op_ids);
 
 // Like the above, but for a single replica.
 Status GetLastOpIdForReplica(const std::string& tablet_id,
-                             TServerDetails* replica,
+                             const TServerDetails* replica,
                              consensus::OpIdType opid_type,
                              const MonoDelta& timeout,
                              consensus::OpId* op_id);
@@ -144,14 +145,6 @@ Status WaitForServersToAgree(
     int64_t minimum_index,
     consensus::OpIdType op_id_type = consensus::RECEIVED_OPID);
 
-// Wait until all specified replicas have logged at least the given index.
-// Unlike WaitForServersToAgree(), the servers do not actually have to converge
-// or quiesce. They only need to progress to or past the given index.
-Status WaitUntilAllReplicasHaveOp(const int64_t log_index,
-                                  const std::string& tablet_id,
-                                  const std::vector<TServerDetails*>& replicas,
-                                  const MonoDelta& timeout);
-
 // Get the consensus state from the given replica.
 Status GetConsensusState(const TServerDetails* replica,
                          const std::string& tablet_id,
diff --git a/src/kudu/integration-tests/ts_itest-base.cc b/src/kudu/integration-tests/ts_itest-base.cc
index 2e6f742..99b9aab 100644
--- a/src/kudu/integration-tests/ts_itest-base.cc
+++ b/src/kudu/integration-tests/ts_itest-base.cc
@@ -37,7 +37,6 @@
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/metadata.pb.h"
-#include "kudu/consensus/opid.pb.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/split.h"
@@ -445,26 +444,6 @@ void TabletServerIntegrationTestBase::GetOnlyLiveFollowerReplicas(
   }
 }
 
-// Return the index within 'replicas' for the replica which is farthest ahead.
-int64_t TabletServerIntegrationTestBase::GetFurthestAheadReplicaIdx(
-    const string& tablet_id, const vector<TServerDetails*>& replicas) {
-  vector<consensus::OpId> op_ids;
-  CHECK_OK(GetLastOpIdForEachReplica(tablet_id, replicas, consensus::RECEIVED_OPID,
-                                     MonoDelta::FromSeconds(10), &op_ids));
-  int64_t max_index = 0;
-  int max_replica_index = -1;
-  for (int i = 0; i < op_ids.size(); i++) {
-    if (op_ids[i].index() > max_index) {
-      max_index = op_ids[i].index();
-      max_replica_index = i;
-    }
-  }
-
-  CHECK_NE(max_replica_index, -1);
-
-  return max_replica_index;
-}
-
 Status TabletServerIntegrationTestBase::ShutdownServerWithUUID(const string& uuid) {
   for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
     ExternalTabletServer* ts = cluster_->tablet_server(i);
diff --git a/src/kudu/integration-tests/ts_itest-base.h b/src/kudu/integration-tests/ts_itest-base.h
index b0ca73d..e49d33f 100644
--- a/src/kudu/integration-tests/ts_itest-base.h
+++ b/src/kudu/integration-tests/ts_itest-base.h
@@ -16,7 +16,6 @@
 // under the License.
 #pragma once
 
-#include <cstdint>
 #include <memory>
 #include <string>
 #include <unordered_set>
@@ -98,10 +97,6 @@ class TabletServerIntegrationTestBase : public TabletServerTestBase {
   void GetOnlyLiveFollowerReplicas(const std::string& tablet_id,
                                    std::vector<itest::TServerDetails*>* followers);
 
-  // Return the index within 'replicas' for the replica which is farthest ahead.
-  int64_t GetFurthestAheadReplicaIdx(const std::string& tablet_id,
-                                     const std::vector<itest::TServerDetails*>& replicas);
-
   Status ShutdownServerWithUUID(const std::string& uuid);
 
   Status RestartServerWithUUID(const std::string& uuid);