You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/06/06 06:18:27 UTC

kudu git commit: KUDU-2020: tserver failure causes multiple tablet copy operations per under-replicated tablet

Repository: kudu
Updated Branches:
  refs/heads/branch-1.3.x 713703ea1 -> 0ddea5d46


KUDU-2020: tserver failure causes multiple tablet copy operations per under-replicated tablet

The 'active ingredient' in this patch is the change to
TsTabletManager::StartTabletCopy that causes an ALREADY_INPROGRESS
response to be returned if the tablet is currently being copied and the
tablet copy thread pool is full. Previously an ALREADY_INPROGRESS
response would only occur if the tablet was currently being copied, and
the threadpool was not full.

The effect of the failure to return ALREADY_INPROGRESS was that a leader
would be much more likely consider a tablet server failed and to
subsequently drop the replica from the Raft config. As a result, on a
highly loaded cluster, a tablet copy could be started at the same time,
300 seconds apart, on many tablet servers.

The remaining changes are to return more specific errors out of the
tablet copy service, which aids with testing specific codepaths. One of
the existing tablet_copy-itest cases has been beefed up to cover the
tablet copy threadpool full path. Without the changes outlined before it
fails with:

../../src/kudu/integration-tests/tablet_copy-itest.cc:961: Failure
Expected: (num_inprogress) > (0), actual: 0 vs 0

which is exactly what we would expect; the tablet server is failing to
return INPROGRESS errors.

Anecdotally, this patch has improved TTR times 5-10x on highly loaded
clusters. It's still possible for tablets to be bounced around during
re-replication if the copying tablet server has a full RPC queue, or
it's unable to start the tablet copy for 300 seconds, but both of these
conditions indicate that it's probably best to drop that tserver and
retry on a (hopefully) less stressed server.

Change-Id: Iffa1f0fec4e882beabfee6e0f2672096caccdf75
Reviewed-on: http://gerrit.cloudera.org:8080/6925
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>
Reviewed-on: http://gerrit.cloudera.org:8080/7057


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

Branch: refs/heads/branch-1.3.x
Commit: 0ddea5d46627c972bcd37da0c8c07b6d27d49059
Parents: 713703e
Author: Dan Burkert <da...@apache.org>
Authored: Thu May 18 19:04:43 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Jun 6 06:17:24 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus_peers.cc           | 45 ++++++------
 src/kudu/consensus/consensus_queue.cc           |  6 +-
 src/kudu/consensus/consensus_queue.h            | 11 +--
 .../integration-tests/cluster_itest_util.cc     | 11 ++-
 src/kudu/integration-tests/cluster_itest_util.h | 13 ++--
 src/kudu/integration-tests/tablet_copy-itest.cc | 76 ++++++++++++++------
 src/kudu/tserver/tablet_service.cc              | 14 ++--
 src/kudu/tserver/ts_tablet_manager.cc           | 46 ++++++++++--
 8 files changed, 155 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index d2acdb7..62e8d84 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -257,7 +257,7 @@ void Peer::ProcessResponse() {
       // However, a RemoteError wraps some other error propagated from the
       // remote peer, so we know the remote is alive. Therefore, we will let
       // the queue know that the remote is responsive.
-      queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
+      queue_->NotifyPeerIsResponsive(peer_pb_.permanent_uuid());
     }
     ProcessResponseError(controller_.status());
     return;
@@ -271,7 +271,7 @@ void Peer::ProcessResponse() {
           response_.status().error().code() == consensus::ConsensusErrorPB::CANNOT_PREPARE)) {
     // Again, let the queue know that the remote is still responsive, since we
     // will not be sending this error response through to the queue.
-    queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
+    queue_->NotifyPeerIsResponsive(peer_pb_.permanent_uuid());
     ProcessResponseError(StatusFromPB(response_.error().status()));
     return;
   }
