Skip to content

Commit

Permalink
[consensus] remove ForTests suffix from WaitUntilLeader
Browse files Browse the repository at this point in the history
I found WaitUntilLeaderForTests useful in making the master rebuilder
tool more robust to slow startup. Since I intend on using it outside of
tests, I removed the ForTests suffix.

I also left a note in the header mentioning it should be used sparingly,
given the implementation is admittedly simple and not suited for heavy
usage.

This patch contains no functional changes.

Change-Id: I4f5d61a59651c3a5f11e317a77c0344f5dd3e707
Reviewed-on: http://gerrit.cloudera.org:8080/17763
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <[email protected]>
  • Loading branch information
andrwng committed Aug 10, 2021
1 parent a5307b3 commit 6be5282
Show file tree
Hide file tree
Showing 11 changed files with 36 additions and 45 deletions.
2 changes: 1 addition & 1 deletion src/kudu/consensus/raft_consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -547,7 +547,7 @@ Status RaftConsensus::StartElection(ElectionMode mode, ElectionReason reason) {
return Status::OK();
}

Status RaftConsensus::WaitUntilLeaderForTests(const MonoDelta& timeout) {
Status RaftConsensus::WaitUntilLeader(const MonoDelta& timeout) {
MonoTime deadline = MonoTime::Now() + timeout;
while (role() != consensus::RaftPeerPB::LEADER) {
if (MonoTime::Now() >= deadline) {
Expand Down
5 changes: 4 additions & 1 deletion src/kudu/consensus/raft_consensus.h
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,10 @@ class RaftConsensus : public std::enable_shared_from_this<RaftConsensus>,

// Wait until the node has LEADER role.
// Returns Status::TimedOut if the role is not LEADER within 'timeout'.
Status WaitUntilLeaderForTests(const MonoDelta& timeout);
// NOTE: the implementation is a busy loop; as such, this method should be
// used sparingly, e.g. only in tests, or in applications that don't require
// high concurrency.
Status WaitUntilLeader(const MonoDelta& timeout) WARN_UNUSED_RESULT;

// Return a copy of the failure detector instance. Only for use in tests.
std::shared_ptr<rpc::PeriodicTimer> GetFailureDetectorForTests() const {
Expand Down
14 changes: 1 addition & 13 deletions src/kudu/consensus/raft_consensus_quorum-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -96,18 +96,6 @@ const char* kTestTablet = "TestTablet";
void DoNothing(const string& s) {
}

Status WaitUntilLeaderForTests(RaftConsensus* raft) {
MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(15);
while (MonoTime::Now() < deadline) {
if (raft->role() == RaftPeerPB::LEADER) {
return Status::OK();
}
SleepFor(MonoDelta::FromMilliseconds(10));
}

return Status::TimedOut("Timed out waiting to become leader");
}

// Test suite for tests that focus on multiple peer interaction, but
// without integrating with other components, such as ops.
class RaftConsensusQuorumTest : public KuduTest {
Expand Down Expand Up @@ -902,7 +890,7 @@ TEST_F(RaftConsensusQuorumTest, TestLeaderElectionWithQuiescedQuorum) {
LOG(INFO) << "Running election for future leader with index " << (current_config_size - 1);
ASSERT_OK(new_leader->StartElection(RaftConsensus::ELECT_EVEN_IF_LEADER_IS_ALIVE,
RaftConsensus::EXTERNAL_REQUEST));
WaitUntilLeaderForTests(new_leader.get());
ASSERT_OK(new_leader->WaitUntilLeader(MonoDelta::FromSeconds(15)));
LOG(INFO) << "Election won";
int64_t flush_count_after =
new_leader->consensus_metadata_for_tests()->flush_count_for_tests();
Expand Down
2 changes: 1 addition & 1 deletion src/kudu/integration-tests/alter_table-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ class AlterTableTest : public KuduTest {

if (num_replicas() == 1) {
tablet_replica_ = LookupTabletReplica();
ASSERT_OK(tablet_replica_->consensus()->WaitUntilLeaderForTests(MonoDelta::FromSeconds(10)));
ASSERT_OK(tablet_replica_->consensus()->WaitUntilLeader(MonoDelta::FromSeconds(10)));
}
LOG(INFO) << "Tablet successfully located";
}
Expand Down
2 changes: 1 addition & 1 deletion src/kudu/integration-tests/fuzz-itest.cc
Original file line number Diff line number Diff line change
Expand Up @@ -382,7 +382,7 @@ class FuzzTest : public KuduTest {

Status CallParticipantOpCheckResp(int64_t txn_id, ParticipantOpPB::ParticipantOpType op_type,
int64_t ts_val) {
RETURN_NOT_OK(tablet_replica_->consensus()->WaitUntilLeaderForTests(
RETURN_NOT_OK(tablet_replica_->consensus()->WaitUntilLeader(
MonoDelta::FromSeconds(10)));
ParticipantResponsePB resp;
RETURN_NOT_OK(CallParticipantOp(tablet_replica_.get(), txn_id, op_type, ts_val, &resp));
Expand Down
2 changes: 1 addition & 1 deletion src/kudu/integration-tests/ts_tablet_manager-itest.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1082,7 +1082,7 @@ class TxnStatusTabletManagementTest : public TsTabletManagerITest {

// Wait for the tablet to be in RUNNING state and its consensus running too.
RETURN_NOT_OK(r->WaitUntilConsensusRunning(kTimeout));
auto s = r->consensus()->WaitUntilLeaderForTests(kTimeout);
auto s = r->consensus()->WaitUntilLeader(kTimeout);
if (replica) {
*replica = std::move(r);
}
Expand Down
42 changes: 21 additions & 21 deletions src/kudu/integration-tests/txn_participant-itest.cc
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,7 @@ TEST_F(TxnParticipantITest, TestReplicateParticipantOps) {
// tserver so we can ensure a specific leader.
const int kLeaderIdx = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
// Try submitting the ops on all replicas. They should succeed on the leaders
// and fail on followers.
const int64_t kTxnId = 1;
Expand Down Expand Up @@ -369,7 +369,7 @@ TEST_P(ParticipantCopyITest, TestCopyParticipantOps) {
constexpr const int kDeadServerIdx = kLeaderIdx + 1;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));

// Apply some operations.
vector<TxnParticipant::TxnEntry> expected_txns;
Expand Down Expand Up @@ -442,7 +442,7 @@ TEST_F(TxnParticipantITest, TestWaitOnFinalizeCommit) {
auto* follower_replica = replicas[kLeaderIdx + 1];
auto* clock = leader_replica->clock();
const int64_t kTxnId = 1;
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
ASSERT_OK(RunOnReplica(leader_replica, kTxnId, ParticipantOpPB::BEGIN_TXN));
const MonoDelta kAgreeTimeout = kDefaultTimeout;
const auto& tablet_id = leader_replica->tablet()->tablet_id();
Expand Down Expand Up @@ -503,7 +503,7 @@ TEST_F(TxnParticipantITest, TestWaitOnAbortCommit) {
auto* follower_replica = replicas[kLeaderIdx + 1];
auto* clock = leader_replica->clock();
const int64_t kTxnId = 1;
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
ASSERT_OK(RunOnReplica(leader_replica, kTxnId, ParticipantOpPB::BEGIN_TXN));
const MonoDelta kAgreeTimeout = kDefaultTimeout;
const auto& tablet_id = leader_replica->tablet()->tablet_id();
Expand Down Expand Up @@ -541,7 +541,7 @@ TEST_F(TxnParticipantITest, TestProxyBasicCalls) {
constexpr const int kLeaderIdx = 0;
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
auto admin_proxy = cluster_->tserver_admin_proxy(kLeaderIdx);
for (const auto& op : kCommitSequence) {
const auto req = ParticipantRequest(replicas[kLeaderIdx]->tablet_id(), kTxnId, op);
Expand All @@ -555,7 +555,7 @@ TEST_F(TxnParticipantITest, TestBeginCommitAfterFinalize) {
constexpr const int kLeaderIdx = 0;
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
auto admin_proxy = cluster_->tserver_admin_proxy(kLeaderIdx);
const auto tablet_id = replicas[kLeaderIdx]->tablet_id();
{
Expand Down Expand Up @@ -598,7 +598,7 @@ TEST_F(TxnParticipantITest, TestProxyErrorWhenNotBegun) {
constexpr const int kLeaderIdx = 0;
auto txn_id = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
auto admin_proxy = cluster_->tserver_admin_proxy(kLeaderIdx);
const auto tablet_id = replicas[kLeaderIdx]->tablet_id();
for (auto type : { ParticipantOpPB::BEGIN_COMMIT,
Expand All @@ -619,7 +619,7 @@ TEST_F(TxnParticipantITest, TestProxyIllegalStatesInCommitSequence) {
constexpr const int kLeaderIdx = 0;
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
auto admin_proxy = cluster_->tserver_admin_proxy(kLeaderIdx);

// Begin after already beginning.
Expand Down Expand Up @@ -703,7 +703,7 @@ TEST_F(TxnParticipantITest, TestProxyIllegalStatesInAbortSequence) {
constexpr const int kLeaderIdx = 0;
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
auto admin_proxy = cluster_->tserver_admin_proxy(kLeaderIdx);

// Try our illegal ops when our transaction is open.
Expand Down Expand Up @@ -755,7 +755,7 @@ TEST_F(TxnParticipantITest, TestProxyNonLeader) {
constexpr const int kNonLeaderIdx = kLeaderIdx + 1;
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeader(kDefaultTimeout));
auto admin_proxy = cluster_->tserver_admin_proxy(kNonLeaderIdx);
for (const auto& op : kCommitSequence) {
const auto req = ParticipantRequest(replicas[kLeaderIdx]->tablet_id(), kTxnId, op);
Expand All @@ -774,7 +774,7 @@ TEST_F(TxnParticipantITest, TestProxyTabletBootstrapping) {
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));

FLAGS_tablet_bootstrap_inject_latency_ms = 1000;
cluster_->mini_tablet_server(kLeaderIdx)->Shutdown();
Expand All @@ -799,7 +799,7 @@ TEST_F(TxnParticipantITest, TestProxyTabletNotRunning) {
constexpr const int kTxnId = 0;
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
auto* tablet_manager = cluster_->mini_tablet_server(kLeaderIdx)->server()->tablet_manager();
ASSERT_OK(tablet_manager->DeleteTablet(leader_replica->tablet_id(),
tablet::TABLET_DATA_TOMBSTONED, boost::none));
Expand Down Expand Up @@ -842,7 +842,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientGetMetadata) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));

// Get commit-related metadata.
TxnMetadataPB meta_pb;
Expand Down Expand Up @@ -900,7 +900,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientBeginTxnDoesntLock) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));

// Start a transaction and make sure it results in the expected state
// server-side.
Expand All @@ -926,7 +926,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientCommitSequence) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));

// Start a transaction and make sure it results in the expected state
// server-side.
Expand Down Expand Up @@ -1000,7 +1000,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientAbortSequence) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
unique_ptr<TxnSystemClient> txn_client;
ASSERT_OK(TxnSystemClient::Create(cluster_->master_rpc_addrs(),
cluster_->messenger()->sasl_proto_name(),
Expand Down Expand Up @@ -1056,7 +1056,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientErrorWhenNotBegun) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
unique_ptr<TxnSystemClient> txn_client;
ASSERT_OK(TxnSystemClient::Create(cluster_->master_rpc_addrs(),
cluster_->messenger()->sasl_proto_name(),
Expand All @@ -1082,7 +1082,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientRepeatCalls) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
unique_ptr<TxnSystemClient> txn_client;
ASSERT_OK(TxnSystemClient::Create(cluster_->master_rpc_addrs(),
cluster_->messenger()->sasl_proto_name(),
Expand Down Expand Up @@ -1115,7 +1115,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientTimeoutWhenNoMajority) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
// Bring down the other servers so we can't get a majority.
for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
if (i == kLeaderIdx) continue;
Expand Down Expand Up @@ -1176,7 +1176,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientSucceedsOnBootstrap) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
// Start a thread that sends participant ops to the tablet.
int next_txn_id = 0;
unique_ptr<TxnSystemClient> txn_client;
Expand Down Expand Up @@ -1230,7 +1230,7 @@ TEST_F(TxnParticipantITest, TestTxnSystemClientRetriesWhenReplicaNotFound) {
vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
auto* leader_replica = replicas[kLeaderIdx];
const auto tablet_id = leader_replica->tablet_id();
ASSERT_OK(leader_replica->consensus()->WaitUntilLeaderForTests(kDefaultTimeout));
ASSERT_OK(leader_replica->consensus()->WaitUntilLeader(kDefaultTimeout));
// Start a thread that sends participant ops to the tablet.
int next_txn_id = 0;
unique_ptr<TxnSystemClient> txn_client;
Expand Down
4 changes: 2 additions & 2 deletions src/kudu/tablet/tablet_replica-test-base.cc
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ Status TabletReplicaTestBase::StartReplica(const ConsensusBootstrapInfo& info) {

Status TabletReplicaTestBase::StartReplicaAndWaitUntilLeader(const ConsensusBootstrapInfo& info) {
RETURN_NOT_OK(StartReplica(info));
return tablet_replica_->consensus()->WaitUntilLeaderForTests(kLeadershipTimeout);
return tablet_replica_->consensus()->WaitUntilLeader(kLeadershipTimeout);
}

Status TabletReplicaTestBase::RestartReplica(bool reset_tablet) {
Expand Down Expand Up @@ -226,7 +226,7 @@ Status TabletReplicaTestBase::RestartReplica(bool reset_tablet) {
prepare_pool_.get(),
dns_resolver_.get()));
// Wait for the replica to be usable.
return tablet_replica_->consensus()->WaitUntilLeaderForTests(kLeadershipTimeout);
return tablet_replica_->consensus()->WaitUntilLeader(kLeadershipTimeout);
}

} // namespace tablet
Expand Down
4 changes: 2 additions & 2 deletions src/kudu/tserver/tablet_copy_source_session-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,9 @@
#include "kudu/common/common.pb.h"
#include "kudu/common/partial_row.h"
#include "kudu/common/row_operations.h"
#include "kudu/common/row_operations.pb.h"
#include "kudu/common/schema.h"
#include "kudu/common/wire_protocol.h"
#include "kudu/common/wire_protocol.pb.h"
#include "kudu/consensus/consensus_meta_manager.h"
#include "kudu/consensus/log.h"
#include "kudu/consensus/log_anchor_registry.h"
Expand Down Expand Up @@ -189,7 +189,7 @@ class TabletCopyTest : public KuduTabletTest {
prepare_pool_.get(),
dns_resolver_.get()));
ASSERT_OK(tablet_replica_->WaitUntilConsensusRunning(MonoDelta::FromSeconds(10)));
ASSERT_OK(tablet_replica_->consensus()->WaitUntilLeaderForTests(MonoDelta::FromSeconds(10)));
ASSERT_OK(tablet_replica_->consensus()->WaitUntilLeader(MonoDelta::FromSeconds(10)));
}

void TabletReplicaStateChangedCallback(const string& tablet_id, const string& reason) {
Expand Down
2 changes: 1 addition & 1 deletion src/kudu/tserver/tablet_server-test-base.cc
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ Status TabletServerTestBase::WaitForTabletRunning(const char *tablet_id) {
RETURN_NOT_OK(tablet_manager->GetTabletReplica(tablet_id, &tablet_replica));
RETURN_NOT_OK(tablet_replica->WaitUntilConsensusRunning(kTimeout));
RETURN_NOT_OK(
tablet_replica->consensus()->WaitUntilLeaderForTests(kTimeout));
tablet_replica->consensus()->WaitUntilLeader(kTimeout));

// KUDU-2463: Even though the tablet thinks its leader, for correctness, it
// must wait to finish replicating its no-op (even as a single replica)
Expand Down
2 changes: 1 addition & 1 deletion src/kudu/tserver/ts_tablet_manager-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ class TsTabletManagerTest : public KuduTest {
}

RETURN_NOT_OK(tablet_replica->WaitUntilConsensusRunning(MonoDelta::FromMilliseconds(2000)));
return tablet_replica->consensus()->WaitUntilLeaderForTests(MonoDelta::FromSeconds(10));
return tablet_replica->consensus()->WaitUntilLeader(MonoDelta::FromSeconds(10));
}

void GenerateFullTabletReport(TabletReportPB* report) {
Expand Down

0 comments on commit 6be5282

Please sign in to comment.