You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by jd...@apache.org on 2016/02/06 19:42:08 UTC

[1/3] incubator-kudu git commit: KUDU-1278 - Add a way to inject latency into remote bootstrap client and a failing test

Repository: incubator-kudu
Updated Branches:
  refs/heads/master c68509b4d -> 802d0e4c5


KUDU-1278 - Add a way to inject latency into remote bootstrap client and a failing test

This adds a way to inject latency into the remote bootstrapping client and a
failing test which reproduces the bug documented in KUDU-1278.

Change-Id: I5921a0fea65e47fd906ee64b5979dfacf652c97d
Reviewed-on: http://gerrit.cloudera.org:8080/2079
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 4a9e2ca0713741bbb7edc8925dadb8b36d8d5560
Parents: c68509b
Author: David Alves <da...@cloudera.com>
Authored: Fri Feb 5 14:41:15 2016 -0800
Committer: David Ribeiro Alves <da...@cloudera.com>
Committed: Fri Feb 5 23:32:44 2016 +0000

----------------------------------------------------------------------
 .../integration-tests/remote_bootstrap-itest.cc | 60 ++++++++++++++++++++
 src/kudu/tserver/remote_bootstrap_client.cc     | 11 ++++
 2 files changed, 71 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4a9e2ca0/src/kudu/integration-tests/remote_bootstrap-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/remote_bootstrap-itest.cc b/src/kudu/integration-tests/remote_bootstrap-itest.cc
index 914e140..97df129 100644
--- a/src/kudu/integration-tests/remote_bootstrap-itest.cc
+++ b/src/kudu/integration-tests/remote_bootstrap-itest.cc
@@ -683,4 +683,64 @@ TEST_F(RemoteBootstrapITest, TestDisableRemoteBootstrap_NoTightLoopWhenTabletDel
   EXPECT_LT(num_logs_per_second, 20);
 }
 
+// Test that if a remote bootstrap is taking a long time but the client peer is still responsive,
+// the leader won't mark it as failed.
+TEST_F(RemoteBootstrapITest, DISABLED_TestSlowBootstrapDoesntFail) {
+  MonoDelta timeout = MonoDelta::FromSeconds(10);
+  vector<string> ts_flags, master_flags;
+  ts_flags.push_back("--enable_leader_failure_detection=false");
+  ts_flags.push_back("--remote_bootstrap_dowload_file_inject_latency_ms=5000");
+  ts_flags.push_back("--follower_unavailable_considered_failed_sec=2");
+  master_flags.push_back("--catalog_manager_wait_for_new_tablets_to_elect_leader=false");
+  NO_FATALS(StartCluster(ts_flags, master_flags));
+
+  TestWorkload workload(cluster_.get());
+  // TODO(KUDU-1322): the client should handle retrying on different replicas
+  // if the tablet isn't found, rather than giving us this error.
+  workload.set_not_found_allowed(true);
+  workload.set_write_batch_size(1);
+  workload.Setup();
+
+  // Figure out the tablet id of the created tablet.
+  vector<ListTabletsResponsePB::StatusAndSchemaPB> tablets;
+  ExternalTabletServer* replica_ets = cluster_->tablet_server(1);
+  TServerDetails* replica_ts = ts_map_[replica_ets->uuid()];
+  ASSERT_OK(WaitForNumTabletsOnTS(replica_ts, 1, timeout, &tablets));
+  string tablet_id = tablets[0].tablet_status().tablet_id();
+
+  // Wait until all replicas are up and running.
+  for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
+    ASSERT_OK(itest::WaitUntilTabletRunning(ts_map_[cluster_->tablet_server(i)->uuid()],
+                                            tablet_id, timeout));
+  }
+
+  // Elect a leader (TS 0)
+  ExternalTabletServer* leader_ts = cluster_->tablet_server(0);
+  ASSERT_OK(itest::StartElection(ts_map_[leader_ts->uuid()], tablet_id, timeout));
+
+  // Start writing, wait for some rows to be inserted.
+  workload.Start();
+  while (workload.rows_inserted() < 100) {
+    SleepFor(MonoDelta::FromMilliseconds(10));
+  }
+
+
+  // Tombstone the follower.
+  LOG(INFO) << "Tombstoning follower tablet " << tablet_id << " on TS " << replica_ts->uuid();
+  ASSERT_OK(itest::DeleteTablet(replica_ts, tablet_id, TABLET_DATA_TOMBSTONED, boost::none,
+                                timeout));
+
+  // Wait for remote bootstrap to start.
+  ASSERT_OK(inspect_->WaitForTabletDataStateOnTS(1, tablet_id,
+                                                 tablet::TABLET_DATA_COPYING, timeout));
+
+  workload.StopAndJoin();
+  ASSERT_OK(WaitForServersToAgree(timeout, ts_map_, tablet_id, 1));
+
+  ClusterVerifier v(cluster_.get());
+  NO_FATALS(v.CheckCluster());
+  NO_FATALS(v.CheckRowCount(workload.table_name(), ClusterVerifier::AT_LEAST,
+                            workload.rows_inserted()));
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4a9e2ca0/src/kudu/tserver/remote_bootstrap_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_client.cc b/src/kudu/tserver/remote_bootstrap_client.cc
index bac717e..4e00a06 100644
--- a/src/kudu/tserver/remote_bootstrap_client.cc
+++ b/src/kudu/tserver/remote_bootstrap_client.cc
@@ -56,6 +56,11 @@ TAG_FLAG(remote_bootstrap_save_downloaded_metadata, advanced);
 TAG_FLAG(remote_bootstrap_save_downloaded_metadata, hidden);
 TAG_FLAG(remote_bootstrap_save_downloaded_metadata, runtime);
 