@@ -325,24 +325,27 @@ Status Peer::PrepareTabletCopyRequest() {
 
 void Peer::ProcessTabletCopyResponse() {
   // If the peer is already closed return.
-  {
-    std::unique_lock<simple_spinlock> lock(peer_lock_);
-    if (closed_) {
-      return;
-    }
-    CHECK(request_pending_);
-    request_pending_ = false;
+  std::unique_lock<simple_spinlock> lock(peer_lock_);
+  if (closed_) {
+    return;
   }
+  CHECK(request_pending_);
+  request_pending_ = false;
 
-  if (controller_.status().ok() && tc_response_.has_error()) {
-    // ALREADY_INPROGRESS is expected, so we do not log this error.
-    if (tc_response_.error().code() ==
-        TabletServerErrorPB::TabletServerErrorPB::ALREADY_INPROGRESS) {
-      queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
-    } else {
-      LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin Tablet Copy on peer: "
-                                        << SecureShortDebugString(tc_response_);
-    }
+  // If the response is OK, or ALREADY_INPROGRESS, then consider the RPC successful.
+  bool success =
+    controller_.status().ok() &&
+    (!tc_response_.has_error() ||
+     tc_response_.error().code() == TabletServerErrorPB::TabletServerErrorPB::ALREADY_INPROGRESS);
+
+  if (success) {
+    lock.unlock();
+    queue_->NotifyPeerIsResponsive(peer_pb_.permanent_uuid());
+  } else {
+    LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin Tablet Copy on peer: "
+                                      << (controller_.status().ok() ?
+                                          SecureShortDebugString(tc_response_) :
+                                          controller_.status().ToString());
   }
 }
 
@@ -412,9 +415,9 @@ void RpcPeerProxy::RequestConsensusVoteAsync(const VoteRequestPB* request,
 }
 
 void RpcPeerProxy::StartTabletCopy(const StartTabletCopyRequestPB* request,
-                                        StartTabletCopyResponsePB* response,
-                                        rpc::RpcController* controller,
-                                        const rpc::ResponseCallback& callback) {
+                                   StartTabletCopyResponsePB* response,
+                                   rpc::RpcController* controller,
+                                   const rpc::ResponseCallback& callback) {
   consensus_proxy_->StartTabletCopyAsync(*request, response, controller, callback);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/consensus/consensus_queue.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue.cc b/src/kudu/consensus/consensus_queue.cc
index bf746f3..997697a 100644
--- a/src/kudu/consensus/consensus_queue.cc
+++ b/src/kudu/consensus/consensus_queue.cc
@@ -376,10 +376,8 @@ Status PeerMessageQueue::RequestForPeer(const string& uuid,
       NotifyObserversOfFailedFollower(uuid, queue_state_.current_term, msg);
     }
   }
-
   if (PREDICT_FALSE(peer->needs_tablet_copy)) {
-    KLOG_EVERY_N_SECS_THROTTLER(INFO, 3, peer->status_log_throttler, "tablet copy")
-        << LogPrefixUnlocked() << "Peer " << uuid << " needs tablet copy" << THROTTLE_MSG;
+    VLOG(3) << LogPrefixUnlocked() << "Peer " << uuid << " needs tablet copy" << THROTTLE_MSG;
     *needs_tablet_copy = true;
     return Status::OK();
   }
@@ -582,7 +580,7 @@ void PeerMessageQueue::UpdateFollowerWatermarks(int64_t committed_index,
   UpdateMetrics();
 }
 
-void PeerMessageQueue::NotifyPeerIsResponsiveDespiteError(const std::string& peer_uuid) {
+void PeerMessageQueue::NotifyPeerIsResponsive(const std::string& peer_uuid) {
   std::lock_guard<simple_spinlock> l(queue_lock_);
   TrackedPeer* peer = FindPtrOrNull(peers_map_, peer_uuid);
   if (!peer) return;

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/consensus/consensus_queue.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue.h b/src/kudu/consensus/consensus_queue.h
index 3d25553..bd7ef53 100644
--- a/src/kudu/consensus/consensus_queue.h
+++ b/src/kudu/consensus/consensus_queue.h
@@ -222,11 +222,12 @@ class PeerMessageQueue {
   Status GetTabletCopyRequestForPeer(const std::string& uuid,
                                      StartTabletCopyRequestPB* req);
 
-  // Update the last successful communication timestamp for the given peer
-  // to the current time. This should be called when a non-network related
-  // error is received from the peer, indicating that it is alive, even if it
-  // may not be fully up and running or able to accept updates.
-  void NotifyPeerIsResponsiveDespiteError(const std::string& peer_uuid);
+  // Update the last successful communication timestamp for the given peer to
+  // the current time.
+  //
+  // This should be called when the peer responds with a message indicating that
+  // it is alive and making progress.
+  void NotifyPeerIsResponsive(const std::string& peer_uuid);
 
   // Updates the request queue with the latest response of a peer, returns
   // whether this peer has more requests pending.

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/integration-tests/cluster_itest_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.cc b/src/kudu/integration-tests/cluster_itest_util.cc
index df8e0af..cbf50f8 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -756,7 +756,7 @@ Status WaitUntilTabletRunning(TServerDetails* ts,
 
 Status DeleteTablet(const TServerDetails* ts,
                     const std::string& tablet_id,
-                    const TabletDataState delete_type,
+                    const TabletDataState& delete_type,
                     const boost::optional<int64_t>& cas_config_opid_index_less_or_equal,
                     const MonoDelta& timeout,
                     tserver::TabletServerErrorPB::Code* error_code) {
@@ -806,7 +806,8 @@ Status StartTabletCopy(const TServerDetails* ts,
                        const string& copy_source_uuid,
                        const HostPort& copy_source_addr,
                        int64_t caller_term,
-                       const MonoDelta& timeout) {
+                       const MonoDelta& timeout,
+                       tserver::TabletServerErrorPB::Code* error_code) {
   consensus::StartTabletCopyRequestPB req;
   consensus::StartTabletCopyResponsePB resp;
   RpcController rpc;
@@ -820,6 +821,12 @@ Status StartTabletCopy(const TServerDetails* ts,
 
   RETURN_NOT_OK(ts->consensus_proxy->StartTabletCopy(req, &resp, &rpc));
   if (resp.has_error()) {
+    CHECK(resp.error().has_code()) << "Tablet copy error response has no code";
+    CHECK(tserver::TabletServerErrorPB::Code_IsValid(resp.error().code()))
+        << "Tablet copy error response code is not valid";
+    if (error_code) {
+      *error_code = resp.error().code();
+    }
     return StatusFromPB(resp.error().status());
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/integration-tests/cluster_itest_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.h b/src/kudu/integration-tests/cluster_itest_util.h
index 2d73661..1cab7d2 100644
--- a/src/kudu/integration-tests/cluster_itest_util.h
+++ b/src/kudu/integration-tests/cluster_itest_util.h
@@ -197,7 +197,7 @@ Status StartElection(const TServerDetails* replica,
 Status LeaderStepDown(const TServerDetails* replica,
                       const std::string& tablet_id,
                       const MonoDelta& timeout,
-                      tserver::TabletServerErrorPB* error = NULL);
+                      tserver::TabletServerErrorPB* error = nullptr);
 
 // Write a "simple test schema" row to the specified tablet on the given
 // replica. This schema is commonly used by tests and is defined in
@@ -220,7 +220,7 @@ Status AddServer(const TServerDetails* leader,
                  consensus::RaftPeerPB::MemberType member_type,
                  const boost::optional<int64_t>& cas_config_opid_index,
                  const MonoDelta& timeout,
-                 tserver::TabletServerErrorPB::Code* error_code = NULL);
+                 tserver::TabletServerErrorPB::Code* error_code = nullptr);
 
 // Run a ConfigChange to REMOVE_SERVER on 'replica_to_remove'.
 // The RPC request is sent to 'leader'.
@@ -229,7 +229,7 @@ Status RemoveServer(const TServerDetails* leader,
                     const TServerDetails* replica_to_remove,
                     const boost::optional<int64_t>& cas_config_opid_index,
                     const MonoDelta& timeout,
-                    tserver::TabletServerErrorPB::Code* error_code = NULL);
+                    tserver::TabletServerErrorPB::Code* error_code = nullptr);
 
 // Get the list of tablets from the remote server.
 Status ListTablets(const TServerDetails* ts,
@@ -294,10 +294,10 @@ Status WaitUntilTabletRunning(TServerDetails* ts,
 // Send a DeleteTablet() to the server at 'ts' of the specified 'delete_type'.
 Status DeleteTablet(const TServerDetails* ts,
                     const std::string& tablet_id,
-                    const tablet::TabletDataState delete_type,
+                    const tablet::TabletDataState& delete_type,
                     const boost::optional<int64_t>& cas_config_opid_index_less_or_equal,
                     const MonoDelta& timeout,
-                    tserver::TabletServerErrorPB::Code* error_code = NULL);
+                    tserver::TabletServerErrorPB::Code* error_code = nullptr);
 
 // Repeatedly try to delete the tablet, retrying on failure up to the
 // specified timeout. Deletion can fail when other operations, such as
@@ -315,7 +315,8 @@ Status StartTabletCopy(const TServerDetails* ts,
                        const std::string& copy_source_uuid,
                        const HostPort& copy_source_addr,
                        int64_t caller_term,
-                       const MonoDelta& timeout);
+                       const MonoDelta& timeout,
+                       tserver::TabletServerErrorPB::Code* error_code = nullptr);
 
 } // namespace itest
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/integration-tests/tablet_copy-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tablet_copy-itest.cc b/src/kudu/integration-tests/tablet_copy-itest.cc
index 41980db..a5d3dc4 100644
--- a/src/kudu/integration-tests/tablet_copy-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy-itest.cc
@@ -16,13 +16,15 @@
 // under the License.
 
 #include <atomic>
-#include <boost/optional.hpp>
-#include <gflags/gflags.h>
+#include <set>
 #include <string>
 #include <thread>
 #include <unordered_map>
 #include <vector>
 
+#include <boost/optional.hpp>
+#include <gflags/gflags.h>
+
 #include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
 #include "kudu/common/wire_protocol-test-util.h"
@@ -59,6 +61,7 @@ using kudu::tablet::TABLET_DATA_DELETED;
 using kudu::tablet::TABLET_DATA_TOMBSTONED;
 using kudu::tserver::ListTabletsResponsePB;
 using kudu::tserver::TabletCopyClient;
+using std::set;
 using std::string;
 using std::unordered_map;
 using std::vector;
@@ -880,8 +883,15 @@ TEST_F(TabletCopyITest, TestTabletCopyThrottling) {
   TServerDetails* ts0 = ts_map_[cluster_->tablet_server(0)->uuid()];
   TServerDetails* ts1 = ts_map_[cluster_->tablet_server(1)->uuid()];
 
-  vector<ListTabletsResponsePB::StatusAndSchemaPB> tablets;
-  ASSERT_OK(WaitForNumTabletsOnTS(ts0, kNumTablets, kTimeout, &tablets));
+  vector<ListTabletsResponsePB::StatusAndSchemaPB> tablet_states;
+  ASSERT_OK(WaitForNumTabletsOnTS(ts0, kNumTablets, kTimeout, &tablet_states));
+
+  // Gather the set of tablet IDs.
+  set<string> tablets;
+  for (const auto& state : tablet_states) {
+    tablets.insert(state.tablet_status().tablet_id());
+  }
+  ASSERT_EQ(4, tablets.size());
 
   workload.StopAndJoin();
 
@@ -891,28 +901,54 @@ TEST_F(TabletCopyITest, TestTabletCopyThrottling) {
   // ServiceUnavailable error.
   ASSERT_OK(cluster_->tablet_server(1)->Restart());
 
-  // Attempt to copy all of the tablets from TS0 to TS1.
-  // Collect the status messages.
-  vector<Status> statuses(kNumTablets);
-  for (const auto& t : tablets) {
-    HostPort src_addr;
-    ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &src_addr));
-    statuses.push_back(StartTabletCopy(ts1, t.tablet_status().tablet_id(), ts0->uuid(),
-                                       src_addr, 0, kTimeout));
-  }
+  HostPort ts0_hostport;
+  ASSERT_OK(HostPortFromPB(ts0->registration.rpc_addresses(0), &ts0_hostport));
 
-  // The "Service unavailable" messages are serialized as RemoteError type.
-  // Ensure that we got at least one.
+  // Attempt to copy all of the tablets from TS0 to TS1 in parallel. Tablet
+  // copies are repeated periodically until complete.
   int num_service_unavailable = 0;
-  for (const Status& s : statuses) {
-    if (!s.ok()) {
-      ASSERT_TRUE(s.IsRemoteError()) << s.ToString();
-      ASSERT_STR_CONTAINS(s.ToString(), "Service unavailable: Thread pool is at capacity");
-      num_service_unavailable++;
+  int num_inprogress = 0;
+  while (!tablets.empty()) {
+    for (auto tablet_id = tablets.begin(); tablet_id != tablets.end(); ) {
+      tserver::TabletServerErrorPB::Code error_code = tserver::TabletServerErrorPB::UNKNOWN_ERROR;
+      Status s = StartTabletCopy(ts1, *tablet_id, ts0->uuid(),
+                                 ts0_hostport, 0, kTimeout, &error_code);
+      if (!s.ok()) {
+        switch (error_code) {
+          case tserver::TabletServerErrorPB::THROTTLED:
+            // The tablet copy threadpool is full.
+            ASSERT_TRUE(s.IsServiceUnavailable()) << s.ToString();
+            ++num_service_unavailable;
+            break;
+          case tserver::TabletServerErrorPB::ALREADY_INPROGRESS:
+            // The tablet is already being copied
+            ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+            ++num_inprogress;
+            break;
+          case tserver::TabletServerErrorPB::INVALID_CONFIG:
+            // The tablet copy has already completed, and the remote tablet now
+            // has the correct term (not 0).
+            ASSERT_STR_MATCHES(s.ToString(),
+                               "Leader has replica of tablet .* with term 0, "
+                               "which is lower than last-logged term .* on local replica. "
+                               "Rejecting tablet copy request");
+            tablet_id = tablets.erase(tablet_id);
+            continue;
+          default:
+            FAIL() << "Unexpected tablet copy failure: " << s.ToString()
+                  << ": " << tserver::TabletServerErrorPB::Code_Name(error_code);
+        }
+      }
+      ++tablet_id;
     }
+    SleepFor(MonoDelta::FromMilliseconds(1));
   }
+
+  // Ensure that we get at least one service unavailable and copy in progress responses.
   ASSERT_GT(num_service_unavailable, 0);
+  ASSERT_GT(num_inprogress, 0);
   LOG(INFO) << "Number of Service unavailable responses: " << num_service_unavailable;
+  LOG(INFO) << "Number of in progress responses: " << num_inprogress;
 }
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index be7919b..3835486 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -1025,12 +1025,16 @@ void ConsensusServiceImpl::StartTabletCopy(const StartTabletCopyRequestPB* req,
   if (!CheckUuidMatchOrRespond(tablet_manager_, "StartTabletCopy", req, resp, context)) {
     return;
   }
-  auto response_callback = [context, resp](const Status& s, TabletServerErrorPB::Code error_code) {
-    if (!s.ok()) {
-      SetupErrorAndRespond(resp->mutable_error(), s, error_code, context);
-      return;
+  auto response_callback = [context, resp](const Status& s, TabletServerErrorPB::Code code) {
+    if (s.ok()) {
+      context->RespondSuccess();
+    } else {
+      // Skip calling SetupErrorAndRespond since this path doesn't need the
+      // error to be transformed.
+      StatusToPB(s, resp->mutable_error()->mutable_status());
+      resp->mutable_error()->set_code(code);
+      context->RespondNoCache();
     }
-    context->RespondSuccess();
   };
   tablet_manager_->StartTabletCopy(req, response_callback);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ddea5d4/src/kudu/tserver/ts_tablet_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index 8214655..3aab42c 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -386,17 +386,42 @@ class TabletCopyRunnable : public Runnable {
 void TSTabletManager::StartTabletCopy(
     const StartTabletCopyRequestPB* req,
     std::function<void(const Status&, TabletServerErrorPB::Code)> cb) {
+  // Attempt to submit the tablet copy task to the threadpool. The threadpool
+  // is configured with 0 queue slots, so if there is not a thread immediately
+  // available the submit will fail. When successful, the table copy task will
+  // immediately check whether the tablet is already being copied, and if so,
+  // return ALREADY_INPROGRESS.
+  string tablet_id = req->tablet_id();
   shared_ptr<TabletCopyRunnable> runnable(new TabletCopyRunnable(this, req, cb));
   Status s = tablet_copy_pool_->Submit(runnable);
   if (PREDICT_TRUE(s.ok())) {
     return;
   }
 
-  // We were unable to submit the TabletCopyRunnable to the ThreadPool. We will
+  // We were unable to submit the tablet copy task to the thread pool. We will
   // invoke the callback ourselves, so disable the automatic callback mechanism.
   runnable->DisableCallback();
 
-  // Thread pool is at capacity.
+  // Check if the tablet is already in transition (i.e. being copied).
+  boost::optional<string> transition;
+  {
+    // Lock must be dropped before executing callbacks.
+    shared_lock<rw_spinlock> lock(lock_);
+    auto* t = FindOrNull(transition_in_progress_, tablet_id);
+    if (t) {
+      transition = *t;
+    }
+  }
+  if (transition) {
+    cb(Status::IllegalState(
+          strings::Substitute("State transition of tablet $0 already in progress: $1",
+                              tablet_id, *transition)),
+          TabletServerErrorPB::ALREADY_INPROGRESS);
+    return;
+  }
+
+  // The tablet is not already being copied, but there are no remaining slots in
+  // the threadpool.
   if (s.IsServiceUnavailable()) {
     cb(s, TabletServerErrorPB::THROTTLED);
     return;
@@ -418,6 +443,15 @@ void TSTabletManager::StartTabletCopy(
     } \
   } while (0)
 
+#define CALLBACK_RETURN_NOT_OK_WITH_ERROR(expr, error) \
+  do { \
+    Status _s = (expr); \
+    if (PREDICT_FALSE(!_s.ok())) { \
+      error_code = (error); \
+      CALLBACK_AND_RETURN(_s); \
+    } \
+  } while (0)
+
 void TSTabletManager::RunTabletCopy(
     const StartTabletCopyRequestPB* req,
     std::function<void(const Status&, TabletServerErrorPB::Code)> cb) {
@@ -458,7 +492,9 @@ void TSTabletManager::RunTabletCopy(
                    << "Found tablet in TABLET_DATA_COPYING state during StartTabletCopy()";
       case TABLET_DATA_TOMBSTONED: {
         int64_t last_logged_term = meta->tombstone_last_logged_opid().term();
-        CALLBACK_RETURN_NOT_OK(CheckLeaderTermNotLower(tablet_id, leader_term, last_logged_term));
+        CALLBACK_RETURN_NOT_OK_WITH_ERROR(
+            CheckLeaderTermNotLower(tablet_id, leader_term, last_logged_term),
+            TabletServerErrorPB::INVALID_CONFIG);
         break;
       }
       case TABLET_DATA_READY: {
@@ -470,7 +506,9 @@ void TSTabletManager::RunTabletCopy(
         OpId last_logged_opid;
         log->GetLatestEntryOpId(&last_logged_opid);
         int64_t last_logged_term = last_logged_opid.term();
-        CALLBACK_RETURN_NOT_OK(CheckLeaderTermNotLower(tablet_id, leader_term, last_logged_term));
+        CALLBACK_RETURN_NOT_OK_WITH_ERROR(
+            CheckLeaderTermNotLower(tablet_id, leader_term, last_logged_term),
+            TabletServerErrorPB::INVALID_CONFIG);
 
         // Tombstone the tablet and store the last-logged OpId.
         old_tablet_peer->Shutdown();