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/11/29 10:29:08 UTC

kudu git commit: KUDU-1097: scenario for fallen-behind-the-WAL replica

Repository: kudu
Updated Branches:
  refs/heads/master 5c87c22be -> aafed6d1d


KUDU-1097: scenario for fallen-behind-the-WAL replica

Added end-to-end test scenario for the case when an existing follower
replica falls behind the WAL GC threshold.  First, the system attempts
to replace it with a non-voter where tablet copying fails.  As soon as
that detected, a new non-voter replica is added and the replacement
succeeds, so the tablet configuration eventually gets rid of failed
replicas and contains exactly 3 healthy replicas.

Change-Id: Ifcaff45c9fa3beb51d99dc53d88138da7dc120af
Reviewed-on: http://gerrit.cloudera.org:8080/8675
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: aafed6d1d749978899a78dae84dd25c028a1763a
Parents: 5c87c22
Author: Alexey Serbin <as...@cloudera.com>
Authored: Tue Nov 28 15:56:31 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Nov 29 10:28:17 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/quorum_util-test.cc          |  18 +++
 src/kudu/consensus/quorum_util.cc               |  13 +-
 .../raft_consensus-itest-base.cc                |  12 +-
 .../raft_consensus-itest-base.h                 |   4 +-
 .../integration-tests/raft_consensus-itest.cc   |   9 +-
 .../raft_consensus_election-itest.cc            |   3 +-
 .../raft_consensus_nonvoter-itest.cc            | 147 +++++++++++++++++++
 src/kudu/master/catalog_manager.cc              |   4 +
 src/kudu/tserver/tablet_copy_service.cc         |   7 +-
 9 files changed, 202 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/consensus/quorum_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util-test.cc b/src/kudu/consensus/quorum_util-test.cc
index cede1db..461ae7a 100644
--- a/src/kudu/consensus/quorum_util-test.cc
+++ b/src/kudu/consensus/quorum_util-test.cc
@@ -382,6 +382,24 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     EXPECT_FALSE(IsUnderReplicated(config, 2));
     EXPECT_TRUE(IsUnderReplicated(config, 3));
   }
+  {
+    RaftConfigPB config;
+    AddPeer(&config, "A", V, '-');
+    AddPeer(&config, "B", V, '+');
+    AddPeer(&config, "C", V, '+');
+    AddPeer(&config, "D", N, '-', {{"PROMOTE", true}});
+    AddPeer(&config, "E", N, '+', {{"PROMOTE", true}});
+    EXPECT_FALSE(IsUnderReplicated(config, 3));
+  }
+  {
+    RaftConfigPB config;
+    AddPeer(&config, "A", V, '-');
+    AddPeer(&config, "B", V, '+');
+    AddPeer(&config, "C", V, '+');
+    AddPeer(&config, "D", N, '-', {{"PROMOTE", true}});
+    AddPeer(&config, "E", N, '+', {{"PROMOTE", false}});
+    EXPECT_TRUE(IsUnderReplicated(config, 3));
+  }
 }
 
 // Verify logic of the kudu::consensus::CanEvictReplica(), anticipating

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/consensus/quorum_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util.cc b/src/kudu/consensus/quorum_util.cc
index e002070..8632e3c 100644
--- a/src/kudu/consensus/quorum_util.cc
+++ b/src/kudu/consensus/quorum_util.cc
@@ -34,6 +34,7 @@
 
 using google::protobuf::RepeatedPtrField;
 using kudu::pb_util::SecureShortDebugString;
+using kudu::pb_util::SecureDebugString;
 using std::map;
 using std::pair;
 using std::set;