+DEFINE_int32(remote_bootstrap_dowload_file_inject_latency_ms, 0,
+             "Injects latency into the loop that downloads files, causing remote bootstrap "
+             "to take much longer. For use in tests only.");
+TAG_FLAG(remote_bootstrap_dowload_file_inject_latency_ms, hidden);
+
 // RETURN_NOT_OK_PREPEND() with a remote-error unwinding step.
 #define RETURN_NOT_OK_UNWIND_PREPEND(status, controller, msg) \
   RETURN_NOT_OK_PREPEND(UnwindRemoteError(status, controller), msg)
@@ -518,6 +523,12 @@ Status RemoteBootstrapClient::DownloadFile(const DataIdPB& data_id,
     // Write the data.
     RETURN_NOT_OK(appendable->Append(resp.chunk().data()));
 
+    if (PREDICT_FALSE(FLAGS_remote_bootstrap_dowload_file_inject_latency_ms > 0)) {
+      LOG_WITH_PREFIX(INFO) << "Injecting latency into file download: " <<
+          FLAGS_remote_bootstrap_dowload_file_inject_latency_ms;
+      SleepFor(MonoDelta::FromMilliseconds(FLAGS_remote_bootstrap_dowload_file_inject_latency_ms));
+    }
+
     if (offset + resp.chunk().data().size() == resp.chunk().total_data_length()) {
       done = true;
     }


[3/3] incubator-kudu git commit: KUDU-1324. Fix SEGV in catalog manager handling under-replicated tablet

Posted by jd...@apache.org.
KUDU-1324. Fix SEGV in catalog manager handling under-replicated tablet

Commit 31278211f1934890e6835c9db164a7dea87d826a introduced some
new logging when starting the 'AsyncAddServer' task in the catalog
manager that tries to send the AddServer RPC for an under-replicated
tablet. However, this can SEGV in the case that the tablet does
not currently have an elected leader.

This crash can be triggered when restarting the master while a tablet
is under-replicated. When it comes back up, the master may
receive the report of the under-replicated tablet. When it tries
to run the AsyncAddServer task, there is no known leader yet (e.g.
because the leader has not yet sent its tablet report), and thus
the task fails immediately and deletes itself. Calling task->description()
then accesses the freed memory and crashes.

An earlier version of this fix tried to fix the issue by keeping a
scoped_refptr to the task. However, this isn't sufficient because
task->description() will crash if there is no known target tablet
server.

In order to fix this regression for the 0.7.0 release, this patch
takes the simplest approach of just changing the log message to
include less detail. A regression test will be included in a later
patch.

Change-Id: I62037fbaa910a1da476a0ac2075afdcdbc460dc8
Reviewed-on: http://gerrit.cloudera.org:8080/2060
Reviewed-by: Jean-Daniel Cryans
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 802d0e4c53f12b4392544ee10dfb530a25812d4f
Parents: b841512
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Feb 4 18:19:12 2016 -0800
Committer: Jean-Daniel Cryans <jd...@gerrit.cloudera.org>
Committed: Sat Feb 6 18:40:14 2016 +0000

----------------------------------------------------------------------
 src/kudu/master/catalog_manager.cc | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/802d0e4c/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 000a6a5..b3c0468 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2476,9 +2476,9 @@ void CatalogManager::SendAddServerRequest(const scoped_refptr<TabletInfo>& table
   tablet->table()->AddTask(task);
   WARN_NOT_OK(task->Run(), "Failed to send new AddServer request");
 
-  // Need to print this after Run() because that's where it picks the TS which description()
-  // needs.
-  LOG(INFO) << "Started AddServer task: " << task->description();
+  // We can't access 'task' here because it may delete itself inside Run() in the
+  // case that the tablet has no known leader.
+  LOG(INFO) << "Started AddServer task for tablet " << tablet->tablet_id();
 }
 
 void CatalogManager::ExtractTabletsToProcess(


[2/3] incubator-kudu git commit: KUDU-1278 - Mark peer responsive when remote bootstrap is in progress

Posted by jd...@apache.org.
KUDU-1278 - Mark peer responsive when remote bootstrap is in progress

This commit fixes a bug in remote bootstrap where, when a peer starts to remote bootstrap
its last_successful_communication_time is not updated, and so if the remote bootstrap lasts
more than peer timeout time, this peer will be evicted and leader will try to remote bootstrap
on a new peer, which will likely timeout again.
Also during remote bootstrap lots of logs are printed, so some log levels are changed from
INFO to VLOG1.

Change-Id: Ic41f2ad31c437f6ab976987262d8b35f9aa9f3ad
Reviewed-on: http://gerrit.cloudera.org:8080/1759
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: b8415129204d26508b5e45b57dce020280bdbc69
Parents: 4a9e2ca
Author: Binglin Chang <de...@gmail.com>
Authored: Sat Jan 9 22:42:37 2016 +0800
Committer: David Ribeiro Alves <da...@cloudera.com>
Committed: Fri Feb 5 23:32:54 2016 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus_peers.cc                | 13 +++++++++----
 src/kudu/consensus/consensus_queue.cc                |  4 ++--
 src/kudu/integration-tests/remote_bootstrap-itest.cc |  2 +-
 src/kudu/master/catalog_manager.cc                   |  4 +++-
 src/kudu/master/catalog_manager.h                    |  4 +++-
 src/kudu/tserver/remote_bootstrap_session.cc         |  5 ++++-
 src/kudu/tserver/tablet_peer_lookup.h                |  5 ++++-
 src/kudu/tserver/tablet_service.cc                   |  5 +++--
 src/kudu/tserver/ts_tablet_manager.cc                | 12 +++++++++---
 src/kudu/tserver/ts_tablet_manager.h                 |  4 +++-
 src/kudu/tserver/tserver.proto                       |  3 +++
 11 files changed, 44 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index c27ae83..37e8cb8 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -301,10 +301,15 @@ Status Peer::SendRemoteBootstrapRequest() {
 }
 
 void Peer::ProcessRemoteBootstrapResponse() {
-  // We treat remote bootstrap as fire-and-forget.
-  if (rb_response_.has_error()) {
-    LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin remote bootstrap on peer: "
-                                      << rb_response_.ShortDebugString();
+  if (controller_.status().ok() && rb_response_.has_error()) {
+    // ALREADY_INPROGRESS is expected, so we do not log this error.
+    if (rb_response_.error().code() ==
+        kudu::tserver::TabletServerErrorPB::TabletServerErrorPB::ALREADY_INPROGRESS) {
+      queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
+    } else {
+      LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin remote bootstrap on peer: "
+                                        << rb_response_.ShortDebugString();
+    }
   }
   sem_.Release();
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/consensus/consensus_queue.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue.cc b/src/kudu/consensus/consensus_queue.cc
index a36d7fe..040a5c2 100644
--- a/src/kudu/consensus/consensus_queue.cc
+++ b/src/kudu/consensus/consensus_queue.cc
@@ -313,7 +313,7 @@ Status PeerMessageQueue::RequestForPeer(const string& uuid,
   }
 
   if (PREDICT_FALSE(peer->needs_remote_bootstrap)) {
-    LOG_WITH_PREFIX_UNLOCKED(INFO) << "Peer needs remote bootstrap: " << peer->ToString();
+    VLOG_WITH_PREFIX_UNLOCKED(1) << "Peer needs remote bootstrap: " << peer->ToString();
     *needs_remote_bootstrap = true;
     return Status::OK();
   }
@@ -499,7 +499,7 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
           << response.ShortDebugString();
 
       peer->needs_remote_bootstrap = true;
-      LOG_WITH_PREFIX_UNLOCKED(INFO) << "Marked peer as needing remote bootstrap: "
+      VLOG_WITH_PREFIX_UNLOCKED(1) << "Marked peer as needing remote bootstrap: "
                                      << peer->ToString();
       *more_pending = true;
       return;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/integration-tests/remote_bootstrap-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/remote_bootstrap-itest.cc b/src/kudu/integration-tests/remote_bootstrap-itest.cc
index 97df129..305d2c8 100644
--- a/src/kudu/integration-tests/remote_bootstrap-itest.cc
+++ b/src/kudu/integration-tests/remote_bootstrap-itest.cc
@@ -685,7 +685,7 @@ TEST_F(RemoteBootstrapITest, TestDisableRemoteBootstrap_NoTightLoopWhenTabletDel
 
 // Test that if a remote bootstrap is taking a long time but the client peer is still responsive,
 // the leader won't mark it as failed.
-TEST_F(RemoteBootstrapITest, DISABLED_TestSlowBootstrapDoesntFail) {
+TEST_F(RemoteBootstrapITest, TestSlowBootstrapDoesntFail) {
   MonoDelta timeout = MonoDelta::FromSeconds(10);
   vector<string> ts_flags, master_flags;
   ts_flags.push_back("--enable_leader_failure_detection=false");

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 9a9344a..000a6a5 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1701,7 +1701,9 @@ const NodeInstancePB& CatalogManager::NodeInstance() const {
   return master_->instance_pb();
 }
 
-Status CatalogManager::StartRemoteBootstrap(const StartRemoteBootstrapRequestPB& req) {
+Status CatalogManager::StartRemoteBootstrap(
+    const StartRemoteBootstrapRequestPB& req,
+    boost::optional<kudu::tserver::TabletServerErrorPB::Code>* error_code) {
   return Status::NotSupported("Remote bootstrap not yet implemented for the master tablet");
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 2365361..b06cb5a 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -401,7 +401,9 @@ class CatalogManager : public tserver::TabletPeerLookupIf {
 
   bool IsInitialized() const;
 
-  virtual Status StartRemoteBootstrap(const consensus::StartRemoteBootstrapRequestPB& req) OVERRIDE;
+  virtual Status StartRemoteBootstrap(
+      const consensus::StartRemoteBootstrapRequestPB& req,
+      boost::optional<kudu::tserver::TabletServerErrorPB::Code>* error_code) OVERRIDE;
 
   // Return OK if this CatalogManager is a leader in a consensus configuration and if
   // the required leader state (metadata for tables and tablets) has

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/tserver/remote_bootstrap_session.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_session.cc b/src/kudu/tserver/remote_bootstrap_session.cc
index c0c6aa1..2d2aced 100644
--- a/src/kudu/tserver/remote_bootstrap_session.cc
+++ b/src/kudu/tserver/remote_bootstrap_session.cc
@@ -90,8 +90,11 @@ Status RemoteBootstrapSession::Init() {
   // All subsequent requests should reuse the opened blocks.
   vector<BlockIdPB> data_blocks;
   TabletMetadata::CollectBlockIdPBs(tablet_superblock_, &data_blocks);
+  LOG(INFO) << "T " << tablet_peer_->tablet_id()
+            << " P " << tablet_peer_->consensus()->peer_uuid()
+            << ": Remote bootstrap: Opening " << data_blocks.size() << " blocks";
   for (const BlockIdPB& block_id : data_blocks) {
-    LOG(INFO) << "Opening block " << block_id.DebugString();
+    VLOG(1) << "Opening block " << block_id.DebugString();
     RETURN_NOT_OK(OpenBlockUnlocked(BlockId::FromPB(block_id)));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/tserver/tablet_peer_lookup.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_peer_lookup.h b/src/kudu/tserver/tablet_peer_lookup.h
index ceb087c..c58b79d 100644
--- a/src/kudu/tserver/tablet_peer_lookup.h
+++ b/src/kudu/tserver/tablet_peer_lookup.h
@@ -17,10 +17,12 @@
 #ifndef KUDU_TSERVER_TABLET_PEER_LOOKUP_H_
 #define KUDU_TSERVER_TABLET_PEER_LOOKUP_H_
 
+#include <boost/optional/optional_fwd.hpp>
 #include <memory>
 #include <string>
 
 #include "kudu/gutil/ref_counted.h"
+#include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -49,7 +51,8 @@ class TabletPeerLookupIf {
 
   virtual const NodeInstancePB& NodeInstance() const = 0;
 
-  virtual Status StartRemoteBootstrap(const consensus::StartRemoteBootstrapRequestPB& req) = 0;
+  virtual Status StartRemoteBootstrap(const consensus::StartRemoteBootstrapRequestPB& req,
+                                      boost::optional<TabletServerErrorPB::Code>* error_code) = 0;
 };
 
 } // namespace tserver

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index 10d8f3b..8787c43 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -956,10 +956,11 @@ void ConsensusServiceImpl::StartRemoteBootstrap(const StartRemoteBootstrapReques
   if (!CheckUuidMatchOrRespond(tablet_manager_, "StartRemoteBootstrap", req, resp, context)) {
     return;
   }
-  Status s = tablet_manager_->StartRemoteBootstrap(*req);
+  boost::optional<TabletServerErrorPB::Code> error_code;
+  Status s = tablet_manager_->StartRemoteBootstrap(*req, &error_code);
   if (!s.ok()) {
     SetupErrorAndRespond(resp->mutable_error(), s,
-                         TabletServerErrorPB::UNKNOWN_ERROR,
+                         error_code.get_value_or(TabletServerErrorPB::UNKNOWN_ERROR),
                          context);
     return;
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/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 2e89e89..a26699e 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -337,7 +337,9 @@ Status TSTabletManager::CheckLeaderTermNotLower(const string& tablet_id,
   return Status::OK();
 }
 
-Status TSTabletManager::StartRemoteBootstrap(const StartRemoteBootstrapRequestPB& req) {
+Status TSTabletManager::StartRemoteBootstrap(
+    const StartRemoteBootstrapRequestPB& req,
+    boost::optional<TabletServerErrorPB::Code>* error_code) {
   const string& tablet_id = req.tablet_id();
   const string& bootstrap_peer_uuid = req.bootstrap_peer_uuid();
   HostPort bootstrap_peer_addr;
@@ -356,8 +358,12 @@ Status TSTabletManager::StartRemoteBootstrap(const StartRemoteBootstrapRequestPB
       meta = old_tablet_peer->tablet_metadata();
       replacing_tablet = true;
     }
-    RETURN_NOT_OK(StartTabletStateTransitionUnlocked(tablet_id, "remote bootstrapping tablet",
-                                                     &deleter));
+    Status ret = StartTabletStateTransitionUnlocked(tablet_id, "remote bootstrapping tablet",
+                                                    &deleter);
+    if (!ret.ok()) {
+      *error_code = TabletServerErrorPB::ALREADY_INPROGRESS;
+      return ret;
+    }
   }
 
   if (replacing_tablet) {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/tserver/ts_tablet_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.h b/src/kudu/tserver/ts_tablet_manager.h
index 2d63142..7fdb335 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -145,7 +145,9 @@ class TSTabletManager : public tserver::TabletPeerLookupIf {
   // See the StartRemoteBootstrap() RPC declaration in consensus.proto for details.
   // Currently this runs the entire procedure synchronously.
   // TODO: KUDU-921: Run this procedure on a background thread.
-  virtual Status StartRemoteBootstrap(const consensus::StartRemoteBootstrapRequestPB& req) OVERRIDE;
+  virtual Status StartRemoteBootstrap(
+      const consensus::StartRemoteBootstrapRequestPB& req,
+      boost::optional<TabletServerErrorPB::Code>* error_code) OVERRIDE;
 
   // Generate an incremental tablet report.
   //

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/b8415129/src/kudu/tserver/tserver.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver.proto b/src/kudu/tserver/tserver.proto
index 1762b22..5d7933e 100644
--- a/src/kudu/tserver/tserver.proto
+++ b/src/kudu/tserver/tserver.proto
@@ -85,6 +85,9 @@ message TabletServerErrorPB {
 
     // The compare-and-swap specified by an atomic RPC operation failed.
     CAS_FAILED = 17;
+
+    // The requested operation is already inprogress, e.g. RemoteBootstrap.
+    ALREADY_INPROGRESS = 18;
   }
 
   // The error code.