@@ -408,6 +409,7 @@ bool IsUnderReplicated(const RaftConfigPB& config, int replication_factor) {
   // While working with the optional fields related to per-replica health status
   // and attributes, has_a_field()-like methods are not called because of
   // the appropriate default values of those fields.
+  VLOG(2) << "config to evaluate: " << SecureDebugString(config);
   for (const RaftPeerPB& peer : config.peers()) {
     switch (peer.member_type()) {
       case RaftPeerPB::VOTER:
@@ -428,8 +430,11 @@ bool IsUnderReplicated(const RaftConfigPB& config, int replication_factor) {
         break;
     }
   }
-  return replication_factor > (num_voters_total - num_voters_need_replacement) +
-      num_non_voters_to_promote;
+  const bool is_under_replicated = replication_factor >
+      (num_voters_total - num_voters_need_replacement) + num_non_voters_to_promote;
+  VLOG(2) << "decision: the config is" << (is_under_replicated ? " " : " not ")
+          << "under-replicated";
+  return is_under_replicated;
 }
 
 // Whether there is an excess replica to evict.
@@ -453,6 +458,7 @@ bool CanEvictReplica(const RaftConfigPB& config,
   // While working with the optional fields related to per-replica health status
   // and attributes, has_a_field()-like methods are not called because of
   // the appropriate default values of those fields.
+  VLOG(2) << "config to evaluate: " << SecureDebugString(config);
   for (const RaftPeerPB& peer : config.peers()) {
     DCHECK(peer.has_permanent_uuid() && !peer.permanent_uuid().empty());
     const string& peer_uuid = peer.permanent_uuid();
@@ -566,6 +572,9 @@ bool CanEvictReplica(const RaftConfigPB& config,
       *uuid_to_evict = to_evict;
     }
   }
+  VLOG(2) << "decision: can"
+          << (can_evict ? "" : "not") << " evict replica "
+          << (can_evict ? to_evict : "");
   return can_evict;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/integration-tests/raft_consensus-itest-base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest-base.cc b/src/kudu/integration-tests/raft_consensus-itest-base.cc
index 1c7a0e1..5b5686d 100644
--- a/src/kudu/integration-tests/raft_consensus-itest-base.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest-base.cc
@@ -190,13 +190,14 @@ void RaftConsensusITestBase::AddFlagsForLogRolls(vector<string>* extra_tserver_f
 }
 
 void RaftConsensusITestBase::CauseFollowerToFallBehindLogGC(
+    const itest::TabletServerMap& tablet_servers,
     string* leader_uuid,
     int64_t* orig_term,
     string* fell_behind_uuid) {
   MonoDelta kTimeout = MonoDelta::FromSeconds(10);
   // Wait for all of the replicas to have acknowledged the elected
   // leader and logged the first NO_OP.
-  ASSERT_OK(WaitForServersToAgree(kTimeout, tablet_servers_, tablet_id_, 1));
+  ASSERT_OK(WaitForServersToAgree(kTimeout, tablet_servers, tablet_id_, 1));
 
   // Pause one server. This might be the leader, but pausing it will cause
   // a leader election to happen.
@@ -261,10 +262,11 @@ void RaftConsensusITestBase::CauseFollowerToFallBehindLogGC(
   ASSERT_OK(replica_ets->Resume());
 
   // Ensure that none of the tablet servers crashed.
-  for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
-    // Make sure it didn't crash.
-    ASSERT_TRUE(cluster_->tablet_server(i)->IsProcessAlive())
-      << "Tablet server " << i << " crashed";
+  for (const auto& e: tablet_servers) {
+  //for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
+    // Make sure the involved servsers didn't crash.
+    ASSERT_TRUE(cluster_->tablet_server_by_uuid(e.first)->IsProcessAlive())
+        << "Tablet server " << e.first << " crashed";
   }
   *fell_behind_uuid = replica->uuid();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/integration-tests/raft_consensus-itest-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest-base.h b/src/kudu/integration-tests/raft_consensus-itest-base.h
index 18aa82a..8785e0b 100644
--- a/src/kudu/integration-tests/raft_consensus-itest-base.h
+++ b/src/kudu/integration-tests/raft_consensus-itest-base.h
@@ -21,6 +21,7 @@
 #include <string>
 #include <vector>
 
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/ts_itest-base.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/status.h"
@@ -67,7 +68,8 @@ class RaftConsensusITestBase : public TabletServerIntegrationTestBase {
   //
   // Certain flags should be set. You can add the required flags with
   // AddFlagsForLogRolls() before starting the cluster.
-  void CauseFollowerToFallBehindLogGC(std::string* leader_uuid,
+  void CauseFollowerToFallBehindLogGC(const itest::TabletServerMap& tablet_servers,
+                                      std::string* leader_uuid,
                                       int64_t* orig_term,
                                       std::string* fell_behind_uuid);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index 3ceed43..f441024 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -843,7 +843,8 @@ TEST_F(RaftConsensusITest, TestFollowerFallsBehindLeaderGC) {
   string leader_uuid;
   int64_t orig_term;
   string follower_uuid;
-  NO_FATALS(CauseFollowerToFallBehindLogGC(&leader_uuid, &orig_term, &follower_uuid));
+  NO_FATALS(CauseFollowerToFallBehindLogGC(
+      tablet_servers_, &leader_uuid, &orig_term, &follower_uuid));
   SCOPED_TRACE(Substitute("leader: $0 follower: $1", leader_uuid, follower_uuid));
 
   // Wait for remaining majority to agree.
@@ -2244,7 +2245,8 @@ TEST_F(RaftConsensusITest, TestEvictAbandonedFollowers) {
   string leader_uuid;
   int64_t orig_term;
   string follower_uuid;
-  NO_FATALS(CauseFollowerToFallBehindLogGC(&leader_uuid, &orig_term, &follower_uuid));
+  NO_FATALS(CauseFollowerToFallBehindLogGC(
+      tablet_servers_, &leader_uuid, &orig_term, &follower_uuid));
 
   // Wait for the abandoned follower to be evicted.
   ASSERT_OK(WaitUntilCommittedConfigNumVotersIs(2, tablet_servers_[leader_uuid],
@@ -2265,7 +2267,8 @@ TEST_F(RaftConsensusITest, TestMasterReplacesEvictedFollowers) {
   string leader_uuid;
   int64_t orig_term;
   string follower_uuid;
-  NO_FATALS(CauseFollowerToFallBehindLogGC(&leader_uuid, &orig_term, &follower_uuid));
+  NO_FATALS(CauseFollowerToFallBehindLogGC(
+      tablet_servers_, &leader_uuid, &orig_term, &follower_uuid));
 
   // The follower will be evicted. Now wait for the master to cause it to be
   // copied.

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/integration-tests/raft_consensus_election-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus_election-itest.cc b/src/kudu/integration-tests/raft_consensus_election-itest.cc
index d537738..92d0215 100644
--- a/src/kudu/integration-tests/raft_consensus_election-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus_election-itest.cc
@@ -485,7 +485,8 @@ TEST_F(RaftConsensusElectionITest, TombstonedVoteAfterFailedTabletCopy) {
   string leader_uuid;
   int64_t orig_term;
   string follower_uuid;
-  NO_FATALS(CauseFollowerToFallBehindLogGC(&leader_uuid, &orig_term, &follower_uuid));
+  NO_FATALS(CauseFollowerToFallBehindLogGC(
+      active_tablet_servers, &leader_uuid, &orig_term, &follower_uuid));
 
   // Wait for the abandoned follower to be evicted.
   ASSERT_OK(WaitUntilCommittedConfigNumVotersIs(2, tablet_servers_[leader_uuid],

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
index 8c37676..e8554dd 100644
--- a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
@@ -45,6 +45,7 @@
 #include "kudu/tserver/tablet_server-test-base.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
@@ -118,6 +119,10 @@ class RaftConsensusNonVoterITest : public RaftConsensusITestBase {
   // Get a tablet server with at least one replica of the test tablet identified
   // by the 'tablet_id_' member.
   ExternalTabletServer* GetServerWithReplica() const;
+
+  // Get tablet servers not hosting replicas of the test tablet identified
+  // by the 'tablet_id_' member.
+  void GetServersWithoutReplica(vector<ExternalTabletServer*>* servers) const;
 };
 
 Status RaftConsensusNonVoterITest::GetTabletCopySourceSessionsCount(
@@ -208,6 +213,27 @@ ExternalTabletServer* RaftConsensusNonVoterITest::GetServerWithReplica() const {
   return ts;
 }
 
+void RaftConsensusNonVoterITest::GetServersWithoutReplica(
+    vector<ExternalTabletServer*>* servers) const {
+  ASSERT_NE(nullptr, servers);
+  servers->clear();
+
+  TabletServerMap replica_servers;
+  for (const auto& e : tablet_replicas_) {
+    if (e.first == tablet_id_) {
+      replica_servers.emplace(e.second->uuid(), e.second);
+    }
+  }
+
+  for (const auto& ts : tablet_servers_) {
+    if (replica_servers.find(ts.first) == replica_servers.end()) {
+      auto* s = cluster_->tablet_server_by_uuid(ts.second->uuid());
+      ASSERT_NE(nullptr, s);
+      servers->push_back(s);
+    }
+  }
+}
+
 // Ensure that adding a NON_VOTER replica is properly handled by the system:
 //
 //   * Updating Raft configuration for tablet by adding a NON_VOTER replica
@@ -1158,5 +1184,126 @@ TEST_F(RaftConsensusNonVoterITest, TabletServerIsGoneAndBack) {
   NO_FATALS(cluster_->AssertNoCrashes());
 }
 
+// A two-step sceanario: first, an existing tablet replica fails because it's
+// fails behind the threshold of the GCed WAL segment threshold. The catalog
+// manager should notice that and add a new non-voter replica in attempt to
+// replace the failed replica. Then, the newly added non-voter replica becomes
+// unavailable before completing the tablet copy phase. The catalog manager
+// should add a new non-voter replica to make it possible to replace the failed
+// voter replica, so eventually the tablet has appropriate number of functional
+// replicas to guarantee the tablet's replication factor.
+TEST_F(RaftConsensusNonVoterITest, FailedTabletCopy) {
+  if (!AllowSlowTests()) {
+    LOG(WARNING) << "test is skipped; set KUDU_ALLOW_SLOW_TESTS=1 to run";
+    return;
+  }
+
+  const auto kReplicasNum = 3;
+  const auto kConsensusRpcTimeout = MonoDelta::FromSeconds(5);
+  const auto kTimeoutSec = 60;
+  const auto kTimeout = MonoDelta::FromSeconds(kTimeoutSec);
+  FLAGS_num_replicas = kReplicasNum;
+  // Need two extra tablet servers for the scenario.
+  FLAGS_num_tablet_servers = kReplicasNum + 2;
+
+  const vector<string> kMasterFlags = {
+    // The scenario runs with the 3-4-3 replica management scheme.
+    "--raft_prepare_replacement_before_eviction=true",
+
+    // Detect unavailable tablet servers faster.
+    Substitute("--tserver_unresponsive_timeout_ms=$0",
+        kConsensusRpcTimeout.ToMilliseconds() / 2),
+  };
+  vector<string> tserver_flags = {
+    // The scenario runs with the 3-4-3 replica management scheme.
+    "--raft_prepare_replacement_before_eviction=true",
+
+    Substitute("--follower_unavailable_considered_failed_sec=$0",
+        2 * static_cast<int>(kConsensusRpcTimeout.ToSeconds())),
+
+    // Don't wait for the RPC timeout for too long.
+    Substitute("--consensus_rpc_timeout_ms=$0", kConsensusRpcTimeout.ToMilliseconds()),
+  };
+  AddFlagsForLogRolls(&tserver_flags); // For CauseFollowerToFallBehindLogGC().
+
+  NO_FATALS(BuildAndStart(tserver_flags, kMasterFlags));
+  ASSERT_EQ(kReplicasNum + 2, tablet_servers_.size());
+  ASSERT_EQ(kReplicasNum, tablet_replicas_.size());
+
+  // Make sure the tablet copy will fail on one of tablet servers without
+  // tablet replicas.
+  vector<ExternalTabletServer*> servers_without_replica;
+  NO_FATALS(GetServersWithoutReplica(&servers_without_replica));
+  ASSERT_EQ(2, servers_without_replica.size());
+  ExternalTabletServer* ts0 = *servers_without_replica.begin();
+  ASSERT_OK(cluster_->SetFlag(ts0,
+                              "tablet_copy_fault_crash_on_fetch_all", "1.0"));
+  // Make sure the second tablet server is not available as a candidate for
+  // the new non-voter replica.
+  ExternalTabletServer* ts1 = *servers_without_replica.rbegin();
+  ts1->Shutdown();
+  SleepFor(kConsensusRpcTimeout);
+
+  string follower_uuid;
+  {
+    TabletServerMap replica_servers;
+    for (const auto& e : tablet_replicas_) {
+      if (e.first == tablet_id_) {
+        replica_servers.emplace(e.second->uuid(), e.second);
+      }
+    }
+    string leader_uuid;
+    int64_t orig_term;
+    NO_FATALS(CauseFollowerToFallBehindLogGC(
+        replica_servers, &leader_uuid, &orig_term, &follower_uuid));
+  }
+
+  // The leader replica marks the non-responsive replica as failed after it
+  // realized the replica would not be able to catch up, and the catalog
+  // manager should add a new non-voter as a replacement.
+  bool has_leader = false;
+  TabletLocationsPB tablet_locations;
+  ASSERT_OK(WaitForReplicasReportedToMaster(cluster_->master_proxy(),
+                                            kReplicasNum + 1,
+                                            tablet_id_,
+                                            kTimeout,
+                                            WAIT_FOR_LEADER,
+                                            &has_leader,
+                                            &tablet_locations));
+  // Make the second server available for placing a non-voter replica.
+  ASSERT_OK(ts1->Restart());
+
+  // The system should be able to recover, keeping only necessary number of
+  // voter replicas and purging the failed voters and non-voters.
+  ASSERT_OK(WaitForReplicasReportedToMaster(cluster_->master_proxy(),
+                                            kReplicasNum,
+                                            tablet_id_,
+                                            kTimeout,
+                                            WAIT_FOR_LEADER,
+                                            &has_leader,
+                                            &tablet_locations));
+  consensus::ConsensusStatePB cstate;
+  ASSERT_EVENTUALLY([&] {
+    TServerDetails* leader = nullptr;
+    ASSERT_OK(GetLeaderReplicaWithRetries(tablet_id_, &leader));
+    // The reason for the outside ASSERT_EVENTUALLY is that the leader might
+    // have changed in between of these two calls.
+    ASSERT_OK(GetConsensusState(leader, tablet_id_, kTimeout, &cstate));
+  });
+  // The original voter replica fallen behind WAL catchup threshold should be evicted.
+  EXPECT_FALSE(IsRaftConfigMember(follower_uuid, cstate.committed_config()))
+      << pb_util::SecureDebugString(cstate.committed_config())
+      << "fallen behind WAL replica UUID: " << follower_uuid;
+  // The first non-voter replica failed on tablet copy should be evicted.
+  EXPECT_FALSE(IsRaftConfigMember(ts0->uuid(), cstate.committed_config()))
+      << pb_util::SecureDebugString(cstate.committed_config())
+      << "failed tablet copy replica UUID: " << ts0->uuid();
+  // The tablet copy on the restarted server should succeed and this replica
+  // should replace the original failed replica.
+  EXPECT_TRUE(IsRaftConfigMember(ts1->uuid(), cstate.committed_config()))
+      << pb_util::SecureDebugString(cstate.committed_config())
+      << "new replacement replica UUID: " << ts1->uuid();
+}
+
 }  // namespace tserver
 }  // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 2d05290..a7e123c 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -3151,6 +3151,10 @@ bool AsyncAddReplicaTask::SendRequest(int attempt) {
   req.set_cas_config_opid_index(cstate_.committed_config().opid_index());
   RaftPeerPB* peer = req.mutable_server();
   peer->set_permanent_uuid(replacement_replica->permanent_uuid());
+  if (FLAGS_raft_prepare_replacement_before_eviction &&
+      member_type_ == RaftPeerPB::NON_VOTER) {
+    peer->mutable_attrs()->set_promote(true);
+  }
   ServerRegistrationPB peer_reg;
   replacement_replica->GetRegistration(&peer_reg);
   CHECK_GT(peer_reg.rpc_addresses_size(), 0);

http://git-wip-us.apache.org/repos/asf/kudu/blob/aafed6d1/src/kudu/tserver/tablet_copy_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service.cc b/src/kudu/tserver/tablet_copy_service.cc
index 2d10eb7..4467a6b 100644
--- a/src/kudu/tserver/tablet_copy_service.cc
+++ b/src/kudu/tserver/tablet_copy_service.cc
@@ -76,6 +76,7 @@ TAG_FLAG(fault_crash_on_handle_tc_fetch_data, unsafe);
 DEFINE_double(tablet_copy_early_session_timeout_prob, 0,
               "The probability that a tablet copy session will time out early, "
               "resulting in tablet copy failure. (For testing only!)");
+TAG_FLAG(tablet_copy_early_session_timeout_prob, runtime);
 TAG_FLAG(tablet_copy_early_session_timeout_prob, unsafe);
 
 using std::string;
@@ -206,11 +207,11 @@ void TabletCopyServiceImpl::BeginTabletCopySession(
 
   // For testing: Close the session prematurely if unsafe gflag is set but
   // still respond as if it was opened.
-  if (PREDICT_FALSE(FLAGS_tablet_copy_early_session_timeout_prob > 0 &&
-      rand_.NextDoubleFraction() <= FLAGS_tablet_copy_early_session_timeout_prob)) {
+  const auto timeout_prob = FLAGS_tablet_copy_early_session_timeout_prob;
+  if (PREDICT_FALSE(timeout_prob > 0 && rand_.NextDoubleFraction() <= timeout_prob)) {
     LOG_WITH_PREFIX(WARNING) << "Timing out tablet copy session due to flag "
                              << "--tablet_copy_early_session_timeout_prob "
-                             << "being set to " << FLAGS_tablet_copy_early_session_timeout_prob;
+                             << "being set to " << timeout_prob;
     MutexLock l(sessions_lock_);
     TabletCopyErrorPB::Code app_error;
     WARN_NOT_OK(TabletCopyServiceImpl::DoEndTabletCopySessionUnlocked(session_id, &app_error),