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 2016/08/05 21:58:52 UTC

[1/3] kudu git commit: Rename Remote Bootstrap to Tablet Copy (part 1)

Repository: kudu
Updated Branches:
  refs/heads/master c8e29959d -> a6cb942e8


http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 be4ae2e..61d79df 100644
--- a/src/kudu/tserver/remote_bootstrap_session.cc
+++ b/src/kudu/tserver/remote_bootstrap_session.cc
@@ -33,10 +33,10 @@
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/trace.h"
 
-DEFINE_int32(remote_bootstrap_transfer_chunk_size_bytes, 4 * 1024 * 1024,
+DEFINE_int32(tablet_copy_transfer_chunk_size_bytes, 4 * 1024 * 1024,
              "Size of chunks to transfer when copying tablets between "
              "tablet servers.");
-TAG_FLAG(remote_bootstrap_transfer_chunk_size_bytes, hidden);
+TAG_FLAG(tablet_copy_transfer_chunk_size_bytes, hidden);
 
 namespace kudu {
 namespace tserver {
@@ -55,7 +55,7 @@ using tablet::TabletMetadata;
 using tablet::TabletPeer;
 using tablet::TabletSuperBlockPB;
 
-RemoteBootstrapSession::RemoteBootstrapSession(
+TabletCopySession::TabletCopySession(
     const scoped_refptr<TabletPeer>& tablet_peer, std::string session_id,
     std::string requestor_uuid, FsManager* fs_manager)
     : tablet_peer_(tablet_peer),
@@ -65,19 +65,19 @@ RemoteBootstrapSession::RemoteBootstrapSession(
       blocks_deleter_(&blocks_),
       logs_deleter_(&logs_) {}
 
-RemoteBootstrapSession::~RemoteBootstrapSession() {
+TabletCopySession::~TabletCopySession() {
   // No lock taken in the destructor, should only be 1 thread with access now.
   CHECK_OK(UnregisterAnchorIfNeededUnlocked());
 }
 
-Status RemoteBootstrapSession::Init() {
+Status TabletCopySession::Init() {
   MutexLock l(session_lock_);
   CHECK(!initted_);
 
   const string& tablet_id = tablet_peer_->tablet_id();
 
   // Prevent log GC while we grab log segments and Tablet metadata.
-  string anchor_owner_token = Substitute("RemoteBootstrap-$0", session_id_);
+  string anchor_owner_token = Substitute("TabletCopy-$0", session_id_);
   tablet_peer_->log_anchor_registry()->Register(
       MinimumOpId().index(), anchor_owner_token, &log_anchor_);
 
@@ -108,7 +108,7 @@ Status RemoteBootstrapSession::Init() {
   if (!reader) {
     tablet::TabletStatePB tablet_state = tablet_peer_->state();
     return Status::IllegalState(Substitute(
-        "Unable to initialize remote bootstrap session for tablet $0. "
+        "Unable to initialize tablet copy session for tablet $0. "
         "Log reader is not available. Tablet state: $1 ($2)",
         tablet_id, tablet::TabletStatePB_Name(tablet_state), tablet_state));
   }
@@ -125,7 +125,7 @@ Status RemoteBootstrapSession::Init() {
   if (!consensus) {
     tablet::TabletStatePB tablet_state = tablet_peer_->state();
     return Status::IllegalState(Substitute(
-        "Unable to initialize remote bootstrap session for tablet $0. "
+        "Unable to initialize tablet copy session for tablet $0. "
         "Consensus is not available. Tablet state: $1 ($2)",
         tablet_id, tablet::TabletStatePB_Name(tablet_state), tablet_state));
   }
@@ -133,25 +133,25 @@ Status RemoteBootstrapSession::Init() {
 
   // Re-anchor on the highest OpId that was in the log right before we
   // snapshotted the log segments. This helps ensure that we don't end up in a
-  // remote bootstrap loop due to a follower falling too far behind the
-  // leader's log when remote bootstrap is slow. The remote controls when
-  // this anchor is released by ending the remote bootstrap session.
+  // tablet copy loop due to a follower falling too far behind the
+  // leader's log when tablet copy is slow. The remote controls when
+  // this anchor is released by ending the tablet copy session.
   RETURN_NOT_OK(tablet_peer_->log_anchor_registry()->UpdateRegistration(
       last_logged_opid.index(), anchor_owner_token, &log_anchor_));
 
   LOG(INFO) << Substitute(
-      "T $0 P $1: Remote bootstrap: opened $2 blocks and $3 log segments",
+      "T $0 P $1: Tablet Copy: opened $2 blocks and $3 log segments",
       tablet_id, consensus->peer_uuid(), data_blocks.size(), log_segments_.size());
   initted_ = true;
   return Status::OK();
 }
 
-const std::string& RemoteBootstrapSession::tablet_id() const {
+const std::string& TabletCopySession::tablet_id() const {
   DCHECK(initted_);
   return tablet_peer_->tablet_id();
 }
 
-const std::string& RemoteBootstrapSession::requestor_uuid() const {
+const std::string& TabletCopySession::requestor_uuid() const {
   DCHECK(initted_);
   return requestor_uuid_;
 }
@@ -162,10 +162,10 @@ static int64_t DetermineReadLength(int64_t bytes_remaining, int64_t requested_le
   static const int kOverhead = 4096;
 
   if (requested_len <= 0) {
-    requested_len = FLAGS_remote_bootstrap_transfer_chunk_size_bytes;
+    requested_len = FLAGS_tablet_copy_transfer_chunk_size_bytes;
   } else {
     requested_len = std::min<int64_t>(requested_len,
-                                      FLAGS_remote_bootstrap_transfer_chunk_size_bytes);
+                                      FLAGS_tablet_copy_transfer_chunk_size_bytes);
   }
   requested_len = std::min<int64_t>(requested_len, FLAGS_rpc_max_message_size - kOverhead);
   CHECK_GT(requested_len, 0) << "rpc_max_message_size is too low to transfer data: "
@@ -177,10 +177,10 @@ static int64_t DetermineReadLength(int64_t bytes_remaining, int64_t requested_le
 // length, the file itself, and the offset into the file to be read from.
 static Status GetResponseDataSize(int64_t total_size,
                                   uint64_t offset, int64_t client_maxlen,
-                                  RemoteBootstrapErrorPB::Code* error_code, int64_t* data_size) {
+                                  TabletCopyErrorPB::Code* error_code, int64_t* data_size) {
   // If requested offset is off the end of the data, bail.
   if (offset >= total_size) {
-    *error_code = RemoteBootstrapErrorPB::INVALID_REMOTE_BOOTSTRAP_REQUEST;
+    *error_code = TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST;
     return Status::InvalidArgument(
         Substitute("Requested offset ($0) is beyond the data size ($1)",
                    offset, total_size));
@@ -204,7 +204,7 @@ static Status ReadFileChunkToBuf(const Info* info,
                                  uint64_t offset, int64_t client_maxlen,
                                  const string& data_name,
                                  string* data, int64_t* file_size,
-                                 RemoteBootstrapErrorPB::Code* error_code) {
+                                 TabletCopyErrorPB::Code* error_code) {
   int64_t response_data_size = 0;
   RETURN_NOT_OK_PREPEND(GetResponseDataSize(info->size, offset, client_maxlen, error_code,
                                             &response_data_size),
@@ -224,7 +224,7 @@ static Status ReadFileChunkToBuf(const Info* info,
     s = s.CloneAndPrepend(
         Substitute("Unable to read existing file for $0", data_name));
     LOG(WARNING) << s.ToString();
-    *error_code = RemoteBootstrapErrorPB::IO_ERROR;
+    *error_code = TabletCopyErrorPB::IO_ERROR;
     return s;
   }
   // Figure out if Slice points to buf or if Slice points to the mmap.
@@ -233,17 +233,17 @@ static Status ReadFileChunkToBuf(const Info* info,
     memcpy(buf, slice.data(), slice.size());
   }
   chunk_timer.stop();
-  TRACE("Remote bootstrap: $0: $1 total bytes read. Total time elapsed: $2",
+  TRACE("Tablet Copy: $0: $1 total bytes read. Total time elapsed: $2",
         data_name, response_data_size, chunk_timer.elapsed().ToString());
 
   *file_size = info->size;
   return Status::OK();
 }
 
-Status RemoteBootstrapSession::GetBlockPiece(const BlockId& block_id,
+Status TabletCopySession::GetBlockPiece(const BlockId& block_id,
                                              uint64_t offset, int64_t client_maxlen,
                                              string* data, int64_t* block_file_size,
-                                             RemoteBootstrapErrorPB::Code* error_code) {
+                                             TabletCopyErrorPB::Code* error_code) {
   ImmutableReadableBlockInfo* block_info;
   RETURN_NOT_OK(FindBlock(block_id, &block_info, error_code));
 
@@ -259,10 +259,10 @@ Status RemoteBootstrapSession::GetBlockPiece(const BlockId& block_id,
   return Status::OK();
 }
 
-Status RemoteBootstrapSession::GetLogSegmentPiece(uint64_t segment_seqno,
+Status TabletCopySession::GetLogSegmentPiece(uint64_t segment_seqno,
                                                   uint64_t offset, int64_t client_maxlen,
                                                   std::string* data, int64_t* block_file_size,
-                                                  RemoteBootstrapErrorPB::Code* error_code) {
+                                                  TabletCopyErrorPB::Code* error_code) {
   ImmutableRandomAccessFileInfo* file_info;
   RETURN_NOT_OK(FindLogSegment(segment_seqno, &file_info, error_code));
   RETURN_NOT_OK(ReadFileChunkToBuf(file_info, offset, client_maxlen,
@@ -275,7 +275,7 @@ Status RemoteBootstrapSession::GetLogSegmentPiece(uint64_t segment_seqno,
   return Status::OK();
 }
 
-bool RemoteBootstrapSession::IsBlockOpenForTests(const BlockId& block_id) const {
+bool TabletCopySession::IsBlockOpenForTests(const BlockId& block_id) const {
   MutexLock l(session_lock_);
   return ContainsKey(blocks_, block_id);
 }
@@ -300,7 +300,7 @@ static Status AddImmutableFileToMap(Collection* const cache,
   return Status::OK();
 }
 
-Status RemoteBootstrapSession::OpenBlockUnlocked(const BlockId& block_id) {
+Status TabletCopySession::OpenBlockUnlocked(const BlockId& block_id) {
   session_lock_.AssertAcquired();
 
   gscoped_ptr<ReadableBlock> block;
@@ -328,19 +328,19 @@ Status RemoteBootstrapSession::OpenBlockUnlocked(const BlockId& block_id) {
   return s;
 }
 
-Status RemoteBootstrapSession::FindBlock(const BlockId& block_id,
+Status TabletCopySession::FindBlock(const BlockId& block_id,
                                          ImmutableReadableBlockInfo** block_info,
-                                         RemoteBootstrapErrorPB::Code* error_code) {
+                                         TabletCopyErrorPB::Code* error_code) {
   Status s;
   MutexLock l(session_lock_);
   if (!FindCopy(blocks_, block_id, block_info)) {
-    *error_code = RemoteBootstrapErrorPB::BLOCK_NOT_FOUND;
+    *error_code = TabletCopyErrorPB::BLOCK_NOT_FOUND;
     s = Status::NotFound("Block not found", block_id.ToString());
   }
   return s;
 }
 
-Status RemoteBootstrapSession::OpenLogSegmentUnlocked(uint64_t segment_seqno) {
+Status TabletCopySession::OpenLogSegmentUnlocked(uint64_t segment_seqno) {
   session_lock_.AssertAcquired();
 
   scoped_refptr<log::ReadableLogSegment> log_segment;
@@ -366,19 +366,19 @@ Status RemoteBootstrapSession::OpenLogSegmentUnlocked(uint64_t segment_seqno) {
   return s;
 }
 
-Status RemoteBootstrapSession::FindLogSegment(uint64_t segment_seqno,
+Status TabletCopySession::FindLogSegment(uint64_t segment_seqno,
                                               ImmutableRandomAccessFileInfo** file_info,
-                                              RemoteBootstrapErrorPB::Code* error_code) {
+                                              TabletCopyErrorPB::Code* error_code) {
   MutexLock l(session_lock_);
   if (!FindCopy(logs_, segment_seqno, file_info)) {
-    *error_code = RemoteBootstrapErrorPB::WAL_SEGMENT_NOT_FOUND;
+    *error_code = TabletCopyErrorPB::WAL_SEGMENT_NOT_FOUND;
     return Status::NotFound(Substitute("Segment with sequence number $0 not found",
                                        segment_seqno));
   }
   return Status::OK();
 }
 
-Status RemoteBootstrapSession::UnregisterAnchorIfNeededUnlocked() {
+Status TabletCopySession::UnregisterAnchorIfNeededUnlocked() {
   return tablet_peer_->log_anchor_registry()->UnregisterIfAnchored(&log_anchor_);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_session.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_session.h b/src/kudu/tserver/remote_bootstrap_session.h
index b48b99b..7eb2383 100644
--- a/src/kudu/tserver/remote_bootstrap_session.h
+++ b/src/kudu/tserver/remote_bootstrap_session.h
@@ -14,8 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_TSERVER_REMOTE_BOOTSTRAP_SESSION_H_
-#define KUDU_TSERVER_REMOTE_BOOTSTRAP_SESSION_H_
+#ifndef KUDU_TSERVER_TABLET_COPY_SESSION_H_
+#define KUDU_TSERVER_TABLET_COPY_SESSION_H_
 
 #include <memory>
 #include <string>
@@ -80,13 +80,13 @@ struct ImmutableReadableBlockInfo {
   }
 };
 
-// A potential Learner must establish a RemoteBootstrapSession with the leader in order
+// A potential Learner must establish a TabletCopySession with the leader in order
 // to fetch the needed superblock, blocks, and log segments.
 // This class is refcounted to make it easy to remove it from the session map
 // on expiration while it is in use by another thread.
-class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSession> {
+class TabletCopySession : public RefCountedThreadSafe<TabletCopySession> {
  public:
-  RemoteBootstrapSession(const scoped_refptr<tablet::TabletPeer>& tablet_peer,
+  TabletCopySession(const scoped_refptr<tablet::TabletPeer>& tablet_peer,
                          std::string session_id, std::string requestor_uuid,
                          FsManager* fs_manager);
 
@@ -113,7 +113,7 @@ class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSessio
   Status GetBlockPiece(const BlockId& block_id,
                        uint64_t offset, int64_t client_maxlen,
                        std::string* data, int64_t* block_file_size,
-                       RemoteBootstrapErrorPB::Code* error_code);
+                       TabletCopyErrorPB::Code* error_code);
 
   // Get a piece of a log segment.
   // The behavior and params are very similar to GetBlockPiece(), but this one
@@ -121,7 +121,7 @@ class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSessio
   Status GetLogSegmentPiece(uint64_t segment_seqno,
                             uint64_t offset, int64_t client_maxlen,
                             std::string* data, int64_t* log_file_size,
-                            RemoteBootstrapErrorPB::Code* error_code);
+                            TabletCopyErrorPB::Code* error_code);
 
   const tablet::TabletSuperBlockPB& tablet_superblock() const {
     DCHECK(initted_);
@@ -142,12 +142,12 @@ class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSessio
   bool IsBlockOpenForTests(const BlockId& block_id) const;
 
  private:
-  friend class RefCountedThreadSafe<RemoteBootstrapSession>;
+  friend class RefCountedThreadSafe<TabletCopySession>;
 
   typedef std::unordered_map<BlockId, ImmutableReadableBlockInfo*, BlockIdHash> BlockMap;
   typedef std::unordered_map<uint64_t, ImmutableRandomAccessFileInfo*> LogMap;
 
-  ~RemoteBootstrapSession();
+  ~TabletCopySession();
 
   // Open the block and add it to the block map.
   Status OpenBlockUnlocked(const BlockId& block_id);
@@ -155,7 +155,7 @@ class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSessio
   // Look up cached block information.
   Status FindBlock(const BlockId& block_id,
                    ImmutableReadableBlockInfo** block_info,
-                   RemoteBootstrapErrorPB::Code* error_code);
+                   TabletCopyErrorPB::Code* error_code);
 
   // Snapshot the log segment's length and put it into segment map.
   Status OpenLogSegmentUnlocked(uint64_t segment_seqno);
@@ -163,7 +163,7 @@ class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSessio
   // Look up log segment in cache or log segment map.
   Status FindLogSegment(uint64_t segment_seqno,
                         ImmutableRandomAccessFileInfo** file_info,
-                        RemoteBootstrapErrorPB::Code* error_code);
+                        TabletCopyErrorPB::Code* error_code);
 
   // Unregister log anchor, if it's registered.
   Status UnregisterAnchorIfNeededUnlocked();
@@ -190,10 +190,10 @@ class RemoteBootstrapSession : public RefCountedThreadSafe<RemoteBootstrapSessio
 
   log::LogAnchor log_anchor_;
 
-  DISALLOW_COPY_AND_ASSIGN(RemoteBootstrapSession);
+  DISALLOW_COPY_AND_ASSIGN(TabletCopySession);
 };
 
 } // namespace tserver
 } // namespace kudu
 
-#endif // KUDU_TSERVER_REMOTE_BOOTSTRAP_SESSION_H_
+#endif // KUDU_TSERVER_TABLET_COPY_SESSION_H_

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 c58b79d..abc7861 100644
--- a/src/kudu/tserver/tablet_peer_lookup.h
+++ b/src/kudu/tserver/tablet_peer_lookup.h
@@ -31,7 +31,7 @@ class HostPort;
 class NodeInstancePB;
 
 namespace consensus {
-class StartRemoteBootstrapRequestPB;
+class StartTabletCopyRequestPB;
 } // namespace consensus
 
 namespace tablet {
@@ -51,7 +51,7 @@ class TabletPeerLookupIf {
 
   virtual const NodeInstancePB& NodeInstance() const = 0;
 
-  virtual Status StartRemoteBootstrap(const consensus::StartRemoteBootstrapRequestPB& req,
+  virtual Status StartTabletCopy(const consensus::StartTabletCopyRequestPB& req,
                                       boost::optional<TabletServerErrorPB::Code>* error_code) = 0;
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server.cc b/src/kudu/tserver/tablet_server.cc
index d933491..14f2049 100644
--- a/src/kudu/tserver/tablet_server.cc
+++ b/src/kudu/tserver/tablet_server.cc
@@ -114,13 +114,13 @@ Status TabletServer::Start() {
   gscoped_ptr<ServiceIf> consensus_service(new ConsensusServiceImpl(metric_entity(),
                                                                     result_tracker(),
                                                                     tablet_manager_.get()));
-  gscoped_ptr<ServiceIf> remote_bootstrap_service(new RemoteBootstrapServiceImpl(
+  gscoped_ptr<ServiceIf> tablet_copy_service(new TabletCopyServiceImpl(
       fs_manager_.get(), tablet_manager_.get(), metric_entity(), result_tracker()));
 
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(ts_service)));
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(admin_service)));
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(consensus_service)));
-  RETURN_NOT_OK(ServerBase::RegisterService(std::move(remote_bootstrap_service)));
+  RETURN_NOT_OK(ServerBase::RegisterService(std::move(tablet_copy_service)));
   RETURN_NOT_OK(ServerBase::Start());
 
   RETURN_NOT_OK(heartbeater_->Start());

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index f3f6db0..8ae9424 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -108,8 +108,8 @@ using consensus::LeaderStepDownRequestPB;
 using consensus::LeaderStepDownResponsePB;
 using consensus::RunLeaderElectionRequestPB;
 using consensus::RunLeaderElectionResponsePB;
-using consensus::StartRemoteBootstrapRequestPB;
-using consensus::StartRemoteBootstrapResponsePB;
+using consensus::StartTabletCopyRequestPB;
+using consensus::StartTabletCopyResponsePB;
 using consensus::VoteRequestPB;
 using consensus::VoteResponsePB;
 
@@ -980,14 +980,14 @@ void ConsensusServiceImpl::GetConsensusState(const consensus::GetConsensusStateR
   context->RespondSuccess();
 }
 
-void ConsensusServiceImpl::StartRemoteBootstrap(const StartRemoteBootstrapRequestPB* req,
-                                                StartRemoteBootstrapResponsePB* resp,
+void ConsensusServiceImpl::StartTabletCopy(const StartTabletCopyRequestPB* req,
+                                                StartTabletCopyResponsePB* resp,
                                                 rpc::RpcContext* context) {
-  if (!CheckUuidMatchOrRespond(tablet_manager_, "StartRemoteBootstrap", req, resp, context)) {
+  if (!CheckUuidMatchOrRespond(tablet_manager_, "StartTabletCopy", req, resp, context)) {
     return;
   }
   boost::optional<TabletServerErrorPB::Code> error_code;
-  Status s = tablet_manager_->StartRemoteBootstrap(*req, &error_code);
+  Status s = tablet_manager_->StartTabletCopy(*req, &error_code);
   if (!s.ok()) {
     SetupErrorAndRespond(resp->mutable_error(), s,
                          error_code.get_value_or(TabletServerErrorPB::UNKNOWN_ERROR),

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/tablet_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.h b/src/kudu/tserver/tablet_service.h
index 7147d72..430fb13 100644
--- a/src/kudu/tserver/tablet_service.h
+++ b/src/kudu/tserver/tablet_service.h
@@ -159,8 +159,8 @@ class ConsensusServiceImpl : public consensus::ConsensusServiceIf {
                                  consensus::GetConsensusStateResponsePB *resp,
                                  rpc::RpcContext *context) OVERRIDE;
 
-  virtual void StartRemoteBootstrap(const consensus::StartRemoteBootstrapRequestPB* req,
-                                    consensus::StartRemoteBootstrapResponsePB* resp,
+  virtual void StartTabletCopy(const consensus::StartTabletCopyRequestPB* req,
+                                    consensus::StartTabletCopyResponsePB* resp,
                                     rpc::RpcContext* context) OVERRIDE;
 
  private:

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 729498e..402f6cb 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -91,7 +91,7 @@ TAG_FLAG(fault_crash_after_cmeta_deleted, unsafe);
 
 DEFINE_double(fault_crash_after_rb_files_fetched, 0.0,
               "Fraction of the time when the tablet will crash immediately "
-              "after fetching the files during a remote bootstrap but before "
+              "after fetching the files during a tablet copy but before "
               "marking the superblock as TABLET_DATA_READY. "
               "(For testing only!)");
 TAG_FLAG(fault_crash_after_rb_files_fetched, unsafe);
@@ -125,7 +125,7 @@ using consensus::ConsensusStatePB;
 using consensus::OpId;
 using consensus::RaftConfigPB;
 using consensus::RaftPeerPB;
-using consensus::StartRemoteBootstrapRequestPB;
+using consensus::StartTabletCopyRequestPB;
 using log::Log;
 using master::ReportedTabletPB;
 using master::TabletReportPB;
@@ -144,7 +144,7 @@ using tablet::TabletMetadata;
 using tablet::TabletPeer;
 using tablet::TabletStatusListener;
 using tablet::TabletStatusPB;
-using tserver::RemoteBootstrapClient;
+using tserver::TabletCopyClient;
 
 TSTabletManager::TSTabletManager(FsManager* fs_manager,
                                  TabletServer* server,
@@ -325,17 +325,17 @@ Status TSTabletManager::CheckLeaderTermNotLower(const string& tablet_id,
     Status s = Status::InvalidArgument(
         Substitute("Leader has replica of tablet $0 with term $1 "
                     "lower than last logged term $2 on local replica. Rejecting "
-                    "remote bootstrap request",
+                    "tablet copy request",
                     tablet_id,
                     leader_term, last_logged_term));
-    LOG(WARNING) << LogPrefix(tablet_id) << "Remote bootstrap: " << s.ToString();
+    LOG(WARNING) << LogPrefix(tablet_id) << "Tablet Copy: " << s.ToString();
     return s;
   }
   return Status::OK();
 }
 
-Status TSTabletManager::StartRemoteBootstrap(
-    const StartRemoteBootstrapRequestPB& req,
+Status TSTabletManager::StartTabletCopy(
+    const StartTabletCopyRequestPB& req,
     boost::optional<TabletServerErrorPB::Code>* error_code) {
   const string& tablet_id = req.tablet_id();
   const string& bootstrap_source_uuid = req.bootstrap_peer_uuid();
@@ -355,7 +355,7 @@ Status TSTabletManager::StartRemoteBootstrap(
       meta = old_tablet_peer->tablet_metadata();
       replacing_tablet = true;
     }
-    Status ret = StartTabletStateTransitionUnlocked(tablet_id, "remote bootstrapping tablet",
+    Status ret = StartTabletStateTransitionUnlocked(tablet_id, "tablet copyping tablet",
                                                     &deleter);
     if (!ret.ok()) {
       *error_code = TabletServerErrorPB::ALREADY_INPROGRESS;
@@ -369,8 +369,8 @@ Status TSTabletManager::StartRemoteBootstrap(
     switch (data_state) {
       case TABLET_DATA_COPYING:
         // This should not be possible due to the transition_in_progress_ "lock".
-        LOG(FATAL) << LogPrefix(tablet_id) << " Remote bootstrap: "
-                   << "Found tablet in TABLET_DATA_COPYING state during StartRemoteBootstrap()";
+        LOG(FATAL) << LogPrefix(tablet_id) << " Tablet Copy: "
+                   << "Found tablet in TABLET_DATA_COPYING state during StartTabletCopy()";
       case TABLET_DATA_TOMBSTONED: {
         int64_t last_logged_term = meta->tombstone_last_logged_opid().term();
         RETURN_NOT_OK(CheckLeaderTermNotLower(tablet_id, leader_term, last_logged_term));
@@ -393,7 +393,7 @@ Status TSTabletManager::StartRemoteBootstrap(
         // in flight and it may be possible for the same check in the remote
         // bootstrap client Start() method to fail. This will leave the replica in
         // a tombstoned state, and then the leader with the latest log entries
-        // will simply remote bootstrap this replica again. We could try to
+        // will simply tablet copy this replica again. We could try to
         // check again after calling Shutdown(), and if the check fails, try to
         // reopen the tablet. For now, we live with the (unlikely) race.
         RETURN_NOT_OK_PREPEND(DeleteTabletData(meta, TABLET_DATA_TOMBSTONED, last_logged_opid),
@@ -403,19 +403,19 @@ Status TSTabletManager::StartRemoteBootstrap(
       }
       default:
         return Status::IllegalState(
-            Substitute("Found tablet in unsupported state for remote bootstrap. "
+            Substitute("Found tablet in unsupported state for tablet copy. "
                         "Tablet: $0, tablet data state: $1",
                         tablet_id, TabletDataState_Name(data_state)));
     }
   }
 
-  string init_msg = kLogPrefix + Substitute("Initiating remote bootstrap from Peer $0 ($1)",
+  string init_msg = kLogPrefix + Substitute("Initiating tablet copy from Peer $0 ($1)",
                                             bootstrap_source_uuid,
                                             bootstrap_source_addr.ToString());
   LOG(INFO) << init_msg;
   TRACE(init_msg);
 
-  RemoteBootstrapClient rb_client(tablet_id, fs_manager_, server_->messenger());
+  TabletCopyClient rb_client(tablet_id, fs_manager_, server_->messenger());
 
   // Download and persist the remote superblock in TABLET_DATA_COPYING state.
   if (replacing_tablet) {
@@ -434,14 +434,14 @@ Status TSTabletManager::StartRemoteBootstrap(
 
   // Download all of the remote files.
   TOMBSTONE_NOT_OK(rb_client.FetchAll(tablet_peer->status_listener()), meta,
-                   "Remote bootstrap: Unable to fetch data from remote peer " +
+                   "Tablet Copy: Unable to fetch data from remote peer " +
                    bootstrap_source_uuid + " (" + bootstrap_source_addr.ToString() + ")");
 
   MAYBE_FAULT(FLAGS_fault_crash_after_rb_files_fetched);
 
   // Write out the last files to make the new replica visible and update the
   // TabletDataState in the superblock to TABLET_DATA_READY.
-  TOMBSTONE_NOT_OK(rb_client.Finish(), meta, "Remote bootstrap: Failure calling Finish()");
+  TOMBSTONE_NOT_OK(rb_client.Finish(), meta, "Tablet Copy: Failure calling Finish()");
 
   // We run this asynchronously. We don't tombstone the tablet if this fails,
   // because if we were to fail to open the tablet, on next startup, it's in a
@@ -580,12 +580,12 @@ Status TSTabletManager::StartTabletStateTransitionUnlocked(
     // replica of every tablet with the TABLET_DATA_DELETED parameter, which
     // indicates a "permanent" tablet deletion. If a follower services
     // DeleteTablet() before the leader does, it's possible for the leader to
-    // react to the missing replica by asking the follower to remote bootstrap
+    // react to the missing replica by asking the follower to tablet copy
     // itself.
     //
     // If the tablet was permanently deleted, we should not allow it to
     // transition back to "liveness" because that can result in flapping back
-    // and forth between deletion and remote bootstrapping.
+    // and forth between deletion and tablet copyping.
     return Status::IllegalState(
         Substitute("Tablet $0 was permanently deleted. Cannot transition from state $1.",
                    tablet_id, TabletDataState_Name(TABLET_DATA_DELETED)));
@@ -634,7 +634,7 @@ void TSTabletManager::OpenTablet(const scoped_refptr<TabletMetadata>& meta,
   LOG_TIMING_PREFIX(INFO, LogPrefix(tablet_id), "bootstrapping tablet") {
     // Disable tracing for the bootstrap, since this would result in
     // potentially millions of transaction traces being attached to the
-    // RemoteBootstrap trace.
+    // TabletCopy trace.
     ADOPT_TRACE(nullptr);
     // TODO: handle crash mid-creation of tablet? do we ever end up with a
     // partially created tablet here?
@@ -945,12 +945,12 @@ void TSTabletManager::LogAndTombstone(const scoped_refptr<TabletMetadata>& meta,
   const string kLogPrefix = "T " + tablet_id + " P " + fs_manager_->uuid() + ": ";
   LOG(WARNING) << kLogPrefix << msg << ": " << s.ToString();
 
-  // Tombstone the tablet when remote bootstrap fails.
-  LOG(INFO) << kLogPrefix << "Tombstoning tablet after failed remote bootstrap";
+  // Tombstone the tablet when tablet copy fails.
+  LOG(INFO) << kLogPrefix << "Tombstoning tablet after failed tablet copy";
   Status delete_status = DeleteTabletData(meta, TABLET_DATA_TOMBSTONED, boost::optional<OpId>());
   if (PREDICT_FALSE(!delete_status.ok())) {
     // This failure should only either indicate a bug or an IO error.
-    LOG(FATAL) << kLogPrefix << "Failed to tombstone tablet after remote bootstrap: "
+    LOG(FATAL) << kLogPrefix << "Failed to tombstone tablet after tablet copy: "
                << delete_status.ToString();
   }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 347ab5d..8005683 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -144,12 +144,12 @@ class TSTabletManager : public tserver::TabletPeerLookupIf {
 
   virtual const NodeInstancePB& NodeInstance() const OVERRIDE;
 
-  // Initiate remote bootstrap of the specified tablet.
-  // See the StartRemoteBootstrap() RPC declaration in consensus.proto for details.
+  // Initiate tablet copy of the specified tablet.
+  // See the StartTabletCopy() 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,
+  virtual Status StartTabletCopy(
+      const consensus::StartTabletCopyRequestPB& req,
       boost::optional<TabletServerErrorPB::Code>* error_code) OVERRIDE;
 
   // Adds updated tablet information to 'report'.
@@ -259,7 +259,7 @@ class TSTabletManager : public tserver::TabletPeerLookupIf {
                           const boost::optional<consensus::OpId>& last_logged_opid);
 
   // Return Status::IllegalState if leader_term < last_logged_term.
-  // Helper function for use with remote bootstrap.
+  // Helper function for use with tablet copy.
   Status CheckLeaderTermNotLower(const std::string& tablet_id,
                                  int64_t leader_term,
                                  int64_t last_logged_term);

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/tserver.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver.proto b/src/kudu/tserver/tserver.proto
index 5964817..ffafade 100644
--- a/src/kudu/tserver/tserver.proto
+++ b/src/kudu/tserver/tserver.proto
@@ -86,7 +86,7 @@ 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.
+    // The requested operation is already inprogress, e.g. TabletCopy.
     ALREADY_INPROGRESS = 18;
 
     // The request is throttled.


[2/3] kudu git commit: Rename Remote Bootstrap to Tablet Copy (part 1)

Posted by to...@apache.org.
http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/CMakeLists.txt b/src/kudu/tserver/CMakeLists.txt
index 8568fea..a2e34e1 100644
--- a/src/kudu/tserver/CMakeLists.txt
+++ b/src/kudu/tserver/CMakeLists.txt
@@ -20,11 +20,11 @@
 #########################################
 
 KRPC_GENERATE(
-  REMOTE_BOOTSTRAP_KRPC_SRCS REMOTE_BOOTSTRAP_KRPC_HDRS REMOTE_BOOTSTRAP_KRPC_TGTS
+  TABLET_COPY_KRPC_SRCS TABLET_COPY_KRPC_HDRS TABLET_COPY_KRPC_TGTS
   SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
   BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
   PROTO_FILES remote_bootstrap.proto)
-set(REMOTE_BOOTSTRAP_KRPC_LIBS
+set(TABLET_COPY_KRPC_LIBS
   consensus_proto
   krpc
   protobuf
@@ -32,9 +32,9 @@ set(REMOTE_BOOTSTRAP_KRPC_LIBS
   tablet_proto
   wire_protocol_proto)
 ADD_EXPORTABLE_LIBRARY(remote_bootstrap_proto
-  SRCS ${REMOTE_BOOTSTRAP_KRPC_SRCS}
-  DEPS ${REMOTE_BOOTSTRAP_KRPC_LIBS}
-  NONLINK_DEPS ${REMOTE_BOOTSTRAP_KRPC_TGTS})
+  SRCS ${TABLET_COPY_KRPC_SRCS}
+  DEPS ${TABLET_COPY_KRPC_LIBS}
+  NONLINK_DEPS ${TABLET_COPY_KRPC_TGTS})
 
 #########################################
 # tserver_proto

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap-test-base.h b/src/kudu/tserver/remote_bootstrap-test-base.h
index ef5637f..0f7faa1 100644
--- a/src/kudu/tserver/remote_bootstrap-test-base.h
+++ b/src/kudu/tserver/remote_bootstrap-test-base.h
@@ -14,8 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_TSERVER_REMOTE_BOOTSTRAP_TEST_BASE_H_
-#define KUDU_TSERVER_REMOTE_BOOTSTRAP_TEST_BASE_H_
+#ifndef KUDU_TSERVER_TABLET_COPY_TEST_BASE_H_
+#define KUDU_TSERVER_TABLET_COPY_TEST_BASE_H_
 
 #include "kudu/tserver/tablet_server-test-base.h"
 
@@ -39,7 +39,7 @@ using consensus::MinimumOpId;
 // Number of times to roll the log.
 static const int kNumLogRolls = 2;
 
-class RemoteBootstrapTest : public TabletServerTestBase {
+class TabletCopyTest : public TabletServerTestBase {
  public:
   virtual void SetUp() OVERRIDE {
     NO_FATALS(TabletServerTestBase::SetUp());
@@ -47,7 +47,7 @@ class RemoteBootstrapTest : public TabletServerTestBase {
     // Prevent logs from being deleted out from under us until / unless we want
     // to test that we are anchoring correctly. Since GenerateTestData() does a
     // Flush(), Log GC is allowed to eat the logs before we get around to
-    // starting a remote bootstrap session.
+    // starting a tablet copy session.
     tablet_peer_->log_anchor_registry()->Register(
       MinimumOpId().index(), CURRENT_TEST_NAME(), &anchor_);
     NO_FATALS(GenerateTestData());
@@ -76,7 +76,7 @@ class RemoteBootstrapTest : public TabletServerTestBase {
   }
 
   // Generate the test data for the tablet and do the flushing we assume will be
-  // done in the unit tests for remote bootstrap.
+  // done in the unit tests for tablet copy.
   void GenerateTestData() {
     const int kIncr = 50;
     LOG_TIMING(INFO, "Loading test data") {
@@ -123,4 +123,4 @@ class RemoteBootstrapTest : public TabletServerTestBase {
 } // namespace tserver
 } // namespace kudu
 
-#endif // KUDU_TSERVER_REMOTE_BOOTSTRAP_TEST_BASE_H_
+#endif // KUDU_TSERVER_TABLET_COPY_TEST_BASE_H_

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap.proto b/src/kudu/tserver/remote_bootstrap.proto
index 225f3a5..1e89919 100644
--- a/src/kudu/tserver/remote_bootstrap.proto
+++ b/src/kudu/tserver/remote_bootstrap.proto
@@ -24,29 +24,29 @@ import "kudu/fs/fs.proto";
 import "kudu/rpc/rpc_header.proto";
 import "kudu/tablet/metadata.proto";
 
-// RaftConfig remote bootstrap RPC calls.
-service RemoteBootstrapService {
-  // Establish a remote bootstrap session.
-  rpc BeginRemoteBootstrapSession(BeginRemoteBootstrapSessionRequestPB)
-      returns (BeginRemoteBootstrapSessionResponsePB);
+// RaftConfig tablet copy RPC calls.
+service TabletCopyService {
+  // Establish a tablet copy session.
+  rpc BeginTabletCopySession(BeginTabletCopySessionRequestPB)
+      returns (BeginTabletCopySessionResponsePB);
 
   // Check whether the specified session is active.
-  rpc CheckSessionActive(CheckRemoteBootstrapSessionActiveRequestPB)
-      returns (CheckRemoteBootstrapSessionActiveResponsePB);
+  rpc CheckSessionActive(CheckTabletCopySessionActiveRequestPB)
+      returns (CheckTabletCopySessionActiveResponsePB);
 
   // Fetch data (blocks, logs) from the server.
   rpc FetchData(FetchDataRequestPB)
       returns (FetchDataResponsePB);
 
-  // End a remote bootstrap session, allow server to release resources.
-  rpc EndRemoteBootstrapSession(EndRemoteBootstrapSessionRequestPB)
-      returns (EndRemoteBootstrapSessionResponsePB);
+  // End a tablet copy session, allow server to release resources.
+  rpc EndTabletCopySession(EndTabletCopySessionRequestPB)
+      returns (EndTabletCopySessionResponsePB);
 }
 
-// Remote bootstrap-specific errors use this protobuf.
-message RemoteBootstrapErrorPB {
+// Tablet Copy-specific errors use this protobuf.
+message TabletCopyErrorPB {
   extend kudu.rpc.ErrorStatusPB {
-    optional RemoteBootstrapErrorPB remote_bootstrap_error_ext = 102;
+    optional TabletCopyErrorPB tablet_copy_error_ext = 102;
   }
 
   enum Code {
@@ -57,7 +57,7 @@ message RemoteBootstrapErrorPB {
     // able to easily parse the error.
     UNKNOWN_ERROR = 1;
 
-    // The specified remote bootstrap session either never existed or has expired.
+    // The specified tablet copy session either never existed or has expired.
     NO_SESSION = 2;
 
     // Unknown tablet.
@@ -70,7 +70,7 @@ message RemoteBootstrapErrorPB {
     WAL_SEGMENT_NOT_FOUND = 5;
 
     // Invalid request. Possibly missing parameters.
-    INVALID_REMOTE_BOOTSTRAP_REQUEST = 6;
+    INVALID_TABLET_COPY_REQUEST = 6;
 
     // Error reading or transferring data.
     IO_ERROR = 7;
@@ -85,7 +85,7 @@ message RemoteBootstrapErrorPB {
   required AppStatusPB status = 2;
 }
 
-message BeginRemoteBootstrapSessionRequestPB {
+message BeginTabletCopySessionRequestPB {
   // permanent_uuid of the requesting peer.
   required bytes requestor_uuid = 1;
 
@@ -93,7 +93,7 @@ message BeginRemoteBootstrapSessionRequestPB {
   required bytes tablet_id = 2;
 }
 
-message BeginRemoteBootstrapSessionResponsePB {
+message BeginTabletCopySessionResponsePB {
   // Opaque session id assigned by the server.
   // No guarantees are made as to the format of the session id.
   required bytes session_id = 1;
@@ -111,23 +111,23 @@ message BeginRemoteBootstrapSessionResponsePB {
   repeated uint64 wal_segment_seqnos = 4;
 
   // A snapshot of the committed Consensus state at the time that the
-  // remote bootstrap session was started.
+  // tablet copy session was started.
   required consensus.ConsensusStatePB initial_committed_cstate = 5;
 
   // permanent_uuid of the responding peer.
   optional bytes responder_uuid = 6;
 }
 
-message CheckRemoteBootstrapSessionActiveRequestPB {
-  // Valid Session ID returned by a BeginRemoteBootstrapSession() RPC call.
+message CheckTabletCopySessionActiveRequestPB {
+  // Valid Session ID returned by a BeginTabletCopySession() RPC call.
   required bytes session_id = 1;
 
   // Set keepalive to true to reset the session timeout timer.
   optional bool keepalive = 2 [default = false];
 }
 
-message CheckRemoteBootstrapSessionActiveResponsePB {
-  // Whether the given session id represents an active remote bootstrap session.
+message CheckTabletCopySessionActiveResponsePB {
+  // Whether the given session id represents an active tablet copy session.
   required bool session_is_active = 1;
 }
 
@@ -149,7 +149,7 @@ message DataIdPB {
 }
 
 message FetchDataRequestPB {
-  // Valid Session ID returned by a BeginRemoteBootstrapSession() RPC call.
+  // Valid Session ID returned by a BeginTabletCopySession() RPC call.
   required bytes session_id = 1;
 
   // The server will use this ID to determine the key and type of data
@@ -189,16 +189,16 @@ message FetchDataResponsePB {
   required DataChunkPB chunk = 1;
 }
 
-message EndRemoteBootstrapSessionRequestPB {
+message EndTabletCopySessionRequestPB {
   required bytes session_id = 1;
 
   // Set to true if bootstrap is successful.
   required bool is_success = 2;
 
   // Client-provided error message. The server will log this error so that an
-  // admin can identify when bad things are happening with remote bootstrap.
+  // admin can identify when bad things are happening with tablet copy.
   optional AppStatusPB error = 3;
 }
 
-message EndRemoteBootstrapSessionResponsePB {
+message EndTabletCopySessionResponsePB {
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_client-test.cc b/src/kudu/tserver/remote_bootstrap_client-test.cc
index 101faa7..d1bfc16 100644
--- a/src/kudu/tserver/remote_bootstrap_client-test.cc
+++ b/src/kudu/tserver/remote_bootstrap_client-test.cc
@@ -32,10 +32,10 @@ using consensus::RaftPeerPB;
 using tablet::TabletMetadata;
 using tablet::TabletStatusListener;
 
-class RemoteBootstrapClientTest : public RemoteBootstrapTest {
+class TabletCopyClientTest : public TabletCopyTest {
  public:
   virtual void SetUp() OVERRIDE {
-    NO_FATALS(RemoteBootstrapTest::SetUp());
+    NO_FATALS(TabletCopyTest::SetUp());
 
     fs_manager_.reset(new FsManager(Env::Default(), GetTestPath("client_tablet")));
     ASSERT_OK(fs_manager_->CreateInitialFileSystemLayout());
@@ -43,7 +43,7 @@ class RemoteBootstrapClientTest : public RemoteBootstrapTest {
 
     tablet_peer_->WaitUntilConsensusRunning(MonoDelta::FromSeconds(10.0));
     rpc::MessengerBuilder(CURRENT_TEST_NAME()).Build(&messenger_);
-    client_.reset(new RemoteBootstrapClient(GetTabletId(),
+    client_.reset(new TabletCopyClient(GetTabletId(),
                                             fs_manager_.get(),
                                             messenger_));
     ASSERT_OK(GetRaftConfigLeader(tablet_peer_->consensus()
@@ -59,12 +59,12 @@ class RemoteBootstrapClientTest : public RemoteBootstrapTest {
 
   gscoped_ptr<FsManager> fs_manager_;
   shared_ptr<rpc::Messenger> messenger_;
-  gscoped_ptr<RemoteBootstrapClient> client_;
+  gscoped_ptr<TabletCopyClient> client_;
   scoped_refptr<TabletMetadata> meta_;
   RaftPeerPB leader_;
 };
 
-Status RemoteBootstrapClientTest::CompareFileContents(const string& path1, const string& path2) {
+Status TabletCopyClientTest::CompareFileContents(const string& path1, const string& path2) {
   shared_ptr<RandomAccessFile> file1, file2;
   RETURN_NOT_OK(env_util::OpenFileForRandom(fs_manager_->env(), path1, &file1));
   RETURN_NOT_OK(env_util::OpenFileForRandom(fs_manager_->env(), path2, &file2));
@@ -90,15 +90,15 @@ Status RemoteBootstrapClientTest::CompareFileContents(const string& path1, const
   return Status::OK();
 }
 
-// Basic begin / end remote bootstrap session.
-TEST_F(RemoteBootstrapClientTest, TestBeginEndSession) {
+// Basic begin / end tablet copy session.
+TEST_F(TabletCopyClientTest, TestBeginEndSession) {
   TabletStatusListener listener(meta_);
   ASSERT_OK(client_->FetchAll(&listener));
   ASSERT_OK(client_->Finish());
 }
 
 // Basic data block download unit test.
-TEST_F(RemoteBootstrapClientTest, TestDownloadBlock) {
+TEST_F(TabletCopyClientTest, TestDownloadBlock) {
   TabletStatusListener listener(meta_);
   BlockId block_id = FirstColumnBlockId(*client_->superblock_);
   Slice slice;
@@ -120,7 +120,7 @@ TEST_F(RemoteBootstrapClientTest, TestDownloadBlock) {
 }
 
 // Basic WAL segment download unit test.
-TEST_F(RemoteBootstrapClientTest, TestDownloadWalSegment) {
+TEST_F(TabletCopyClientTest, TestDownloadWalSegment) {
   ASSERT_OK(fs_manager_->CreateDirIfMissing(fs_manager_->GetTabletWalDir(GetTabletId())));
 
   uint64_t seqno = client_->wal_seqnos_[0];
@@ -140,7 +140,7 @@ TEST_F(RemoteBootstrapClientTest, TestDownloadWalSegment) {
 }
 
 // Ensure that we detect data corruption at the per-transfer level.
-TEST_F(RemoteBootstrapClientTest, TestVerifyData) {
+TEST_F(TabletCopyClientTest, TestVerifyData) {
   string good = "This is a known good string";
   string bad = "This is a known bad! string";
   const int kGoodOffset = 0;
@@ -204,7 +204,7 @@ vector<BlockId> GetAllSortedBlocks(const tablet::TabletSuperBlockPB& sb) {
 
 } // anonymous namespace
 
-TEST_F(RemoteBootstrapClientTest, TestDownloadAllBlocks) {
+TEST_F(TabletCopyClientTest, TestDownloadAllBlocks) {
   // Download all the blocks.
   ASSERT_OK(client_->DownloadBlocks());
 
@@ -224,7 +224,7 @@ TEST_F(RemoteBootstrapClientTest, TestDownloadAllBlocks) {
 
   // Verify that the old blocks aren't found. We're using a different
   // FsManager than 'tablet_peer', so the only way an old block could end
-  // up in ours is due to a remote bootstrap client bug.
+  // up in ours is due to a tablet copy client bug.
   for (const BlockId& block_id : old_data_blocks) {
     gscoped_ptr<fs::ReadableBlock> block;
     Status s = fs_manager_->OpenBlock(block_id, &block);

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 3eba8db..85b3942 100644
--- a/src/kudu/tserver/remote_bootstrap_client.cc
+++ b/src/kudu/tserver/remote_bootstrap_client.cc
@@ -44,24 +44,24 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/net/net_util.h"
 
-DEFINE_int32(remote_bootstrap_begin_session_timeout_ms, 3000,
-             "Tablet server RPC client timeout for BeginRemoteBootstrapSession calls. "
-             "Also used for EndRemoteBootstrapSession calls.");
-TAG_FLAG(remote_bootstrap_begin_session_timeout_ms, hidden);
+DEFINE_int32(tablet_copy_begin_session_timeout_ms, 3000,
+             "Tablet server RPC client timeout for BeginTabletCopySession calls. "
+             "Also used for EndTabletCopySession calls.");
+TAG_FLAG(tablet_copy_begin_session_timeout_ms, hidden);
 
-DEFINE_bool(remote_bootstrap_save_downloaded_metadata, false,
-            "Save copies of the downloaded remote bootstrap files for debugging purposes. "
+DEFINE_bool(tablet_copy_save_downloaded_metadata, false,
+            "Save copies of the downloaded tablet copy files for debugging purposes. "
             "Note: This is only intended for debugging and should not be normally used!");
-TAG_FLAG(remote_bootstrap_save_downloaded_metadata, advanced);
-TAG_FLAG(remote_bootstrap_save_downloaded_metadata, hidden);
-TAG_FLAG(remote_bootstrap_save_downloaded_metadata, runtime);
+TAG_FLAG(tablet_copy_save_downloaded_metadata, advanced);
+TAG_FLAG(tablet_copy_save_downloaded_metadata, hidden);
+TAG_FLAG(tablet_copy_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 "
+DEFINE_int32(tablet_copy_dowload_file_inject_latency_ms, 0,
+             "Injects latency into the loop that downloads files, causing tablet copy "
              "to take much longer. For use in tests only.");
-TAG_FLAG(remote_bootstrap_dowload_file_inject_latency_ms, hidden);
+TAG_FLAG(tablet_copy_dowload_file_inject_latency_ms, hidden);
 
-DECLARE_int32(remote_bootstrap_transfer_chunk_size_bytes);
+DECLARE_int32(tablet_copy_transfer_chunk_size_bytes);
 
 // RETURN_NOT_OK_PREPEND() with a remote-error unwinding step.
 #define RETURN_NOT_OK_UNWIND_PREPEND(status, controller, msg) \
@@ -91,7 +91,7 @@ using tablet::TabletMetadata;
 using tablet::TabletStatusListener;
 using tablet::TabletSuperBlockPB;
 
-RemoteBootstrapClient::RemoteBootstrapClient(std::string tablet_id,
+TabletCopyClient::TabletCopyClient(std::string tablet_id,
                                              FsManager* fs_manager,
                                              shared_ptr<Messenger> messenger)
     : tablet_id_(std::move(tablet_id)),
@@ -105,12 +105,12 @@ RemoteBootstrapClient::RemoteBootstrapClient(std::string tablet_id,
       session_idle_timeout_millis_(0),
       start_time_micros_(0) {}
 
-RemoteBootstrapClient::~RemoteBootstrapClient() {
-  // Note: Ending the remote bootstrap session releases anchors on the remote.
-  WARN_NOT_OK(EndRemoteSession(), "Unable to close remote bootstrap session");
+TabletCopyClient::~TabletCopyClient() {
+  // Note: Ending the tablet copy session releases anchors on the remote.
+  WARN_NOT_OK(EndRemoteSession(), "Unable to close tablet copy session");
 }
 
-Status RemoteBootstrapClient::SetTabletToReplace(const scoped_refptr<TabletMetadata>& meta,
+Status TabletCopyClient::SetTabletToReplace(const scoped_refptr<TabletMetadata>& meta,
                                                  int64_t caller_term) {
   CHECK_EQ(tablet_id_, meta->tablet_id());
   TabletDataState data_state = meta->tablet_data_state();
@@ -128,7 +128,7 @@ Status RemoteBootstrapClient::SetTabletToReplace(const scoped_refptr<TabletMetad
   if (last_logged_term > caller_term) {
     return Status::InvalidArgument(
         Substitute("Leader has term $0 but the last log entry written by the tombstoned replica "
-                   "for tablet $1 has higher term $2. Refusing remote bootstrap from leader",
+                   "for tablet $1 has higher term $2. Refusing tablet copy from leader",
                    caller_term, tablet_id_, last_logged_term));
   }
 
@@ -138,7 +138,7 @@ Status RemoteBootstrapClient::SetTabletToReplace(const scoped_refptr<TabletMetad
                                      fs_manager_->uuid(), &cmeta);
   if (s.IsNotFound()) {
     // The consensus metadata was not written to disk, possibly due to a failed
-    // remote bootstrap.
+    // tablet copy.
     return Status::OK();
   }
   RETURN_NOT_OK(s);
@@ -146,7 +146,7 @@ Status RemoteBootstrapClient::SetTabletToReplace(const scoped_refptr<TabletMetad
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::Start(const HostPort& bootstrap_source_addr,
+Status TabletCopyClient::Start(const HostPort& bootstrap_source_addr,
                                     scoped_refptr<TabletMetadata>* meta) {
   CHECK(!started_);
   start_time_micros_ = GetCurrentTimeMicros();
@@ -154,34 +154,34 @@ Status RemoteBootstrapClient::Start(const HostPort& bootstrap_source_addr,
   Sockaddr addr;
   RETURN_NOT_OK(SockaddrFromHostPort(bootstrap_source_addr, &addr));
   if (addr.IsWildcard()) {
-    return Status::InvalidArgument("Invalid wildcard address to remote bootstrap from",
+    return Status::InvalidArgument("Invalid wildcard address to tablet copy from",
                                    Substitute("$0 (resolved to $1)",
                                               bootstrap_source_addr.host(), addr.host()));
   }
-  LOG_WITH_PREFIX(INFO) << "Beginning remote bootstrap session"
+  LOG_WITH_PREFIX(INFO) << "Beginning tablet copy session"
                         << " from remote peer at address " << bootstrap_source_addr.ToString();
 
-  // Set up an RPC proxy for the RemoteBootstrapService.
-  proxy_.reset(new RemoteBootstrapServiceProxy(messenger_, addr));
+  // Set up an RPC proxy for the TabletCopyService.
+  proxy_.reset(new TabletCopyServiceProxy(messenger_, addr));
 
-  BeginRemoteBootstrapSessionRequestPB req;
+  BeginTabletCopySessionRequestPB req;
   req.set_requestor_uuid(fs_manager_->uuid());
   req.set_tablet_id(tablet_id_);
 
   rpc::RpcController controller;
   controller.set_timeout(MonoDelta::FromMilliseconds(
-      FLAGS_remote_bootstrap_begin_session_timeout_ms));
+      FLAGS_tablet_copy_begin_session_timeout_ms));
 
-  // Begin the remote bootstrap session with the remote peer.
-  BeginRemoteBootstrapSessionResponsePB resp;
-  RETURN_NOT_OK_UNWIND_PREPEND(proxy_->BeginRemoteBootstrapSession(req, &resp, &controller),
+  // Begin the tablet copy session with the remote peer.
+  BeginTabletCopySessionResponsePB resp;
+  RETURN_NOT_OK_UNWIND_PREPEND(proxy_->BeginTabletCopySession(req, &resp, &controller),
                                controller,
-                               "Unable to begin remote bootstrap session");
+                               "Unable to begin tablet copy session");
   string bootstrap_peer_uuid = resp.has_responder_uuid()
       ? resp.responder_uuid() : "(unknown uuid)";
   if (resp.superblock().tablet_data_state() != tablet::TABLET_DATA_READY) {
     Status s = Status::IllegalState("Remote peer (" + bootstrap_peer_uuid + ")" +
-                                    " is currently remotely bootstrapping itself!",
+                                    " is currently copying itself!",
                                     resp.superblock().ShortDebugString());
     LOG_WITH_PREFIX(WARNING) << s.ToString();
     return s;
@@ -209,7 +209,7 @@ Status RemoteBootstrapClient::Start(const HostPort& bootstrap_source_addr,
       return Status::InvalidArgument(
           Substitute("Tablet $0: Bootstrap source has term $1 but "
                      "tombstoned replica has last-logged opid with higher term $2. "
-                      "Refusing remote bootstrap from source peer $3",
+                      "Refusing tablet copy from source peer $3",
                       tablet_id_,
                       remote_committed_cstate_->current_term(),
                       last_logged_term,
@@ -218,7 +218,7 @@ Status RemoteBootstrapClient::Start(const HostPort& bootstrap_source_addr,
 
     // This will flush to disk, but we set the data state to COPYING above.
     RETURN_NOT_OK_PREPEND(meta_->ReplaceSuperBlock(*superblock_),
-                          "Remote bootstrap unable to replace superblock on tablet " +
+                          "Tablet Copy unable to replace superblock on tablet " +
                           tablet_id_);
   } else {
 
@@ -246,7 +246,7 @@ Status RemoteBootstrapClient::Start(const HostPort& bootstrap_source_addr,
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::FetchAll(TabletStatusListener* status_listener) {
+Status TabletCopyClient::FetchAll(TabletStatusListener* status_listener) {
   CHECK(started_);
   status_listener_ = status_listener;
 
@@ -257,7 +257,7 @@ Status RemoteBootstrapClient::FetchAll(TabletStatusListener* status_listener) {
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::Finish() {
+Status TabletCopyClient::Finish() {
   CHECK(meta_);
   CHECK(started_);
   CHECK(downloaded_wal_);
@@ -268,12 +268,12 @@ Status RemoteBootstrapClient::Finish() {
   // Replace tablet metadata superblock. This will set the tablet metadata state
   // to TABLET_DATA_READY, since we checked above that the response
   // superblock is in a valid state to bootstrap from.
-  LOG_WITH_PREFIX(INFO) << "Remote bootstrap complete. Replacing tablet superblock.";
+  LOG_WITH_PREFIX(INFO) << "Tablet Copy complete. Replacing tablet superblock.";
   UpdateStatusMessage("Replacing tablet superblock");
   new_superblock_->set_tablet_data_state(tablet::TABLET_DATA_READY);
   RETURN_NOT_OK(meta_->ReplaceSuperBlock(*new_superblock_));
 
-  if (FLAGS_remote_bootstrap_save_downloaded_metadata) {
+  if (FLAGS_tablet_copy_save_downloaded_metadata) {
     string meta_path = fs_manager_->GetTabletMetadataPath(tablet_id_);
     string meta_copy_path = Substitute("$0.copy.$1.tmp", meta_path, start_time_micros_);
     RETURN_NOT_OK_PREPEND(CopyFile(Env::Default(), meta_path, meta_copy_path,
@@ -285,20 +285,20 @@ Status RemoteBootstrapClient::Finish() {
 }
 
 // Decode the remote error into a human-readable Status object.
-Status RemoteBootstrapClient::ExtractRemoteError(const rpc::ErrorStatusPB& remote_error) {
-  if (PREDICT_TRUE(remote_error.HasExtension(RemoteBootstrapErrorPB::remote_bootstrap_error_ext))) {
-    const RemoteBootstrapErrorPB& error =
-        remote_error.GetExtension(RemoteBootstrapErrorPB::remote_bootstrap_error_ext);
+Status TabletCopyClient::ExtractRemoteError(const rpc::ErrorStatusPB& remote_error) {
+  if (PREDICT_TRUE(remote_error.HasExtension(TabletCopyErrorPB::tablet_copy_error_ext))) {
+    const TabletCopyErrorPB& error =
+        remote_error.GetExtension(TabletCopyErrorPB::tablet_copy_error_ext);
     return StatusFromPB(error.status()).CloneAndPrepend("Received error code " +
-              RemoteBootstrapErrorPB::Code_Name(error.code()) + " from remote service");
+              TabletCopyErrorPB::Code_Name(error.code()) + " from remote service");
   } else {
-    return Status::InvalidArgument("Unable to decode remote bootstrap RPC error message",
+    return Status::InvalidArgument("Unable to decode tablet copy RPC error message",
                                    remote_error.ShortDebugString());
   }
 }
 
 // Enhance a RemoteError Status message with additional details from the remote.
-Status RemoteBootstrapClient::UnwindRemoteError(const Status& status,
+Status TabletCopyClient::UnwindRemoteError(const Status& status,
                                                 const rpc::RpcController& controller) {
   if (!status.IsRemoteError()) {
     return status;
@@ -307,33 +307,33 @@ Status RemoteBootstrapClient::UnwindRemoteError(const Status& status,
   return status.CloneAndAppend(extension_status.ToString());
 }
 
-void RemoteBootstrapClient::UpdateStatusMessage(const string& message) {
+void TabletCopyClient::UpdateStatusMessage(const string& message) {
   if (status_listener_ != nullptr) {
-    status_listener_->StatusMessage("RemoteBootstrap: " + message);
+    status_listener_->StatusMessage("TabletCopy: " + message);
   }
 }
 
-Status RemoteBootstrapClient::EndRemoteSession() {
+Status TabletCopyClient::EndRemoteSession() {
   if (!started_) {
     return Status::OK();
   }
 
   rpc::RpcController controller;
   controller.set_timeout(MonoDelta::FromMilliseconds(
-        FLAGS_remote_bootstrap_begin_session_timeout_ms));
+        FLAGS_tablet_copy_begin_session_timeout_ms));
 
-  EndRemoteBootstrapSessionRequestPB req;
+  EndTabletCopySessionRequestPB req;
   req.set_session_id(session_id_);
   req.set_is_success(true);
-  EndRemoteBootstrapSessionResponsePB resp;
-  RETURN_NOT_OK_UNWIND_PREPEND(proxy_->EndRemoteBootstrapSession(req, &resp, &controller),
+  EndTabletCopySessionResponsePB resp;
+  RETURN_NOT_OK_UNWIND_PREPEND(proxy_->EndTabletCopySession(req, &resp, &controller),
                                controller,
-                               "Failure ending remote bootstrap session");
+                               "Failure ending tablet copy session");
 
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::DownloadWALs() {
+Status TabletCopyClient::DownloadWALs() {
   CHECK(started_);
 
   // Delete and recreate WAL dir if it already exists, to ensure stray files are
@@ -360,7 +360,7 @@ Status RemoteBootstrapClient::DownloadWALs() {
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::DownloadBlocks() {
+Status TabletCopyClient::DownloadBlocks() {
   CHECK(started_);
 
   // Count up the total number of blocks to download.
@@ -415,7 +415,7 @@ Status RemoteBootstrapClient::DownloadBlocks() {
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::DownloadWAL(uint64_t wal_segment_seqno) {
+Status TabletCopyClient::DownloadWAL(uint64_t wal_segment_seqno) {
   VLOG_WITH_PREFIX(1) << "Downloading WAL segment with seqno " << wal_segment_seqno;
   DataIdPB data_id;
   data_id.set_type(DataIdPB::LOG_SEGMENT);
@@ -433,7 +433,7 @@ Status RemoteBootstrapClient::DownloadWAL(uint64_t wal_segment_seqno) {
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::WriteConsensusMetadata() {
+Status TabletCopyClient::WriteConsensusMetadata() {
   // If we didn't find a previous consensus meta file, create one.
   if (!cmeta_) {
     gscoped_ptr<ConsensusMetadata> cmeta;
@@ -444,11 +444,11 @@ Status RemoteBootstrapClient::WriteConsensusMetadata() {
   }
 
   // Otherwise, update the consensus metadata to reflect the config and term
-  // sent by the remote bootstrap source.
+  // sent by the tablet copy source.
   cmeta_->MergeCommittedConsensusStatePB(*remote_committed_cstate_);
   RETURN_NOT_OK(cmeta_->Flush());
 
-  if (FLAGS_remote_bootstrap_save_downloaded_metadata) {
+  if (FLAGS_tablet_copy_save_downloaded_metadata) {
     string cmeta_path = fs_manager_->GetConsensusMetadataPath(tablet_id_);
     string cmeta_copy_path = Substitute("$0.copy.$1.tmp", cmeta_path, start_time_micros_);
     RETURN_NOT_OK_PREPEND(CopyFile(Env::Default(), cmeta_path, cmeta_copy_path,
@@ -459,7 +459,7 @@ Status RemoteBootstrapClient::WriteConsensusMetadata() {
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::DownloadAndRewriteBlock(BlockIdPB* block_id,
+Status TabletCopyClient::DownloadAndRewriteBlock(BlockIdPB* block_id,
                                                       int* block_count, int num_blocks) {
   BlockId old_block_id(BlockId::FromPB(*block_id));
   UpdateStatusMessage(Substitute("Downloading block $0 ($1/$2)",
@@ -474,7 +474,7 @@ Status RemoteBootstrapClient::DownloadAndRewriteBlock(BlockIdPB* block_id,
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::DownloadBlock(const BlockId& old_block_id,
+Status TabletCopyClient::DownloadBlock(const BlockId& old_block_id,
                                             BlockId* new_block_id) {
   VLOG_WITH_PREFIX(1) << "Downloading block with block_id " << old_block_id.ToString();
 
@@ -495,7 +495,7 @@ Status RemoteBootstrapClient::DownloadBlock(const BlockId& old_block_id,
 }
 
 template<class Appendable>
-Status RemoteBootstrapClient::DownloadFile(const DataIdPB& data_id,
+Status TabletCopyClient::DownloadFile(const DataIdPB& data_id,
                                            Appendable* appendable) {
   uint64_t offset = 0;
   rpc::RpcController controller;
@@ -508,7 +508,7 @@ Status RemoteBootstrapClient::DownloadFile(const DataIdPB& data_id,
     req.set_session_id(session_id_);
     req.mutable_data_id()->CopyFrom(data_id);
     req.set_offset(offset);
-    req.set_max_length(FLAGS_remote_bootstrap_transfer_chunk_size_bytes);
+    req.set_max_length(FLAGS_tablet_copy_transfer_chunk_size_bytes);
 
     FetchDataResponsePB resp;
     RETURN_NOT_OK_UNWIND_PREPEND(proxy_->FetchData(req, &resp, &controller),
@@ -522,10 +522,10 @@ 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)) {
+    if (PREDICT_FALSE(FLAGS_tablet_copy_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));
+          FLAGS_tablet_copy_dowload_file_inject_latency_ms;
+      SleepFor(MonoDelta::FromMilliseconds(FLAGS_tablet_copy_dowload_file_inject_latency_ms));
     }
 
     if (offset + resp.chunk().data().size() == resp.chunk().total_data_length()) {
@@ -537,7 +537,7 @@ Status RemoteBootstrapClient::DownloadFile(const DataIdPB& data_id,
   return Status::OK();
 }
 
-Status RemoteBootstrapClient::VerifyData(uint64_t offset, const DataChunkPB& chunk) {
+Status TabletCopyClient::VerifyData(uint64_t offset, const DataChunkPB& chunk) {
   // Verify the offset is what we expected.
   if (offset != chunk.offset()) {
     return Status::InvalidArgument("Offset did not match what was asked for",
@@ -554,8 +554,8 @@ Status RemoteBootstrapClient::VerifyData(uint64_t offset, const DataChunkPB& chu
   return Status::OK();
 }
 
-string RemoteBootstrapClient::LogPrefix() {
-  return Substitute("T $0 P $1: Remote bootstrap client: ",
+string TabletCopyClient::LogPrefix() {
+  return Substitute("T $0 P $1: Tablet Copy client: ",
                     tablet_id_, fs_manager_->uuid());
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_client.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_client.h b/src/kudu/tserver/remote_bootstrap_client.h
index e461ec9..e475ef1 100644
--- a/src/kudu/tserver/remote_bootstrap_client.h
+++ b/src/kudu/tserver/remote_bootstrap_client.h
@@ -14,8 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_TSERVER_REMOTE_BOOTSTRAP_CLIENT_H
-#define KUDU_TSERVER_REMOTE_BOOTSTRAP_CLIENT_H
+#ifndef KUDU_TSERVER_TABLET_COPY_CLIENT_H
+#define KUDU_TSERVER_TABLET_COPY_CLIENT_H
 
 #include <string>
 #include <memory>
@@ -58,25 +58,25 @@ class TabletSuperBlockPB;
 namespace tserver {
 class DataIdPB;
 class DataChunkPB;
-class RemoteBootstrapServiceProxy;
+class TabletCopyServiceProxy;
 
-// Client class for using remote bootstrap to copy a tablet from another host.
+// Client class for using tablet copy to copy a tablet from another host.
 // This class is not thread-safe.
 //
 // TODO:
 // * Parallelize download of blocks and WAL segments.
 //
-class RemoteBootstrapClient {
+class TabletCopyClient {
  public:
 
-  // Construct the remote bootstrap client.
+  // Construct the tablet copy client.
   // 'fs_manager' and 'messenger' must remain valid until this object is destroyed.
-  RemoteBootstrapClient(std::string tablet_id, FsManager* fs_manager,
+  TabletCopyClient(std::string tablet_id, FsManager* fs_manager,
                         std::shared_ptr<rpc::Messenger> messenger);
 
   // Attempt to clean up resources on the remote end by sending an
-  // EndRemoteBootstrapSession() RPC
-  ~RemoteBootstrapClient();
+  // EndTabletCopySession() RPC
+  ~TabletCopyClient();
 
   // Pass in the existing metadata for a tombstoned tablet, which will be
   // replaced if validation checks pass in Start().
@@ -89,15 +89,15 @@ class RemoteBootstrapClient {
   Status SetTabletToReplace(const scoped_refptr<tablet::TabletMetadata>& meta,
                             int64_t caller_term);
 
-  // Start up a remote bootstrap session to bootstrap from the specified
-  // bootstrap peer. Place a new superblock indicating that remote bootstrap is
+  // Start up a tablet copy session to bootstrap from the specified
+  // bootstrap peer. Place a new superblock indicating that tablet copy is
   // in progress. If the 'metadata' pointer is passed as NULL, it is ignored,
   // otherwise the TabletMetadata object resulting from the initial remote
   // bootstrap response is returned.
   Status Start(const HostPort& bootstrap_source_addr,
                scoped_refptr<tablet::TabletMetadata>* metadata);
 
-  // Runs a "full" remote bootstrap, copying the physical layout of a tablet
+  // Runs a "full" tablet copy, copying the physical layout of a tablet
   // from the leader of the specified consensus configuration.
   Status FetchAll(tablet::TabletStatusListener* status_listener);
 
@@ -106,23 +106,23 @@ class RemoteBootstrapClient {
   Status Finish();
 
  private:
-  FRIEND_TEST(RemoteBootstrapClientTest, TestBeginEndSession);
-  FRIEND_TEST(RemoteBootstrapClientTest, TestDownloadBlock);
-  FRIEND_TEST(RemoteBootstrapClientTest, TestVerifyData);
-  FRIEND_TEST(RemoteBootstrapClientTest, TestDownloadWalSegment);
-  FRIEND_TEST(RemoteBootstrapClientTest, TestDownloadAllBlocks);
+  FRIEND_TEST(TabletCopyClientTest, TestBeginEndSession);
+  FRIEND_TEST(TabletCopyClientTest, TestDownloadBlock);
+  FRIEND_TEST(TabletCopyClientTest, TestVerifyData);
+  FRIEND_TEST(TabletCopyClientTest, TestDownloadWalSegment);
+  FRIEND_TEST(TabletCopyClientTest, TestDownloadAllBlocks);
 
   // Extract the embedded Status message from the given ErrorStatusPB.
-  // The given ErrorStatusPB must extend RemoteBootstrapErrorPB.
+  // The given ErrorStatusPB must extend TabletCopyErrorPB.
   static Status ExtractRemoteError(const rpc::ErrorStatusPB& remote_error);
 
   static Status UnwindRemoteError(const Status& status, const rpc::RpcController& controller);
 
   // Update the bootstrap StatusListener with a message.
-  // The string "RemoteBootstrap: " will be prepended to each message.
+  // The string "TabletCopy: " will be prepended to each message.
   void UpdateStatusMessage(const std::string& message);
 
-  // End the remote bootstrap session.
+  // End the tablet copy session.
   Status EndRemoteSession();
 
   // Download all WAL files sequentially.
@@ -134,7 +134,7 @@ class RemoteBootstrapClient {
   Status DownloadWAL(uint64_t wal_segment_seqno);
 
   // Write out the Consensus Metadata file based on the ConsensusStatePB
-  // downloaded as part of initiating the remote bootstrap session.
+  // downloaded as part of initiating the tablet copy session.
   Status WriteConsensusMetadata();
 
   // Download all blocks belonging to a tablet sequentially.
@@ -177,7 +177,7 @@ class RemoteBootstrapClient {
   FsManager* const fs_manager_;
   const std::shared_ptr<rpc::Messenger> messenger_;
 
-  // State flags that enforce the progress of remote bootstrap.
+  // State flags that enforce the progress of tablet copy.
   bool started_;            // Session started.
   bool downloaded_wal_;     // WAL segments downloaded.
   bool downloaded_blocks_;  // Data blocks downloaded.
@@ -193,7 +193,7 @@ class RemoteBootstrapClient {
   gscoped_ptr<consensus::ConsensusMetadata> cmeta_;
 
   tablet::TabletStatusListener* status_listener_;
-  std::shared_ptr<RemoteBootstrapServiceProxy> proxy_;
+  std::shared_ptr<TabletCopyServiceProxy> proxy_;
   std::string session_id_;
   uint64_t session_idle_timeout_millis_;
   gscoped_ptr<tablet::TabletSuperBlockPB> superblock_;
@@ -202,9 +202,9 @@ class RemoteBootstrapClient {
   std::vector<uint64_t> wal_seqnos_;
   int64_t start_time_micros_;
 
-  DISALLOW_COPY_AND_ASSIGN(RemoteBootstrapClient);
+  DISALLOW_COPY_AND_ASSIGN(TabletCopyClient);
 };
 
 } // namespace tserver
 } // namespace kudu
-#endif /* KUDU_TSERVER_REMOTE_BOOTSTRAP_CLIENT_H */
+#endif /* KUDU_TSERVER_TABLET_COPY_CLIENT_H */

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_service-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_service-test.cc b/src/kudu/tserver/remote_bootstrap_service-test.cc
index 23fd0be..1b5baf2 100644
--- a/src/kudu/tserver/remote_bootstrap_service-test.cc
+++ b/src/kudu/tserver/remote_bootstrap_service-test.cc
@@ -40,8 +40,8 @@
 #define ASSERT_REMOTE_ERROR(status, err, code, str) \
     ASSERT_NO_FATAL_FAILURE(AssertRemoteError(status, err, code, str))
 
-DECLARE_uint64(remote_bootstrap_idle_timeout_ms);
-DECLARE_uint64(remote_bootstrap_timeout_poll_period_ms);
+DECLARE_uint64(tablet_copy_idle_timeout_ms);
+DECLARE_uint64(tablet_copy_timeout_poll_period_ms);
 
 namespace kudu {
 namespace tserver {
@@ -56,39 +56,39 @@ using rpc::RpcController;
 using std::thread;
 using std::vector;
 
-class RemoteBootstrapServiceTest : public RemoteBootstrapTest {
+class TabletCopyServiceTest : public TabletCopyTest {
  public:
-  RemoteBootstrapServiceTest() {
+  TabletCopyServiceTest() {
     // Poll for session expiration every 10 ms for the session timeout test.
-    FLAGS_remote_bootstrap_timeout_poll_period_ms = 10;
+    FLAGS_tablet_copy_timeout_poll_period_ms = 10;
   }
 
  protected:
   void SetUp() OVERRIDE {
-    RemoteBootstrapTest::SetUp();
-    remote_bootstrap_proxy_.reset(
-        new RemoteBootstrapServiceProxy(client_messenger_, mini_server_->bound_rpc_addr()));
+    TabletCopyTest::SetUp();
+    tablet_copy_proxy_.reset(
+        new TabletCopyServiceProxy(client_messenger_, mini_server_->bound_rpc_addr()));
   }
 
-  Status DoBeginRemoteBootstrapSession(const string& tablet_id,
+  Status DoBeginTabletCopySession(const string& tablet_id,
                                        const string& requestor_uuid,
-                                       BeginRemoteBootstrapSessionResponsePB* resp,
+                                       BeginTabletCopySessionResponsePB* resp,
                                        RpcController* controller) {
     controller->set_timeout(MonoDelta::FromSeconds(1.0));
-    BeginRemoteBootstrapSessionRequestPB req;
+    BeginTabletCopySessionRequestPB req;
     req.set_tablet_id(tablet_id);
     req.set_requestor_uuid(requestor_uuid);
     return UnwindRemoteError(
-        remote_bootstrap_proxy_->BeginRemoteBootstrapSession(req, resp, controller), controller);
+        tablet_copy_proxy_->BeginTabletCopySession(req, resp, controller), controller);
   }
 
-  Status DoBeginValidRemoteBootstrapSession(string* session_id,
+  Status DoBeginValidTabletCopySession(string* session_id,
                                             tablet::TabletSuperBlockPB* superblock = nullptr,
                                             uint64_t* idle_timeout_millis = nullptr,
                                             vector<uint64_t>* sequence_numbers = nullptr) {
-    BeginRemoteBootstrapSessionResponsePB resp;
+    BeginTabletCopySessionResponsePB resp;
     RpcController controller;
-    RETURN_NOT_OK(DoBeginRemoteBootstrapSession(GetTabletId(), GetLocalUUID(), &resp, &controller));
+    RETURN_NOT_OK(DoBeginTabletCopySession(GetTabletId(), GetLocalUUID(), &resp, &controller));
     *session_id = resp.session_id();
     if (superblock) {
       *superblock = resp.superblock();
@@ -103,13 +103,13 @@ class RemoteBootstrapServiceTest : public RemoteBootstrapTest {
   }
 
   Status DoCheckSessionActive(const string& session_id,
-                              CheckRemoteBootstrapSessionActiveResponsePB* resp,
+                              CheckTabletCopySessionActiveResponsePB* resp,
                               RpcController* controller) {
     controller->set_timeout(MonoDelta::FromSeconds(1.0));
-    CheckRemoteBootstrapSessionActiveRequestPB req;
+    CheckTabletCopySessionActiveRequestPB req;
     req.set_session_id(session_id);
     return UnwindRemoteError(
-        remote_bootstrap_proxy_->CheckSessionActive(req, resp, controller), controller);
+        tablet_copy_proxy_->CheckSessionActive(req, resp, controller), controller);
   }
 
   Status DoFetchData(const string& session_id, const DataIdPB& data_id,
@@ -127,28 +127,28 @@ class RemoteBootstrapServiceTest : public RemoteBootstrapTest {
       req.set_max_length(*max_length);
     }
     return UnwindRemoteError(
-        remote_bootstrap_proxy_->FetchData(req, resp, controller), controller);
+        tablet_copy_proxy_->FetchData(req, resp, controller), controller);
   }
 
-  Status DoEndRemoteBootstrapSession(const string& session_id, bool is_success,
+  Status DoEndTabletCopySession(const string& session_id, bool is_success,
                                      const Status* error_msg,
-                                     EndRemoteBootstrapSessionResponsePB* resp,
+                                     EndTabletCopySessionResponsePB* resp,
                                      RpcController* controller) {
     controller->set_timeout(MonoDelta::FromSeconds(1.0));
-    EndRemoteBootstrapSessionRequestPB req;
+    EndTabletCopySessionRequestPB req;
     req.set_session_id(session_id);
     req.set_is_success(is_success);
     if (error_msg) {
       StatusToPB(*error_msg, req.mutable_error());
     }
     return UnwindRemoteError(
-        remote_bootstrap_proxy_->EndRemoteBootstrapSession(req, resp, controller), controller);
+        tablet_copy_proxy_->EndTabletCopySession(req, resp, controller), controller);
   }
 
   // Decode the remote error into a Status object.
   Status ExtractRemoteError(const ErrorStatusPB* remote_error) {
-    const RemoteBootstrapErrorPB& error =
-        remote_error->GetExtension(RemoteBootstrapErrorPB::remote_bootstrap_error_ext);
+    const TabletCopyErrorPB& error =
+        remote_error->GetExtension(TabletCopyErrorPB::tablet_copy_error_ext);
     return StatusFromPB(error.status());
   }
 
@@ -163,15 +163,15 @@ class RemoteBootstrapServiceTest : public RemoteBootstrapTest {
   }
 
   void AssertRemoteError(Status status, const ErrorStatusPB* remote_error,
-                         const RemoteBootstrapErrorPB::Code app_code,
+                         const TabletCopyErrorPB::Code app_code,
                          const string& status_code_string) {
     ASSERT_TRUE(status.IsRemoteError()) << "Unexpected status code: " << status.ToString()
                                         << ", app code: "
-                                        << RemoteBootstrapErrorPB::Code_Name(app_code)
+                                        << TabletCopyErrorPB::Code_Name(app_code)
                                         << ", status code string: " << status_code_string;
     const Status app_status = ExtractRemoteError(remote_error);
-    const RemoteBootstrapErrorPB& error =
-        remote_error->GetExtension(RemoteBootstrapErrorPB::remote_bootstrap_error_ext);
+    const TabletCopyErrorPB& error =
+        remote_error->GetExtension(TabletCopyErrorPB::tablet_copy_error_ext);
     ASSERT_EQ(app_code, error.code()) << error.ShortDebugString();
     ASSERT_EQ(status_code_string, app_status.CodeAsString()) << app_status.ToString();
     LOG(INFO) << app_status.ToString();
@@ -185,52 +185,52 @@ class RemoteBootstrapServiceTest : public RemoteBootstrapTest {
     return data_id;
   }
 
-  gscoped_ptr<RemoteBootstrapServiceProxy> remote_bootstrap_proxy_;
+  gscoped_ptr<TabletCopyServiceProxy> tablet_copy_proxy_;
 };
 
-// Test beginning and ending a remote bootstrap session.
-TEST_F(RemoteBootstrapServiceTest, TestSimpleBeginEndSession) {
+// Test beginning and ending a tablet copy session.
+TEST_F(TabletCopyServiceTest, TestSimpleBeginEndSession) {
   string session_id;
   tablet::TabletSuperBlockPB superblock;
   uint64_t idle_timeout_millis;
   vector<uint64_t> segment_seqnos;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id,
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id,
                                                &superblock,
                                                &idle_timeout_millis,
                                                &segment_seqnos));
   // Basic validation of returned params.
   ASSERT_FALSE(session_id.empty());
-  ASSERT_EQ(FLAGS_remote_bootstrap_idle_timeout_ms, idle_timeout_millis);
+  ASSERT_EQ(FLAGS_tablet_copy_idle_timeout_ms, idle_timeout_millis);
   ASSERT_TRUE(superblock.IsInitialized());
   // We should have number of segments = number of rolls + 1 (due to the active segment).
   ASSERT_EQ(kNumLogRolls + 1, segment_seqnos.size());
 
-  EndRemoteBootstrapSessionResponsePB resp;
+  EndTabletCopySessionResponsePB resp;
   RpcController controller;
-  ASSERT_OK(DoEndRemoteBootstrapSession(session_id, true, nullptr, &resp, &controller));
+  ASSERT_OK(DoEndTabletCopySession(session_id, true, nullptr, &resp, &controller));
 }
 
 // Test starting two sessions. The current implementation will silently only create one.
-TEST_F(RemoteBootstrapServiceTest, TestBeginTwice) {
+TEST_F(TabletCopyServiceTest, TestBeginTwice) {
   // Second time through should silently succeed.
   for (int i = 0; i < 2; i++) {
     string session_id;
-    ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id));
+    ASSERT_OK(DoBeginValidTabletCopySession(&session_id));
     ASSERT_FALSE(session_id.empty());
   }
 }
 
 // Regression test for KUDU-1436: race conditions if multiple requests
-// to begin the same remote bootstrap session arrive at more or less the
+// to begin the same tablet copy session arrive at more or less the
 // same time.
-TEST_F(RemoteBootstrapServiceTest, TestBeginConcurrently) {
+TEST_F(TabletCopyServiceTest, TestBeginConcurrently) {
   const int kNumThreads = 5;
   vector<thread> threads;
   vector<tablet::TabletSuperBlockPB> sblocks(kNumThreads);
   for (int i = 0 ; i < kNumThreads; i++) {
     threads.emplace_back([this, &sblocks, i]{
         string session_id;
-        CHECK_OK(DoBeginValidRemoteBootstrapSession(&session_id, &sblocks[i]));
+        CHECK_OK(DoBeginValidTabletCopySession(&session_id, &sblocks[i]));
         CHECK(!session_id.empty());
       });
   }
@@ -244,7 +244,7 @@ TEST_F(RemoteBootstrapServiceTest, TestBeginConcurrently) {
 }
 
 // Test bad session id error condition.
-TEST_F(RemoteBootstrapServiceTest, TestInvalidSessionId) {
+TEST_F(TabletCopyServiceTest, TestInvalidSessionId) {
   vector<string> bad_session_ids;
   bad_session_ids.push_back("hodor");
   bad_session_ids.push_back(GetLocalUUID());
@@ -257,34 +257,34 @@ TEST_F(RemoteBootstrapServiceTest, TestInvalidSessionId) {
     data_id.set_type(DataIdPB::BLOCK);
     data_id.mutable_block_id()->set_id(1);
     Status status = DoFetchData(session_id, data_id, nullptr, nullptr, &resp, &controller);
-    ASSERT_REMOTE_ERROR(status, controller.error_response(), RemoteBootstrapErrorPB::NO_SESSION,
+    ASSERT_REMOTE_ERROR(status, controller.error_response(), TabletCopyErrorPB::NO_SESSION,
                         Status::NotFound("").CodeAsString());
   }
 
   // End a non-existent session.
   for (const string& session_id : bad_session_ids) {
-    EndRemoteBootstrapSessionResponsePB resp;
+    EndTabletCopySessionResponsePB resp;
     RpcController controller;
-    Status status = DoEndRemoteBootstrapSession(session_id, true, nullptr, &resp, &controller);
-    ASSERT_REMOTE_ERROR(status, controller.error_response(), RemoteBootstrapErrorPB::NO_SESSION,
+    Status status = DoEndTabletCopySession(session_id, true, nullptr, &resp, &controller);
+    ASSERT_REMOTE_ERROR(status, controller.error_response(), TabletCopyErrorPB::NO_SESSION,
                         Status::NotFound("").CodeAsString());
   }
 }
 
 // Test bad tablet id error condition.
-TEST_F(RemoteBootstrapServiceTest, TestInvalidTabletId) {
-  BeginRemoteBootstrapSessionResponsePB resp;
+TEST_F(TabletCopyServiceTest, TestInvalidTabletId) {
+  BeginTabletCopySessionResponsePB resp;
   RpcController controller;
   Status status =
-      DoBeginRemoteBootstrapSession("some-unknown-tablet", GetLocalUUID(), &resp, &controller);
-  ASSERT_REMOTE_ERROR(status, controller.error_response(), RemoteBootstrapErrorPB::TABLET_NOT_FOUND,
+      DoBeginTabletCopySession("some-unknown-tablet", GetLocalUUID(), &resp, &controller);
+  ASSERT_REMOTE_ERROR(status, controller.error_response(), TabletCopyErrorPB::TABLET_NOT_FOUND,
                       Status::NotFound("").CodeAsString());
 }
 
 // Test DataIdPB validation.
-TEST_F(RemoteBootstrapServiceTest, TestInvalidBlockOrOpId) {
+TEST_F(TabletCopyServiceTest, TestInvalidBlockOrOpId) {
   string session_id;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id));
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id));
 
   // Invalid BlockId.
   {
@@ -295,7 +295,7 @@ TEST_F(RemoteBootstrapServiceTest, TestInvalidBlockOrOpId) {
     data_id.mutable_block_id()->set_id(1);
     Status status = DoFetchData(session_id, data_id, nullptr, nullptr, &resp, &controller);
     ASSERT_REMOTE_ERROR(status, controller.error_response(),
-                        RemoteBootstrapErrorPB::BLOCK_NOT_FOUND,
+                        TabletCopyErrorPB::BLOCK_NOT_FOUND,
                         Status::NotFound("").CodeAsString());
   }
 
@@ -308,7 +308,7 @@ TEST_F(RemoteBootstrapServiceTest, TestInvalidBlockOrOpId) {
     data_id.set_wal_segment_seqno(31337);
     Status status = DoFetchData(session_id, data_id, nullptr, nullptr, &resp, &controller);
     ASSERT_REMOTE_ERROR(status, controller.error_response(),
-                        RemoteBootstrapErrorPB::WAL_SEGMENT_NOT_FOUND,
+                        TabletCopyErrorPB::WAL_SEGMENT_NOT_FOUND,
                         Status::NotFound("").CodeAsString());
   }
 
@@ -323,7 +323,7 @@ TEST_F(RemoteBootstrapServiceTest, TestInvalidBlockOrOpId) {
     ASSERT_TRUE(status.IsRemoteError()) << status.ToString();
     ASSERT_STR_CONTAINS(status.ToString(),
                         "Invalid argument: invalid parameter for call "
-                        "kudu.tserver.RemoteBootstrapService.FetchData: "
+                        "kudu.tserver.TabletCopyService.FetchData: "
                         "missing fields: data_id.type");
   }
 
@@ -335,7 +335,7 @@ TEST_F(RemoteBootstrapServiceTest, TestInvalidBlockOrOpId) {
     data_id.set_type(DataIdPB::LOG_SEGMENT);
     Status status = DoFetchData(session_id, data_id, nullptr, nullptr, &resp, &controller);
     ASSERT_REMOTE_ERROR(status, controller.error_response(),
-                        RemoteBootstrapErrorPB::INVALID_REMOTE_BOOTSTRAP_REQUEST,
+                        TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST,
                         Status::InvalidArgument("").CodeAsString());
   }
 
@@ -349,16 +349,16 @@ TEST_F(RemoteBootstrapServiceTest, TestInvalidBlockOrOpId) {
     data_id.set_wal_segment_seqno(0);
     Status status = DoFetchData(session_id, data_id, nullptr, nullptr, &resp, &controller);
     ASSERT_REMOTE_ERROR(status, controller.error_response(),
-                        RemoteBootstrapErrorPB::INVALID_REMOTE_BOOTSTRAP_REQUEST,
+                        TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST,
                         Status::InvalidArgument("").CodeAsString());
   }
 }
 
 // Test invalid file offset error condition.
-TEST_F(RemoteBootstrapServiceTest, TestFetchInvalidBlockOffset) {
+TEST_F(TabletCopyServiceTest, TestFetchInvalidBlockOffset) {
   string session_id;
   tablet::TabletSuperBlockPB superblock;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id, &superblock));
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id, &superblock));
 
   FetchDataResponsePB resp;
   RpcController controller;
@@ -367,15 +367,15 @@ TEST_F(RemoteBootstrapServiceTest, TestFetchInvalidBlockOffset) {
   Status status = DoFetchData(session_id, AsDataTypeId(FirstColumnBlockId(superblock)),
                               &offset, nullptr, &resp, &controller);
   ASSERT_REMOTE_ERROR(status, controller.error_response(),
-                      RemoteBootstrapErrorPB::INVALID_REMOTE_BOOTSTRAP_REQUEST,
+                      TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST,
                       Status::InvalidArgument("").CodeAsString());
 }
 
 // Test that we are able to fetch an entire block.
-TEST_F(RemoteBootstrapServiceTest, TestFetchBlockAtOnce) {
+TEST_F(TabletCopyServiceTest, TestFetchBlockAtOnce) {
   string session_id;
   tablet::TabletSuperBlockPB superblock;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id, &superblock));
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id, &superblock));
 
   // Local.
   BlockId block_id = FirstColumnBlockId(superblock);
@@ -393,10 +393,10 @@ TEST_F(RemoteBootstrapServiceTest, TestFetchBlockAtOnce) {
 }
 
 // Test that we are able to incrementally fetch blocks.
-TEST_F(RemoteBootstrapServiceTest, TestFetchBlockIncrementally) {
+TEST_F(TabletCopyServiceTest, TestFetchBlockIncrementally) {
   string session_id;
   tablet::TabletSuperBlockPB superblock;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id, &superblock));
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id, &superblock));
 
   BlockId block_id = FirstColumnBlockId(superblock);
   Slice local_data;
@@ -421,12 +421,12 @@ TEST_F(RemoteBootstrapServiceTest, TestFetchBlockIncrementally) {
 }
 
 // Test that we are able to fetch log segments.
-TEST_F(RemoteBootstrapServiceTest, TestFetchLog) {
+TEST_F(TabletCopyServiceTest, TestFetchLog) {
   string session_id;
   tablet::TabletSuperBlockPB superblock;
   uint64_t idle_timeout_millis;
   vector<uint64_t> segment_seqnos;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id,
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id,
                                                &superblock,
                                                &idle_timeout_millis,
                                                &segment_seqnos));
@@ -462,18 +462,18 @@ TEST_F(RemoteBootstrapServiceTest, TestFetchLog) {
   AssertDataEqual(slice.data(), slice.size(), resp.chunk());
 }
 
-// Test that the remote bootstrap session timeout works properly.
-TEST_F(RemoteBootstrapServiceTest, TestSessionTimeout) {
+// Test that the tablet copy session timeout works properly.
+TEST_F(TabletCopyServiceTest, TestSessionTimeout) {
   // This flag should be seen by the service due to TSO.
   // We have also reduced the timeout polling frequency in SetUp().
-  FLAGS_remote_bootstrap_idle_timeout_ms = 1; // Expire the session almost immediately.
+  FLAGS_tablet_copy_idle_timeout_ms = 1; // Expire the session almost immediately.
 
   // Start session.
   string session_id;
-  ASSERT_OK(DoBeginValidRemoteBootstrapSession(&session_id));
+  ASSERT_OK(DoBeginValidTabletCopySession(&session_id));
 
   MonoTime start_time = MonoTime::Now(MonoTime::FINE);
-  CheckRemoteBootstrapSessionActiveResponsePB resp;
+  CheckTabletCopySessionActiveResponsePB resp;
 
   do {
     RpcController controller;
@@ -484,7 +484,7 @@ TEST_F(RemoteBootstrapServiceTest, TestSessionTimeout) {
     SleepFor(MonoDelta::FromMilliseconds(1)); // 1 ms
   } while (MonoTime::Now(MonoTime::FINE).GetDeltaSince(start_time).ToSeconds() < 10);
 
-  ASSERT_FALSE(resp.session_is_active()) << "Remote bootstrap session did not time out!";
+  ASSERT_FALSE(resp.session_is_active()) << "Tablet Copy session did not time out!";
 }
 
 } // namespace tserver

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_service.cc b/src/kudu/tserver/remote_bootstrap_service.cc
index a6ea959..f5bd2ae 100644
--- a/src/kudu/tserver/remote_bootstrap_service.cc
+++ b/src/kudu/tserver/remote_bootstrap_service.cc
@@ -50,19 +50,19 @@
     } \
   } while (false)
 
-DEFINE_uint64(remote_bootstrap_idle_timeout_ms, 180000,
-              "Amount of time without activity before a remote bootstrap "
+DEFINE_uint64(tablet_copy_idle_timeout_ms, 180000,
+              "Amount of time without activity before a tablet copy "
               "session will expire, in millis");
-TAG_FLAG(remote_bootstrap_idle_timeout_ms, hidden);
+TAG_FLAG(tablet_copy_idle_timeout_ms, hidden);
 
-DEFINE_uint64(remote_bootstrap_timeout_poll_period_ms, 10000,
-              "How often the remote_bootstrap service polls for expired "
-              "remote bootstrap sessions, in millis");
-TAG_FLAG(remote_bootstrap_timeout_poll_period_ms, hidden);
+DEFINE_uint64(tablet_copy_timeout_poll_period_ms, 10000,
+              "How often the tablet_copy service polls for expired "
+              "tablet copy sessions, in millis");
+TAG_FLAG(tablet_copy_timeout_poll_period_ms, hidden);
 
 DEFINE_double(fault_crash_on_handle_rb_fetch_data, 0.0,
               "Fraction of the time when the tablet will crash while "
-              "servicing a RemoteBootstrapService FetchData() RPC call. "
+              "servicing a TabletCopyService FetchData() RPC call. "
               "(For testing only!)");
 TAG_FLAG(fault_crash_on_handle_rb_fetch_data, unsafe);
 
@@ -74,36 +74,36 @@ using strings::Substitute;
 using tablet::TabletPeer;
 
 static void SetupErrorAndRespond(rpc::RpcContext* context,
-                                 RemoteBootstrapErrorPB::Code code,
+                                 TabletCopyErrorPB::Code code,
                                  const string& message,
                                  const Status& s) {
-  LOG(WARNING) << "Error handling RemoteBootstrapService RPC request from "
+  LOG(WARNING) << "Error handling TabletCopyService RPC request from "
                << context->requestor_string() << ": "
                << s.ToString();
-  RemoteBootstrapErrorPB error;
+  TabletCopyErrorPB error;
   StatusToPB(s, error.mutable_status());
   error.set_code(code);
-  context->RespondApplicationError(RemoteBootstrapErrorPB::remote_bootstrap_error_ext.number(),
+  context->RespondApplicationError(TabletCopyErrorPB::tablet_copy_error_ext.number(),
                                    message, error);
 }
 
-RemoteBootstrapServiceImpl::RemoteBootstrapServiceImpl(
+TabletCopyServiceImpl::TabletCopyServiceImpl(
     FsManager* fs_manager,
     TabletPeerLookupIf* tablet_peer_lookup,
     const scoped_refptr<MetricEntity>& metric_entity,
     const scoped_refptr<rpc::ResultTracker>& result_tracker)
-    : RemoteBootstrapServiceIf(metric_entity, result_tracker),
+    : TabletCopyServiceIf(metric_entity, result_tracker),
       fs_manager_(CHECK_NOTNULL(fs_manager)),
       tablet_peer_lookup_(CHECK_NOTNULL(tablet_peer_lookup)),
       shutdown_latch_(1) {
   CHECK_OK(Thread::Create("remote-bootstrap", "rb-session-exp",
-                          &RemoteBootstrapServiceImpl::EndExpiredSessions, this,
+                          &TabletCopyServiceImpl::EndExpiredSessions, this,
                           &session_expiration_thread_));
 }
 
-void RemoteBootstrapServiceImpl::BeginRemoteBootstrapSession(
-        const BeginRemoteBootstrapSessionRequestPB* req,
-        BeginRemoteBootstrapSessionResponsePB* resp,
+void TabletCopyServiceImpl::BeginTabletCopySession(
+        const BeginTabletCopySessionRequestPB* req,
+        BeginTabletCopySessionResponsePB* resp,
         rpc::RpcContext* context) {
   const string& requestor_uuid = req->requestor_uuid();
   const string& tablet_id = req->tablet_id();
@@ -114,27 +114,27 @@ void RemoteBootstrapServiceImpl::BeginRemoteBootstrapSession(
 
   scoped_refptr<TabletPeer> tablet_peer;
   RPC_RETURN_NOT_OK(tablet_peer_lookup_->GetTabletPeer(tablet_id, &tablet_peer),
-                    RemoteBootstrapErrorPB::TABLET_NOT_FOUND,
+                    TabletCopyErrorPB::TABLET_NOT_FOUND,
                     Substitute("Unable to find specified tablet: $0", tablet_id));
 
-  scoped_refptr<RemoteBootstrapSession> session;
+  scoped_refptr<TabletCopySession> session;
   {
     MutexLock l(sessions_lock_);
     if (!FindCopy(sessions_, session_id, &session)) {
       LOG(INFO) << Substitute(
-          "Beginning new remote bootstrap session on tablet $0 from peer $1"
+          "Beginning new tablet copy session on tablet $0 from peer $1"
           " at $2: session id = $3",
           tablet_id, requestor_uuid, context->requestor_string(), session_id);
-      session.reset(new RemoteBootstrapSession(tablet_peer, session_id,
+      session.reset(new TabletCopySession(tablet_peer, session_id,
                                                requestor_uuid, fs_manager_));
       RPC_RETURN_NOT_OK(session->Init(),
-                        RemoteBootstrapErrorPB::UNKNOWN_ERROR,
-                        Substitute("Error initializing remote bootstrap session for tablet $0",
+                        TabletCopyErrorPB::UNKNOWN_ERROR,
+                        Substitute("Error initializing tablet copy session for tablet $0",
                                    tablet_id));
       InsertOrDie(&sessions_, session_id, session);
     } else {
       LOG(INFO) << Substitute(
-          "Re-sending initialization info for existing remote bootstrap session on tablet $0"
+          "Re-sending initialization info for existing tablet copy session on tablet $0"
           " from peer $1 at $2: session_id = $3",
           tablet_id, requestor_uuid, context->requestor_string(), session_id);
     }
@@ -143,7 +143,7 @@ void RemoteBootstrapServiceImpl::BeginRemoteBootstrapSession(
 
   resp->set_responder_uuid(fs_manager_->uuid());
   resp->set_session_id(session_id);
-  resp->set_session_idle_timeout_millis(FLAGS_remote_bootstrap_idle_timeout_ms);
+  resp->set_session_idle_timeout_millis(FLAGS_tablet_copy_idle_timeout_ms);
   resp->mutable_superblock()->CopyFrom(session->tablet_superblock());
   resp->mutable_initial_committed_cstate()->CopyFrom(session->initial_committed_cstate());
 
@@ -154,16 +154,16 @@ void RemoteBootstrapServiceImpl::BeginRemoteBootstrapSession(
   context->RespondSuccess();
 }
 
-void RemoteBootstrapServiceImpl::CheckSessionActive(
-        const CheckRemoteBootstrapSessionActiveRequestPB* req,
-        CheckRemoteBootstrapSessionActiveResponsePB* resp,
+void TabletCopyServiceImpl::CheckSessionActive(
+        const CheckTabletCopySessionActiveRequestPB* req,
+        CheckTabletCopySessionActiveResponsePB* resp,
         rpc::RpcContext* context) {
   const string& session_id = req->session_id();
 
-  // Look up and validate remote bootstrap session.
-  scoped_refptr<RemoteBootstrapSession> session;
+  // Look up and validate tablet copy session.
+  scoped_refptr<TabletCopySession> session;
   MutexLock l(sessions_lock_);
-  RemoteBootstrapErrorPB::Code app_error;
+  TabletCopyErrorPB::Code app_error;
   Status status = FindSessionUnlocked(session_id, &app_error, &session);
   if (status.ok()) {
     if (req->keepalive()) {
@@ -172,7 +172,7 @@ void RemoteBootstrapServiceImpl::CheckSessionActive(
     resp->set_session_is_active(true);
     context->RespondSuccess();
     return;
-  } else if (app_error == RemoteBootstrapErrorPB::NO_SESSION) {
+  } else if (app_error == TabletCopyErrorPB::NO_SESSION) {
     resp->set_session_is_active(false);
     context->RespondSuccess();
     return;
@@ -182,16 +182,16 @@ void RemoteBootstrapServiceImpl::CheckSessionActive(
   }
 }
 
-void RemoteBootstrapServiceImpl::FetchData(const FetchDataRequestPB* req,
+void TabletCopyServiceImpl::FetchData(const FetchDataRequestPB* req,
                                            FetchDataResponsePB* resp,
                                            rpc::RpcContext* context) {
   const string& session_id = req->session_id();
 
-  // Look up and validate remote bootstrap session.
-  scoped_refptr<RemoteBootstrapSession> session;
+  // Look up and validate tablet copy session.
+  scoped_refptr<TabletCopySession> session;
   {
     MutexLock l(sessions_lock_);
-    RemoteBootstrapErrorPB::Code app_error;
+    TabletCopyErrorPB::Code app_error;
     RPC_RETURN_NOT_OK(FindSessionUnlocked(session_id, &app_error, &session),
                       app_error, "No such session");
     ResetSessionExpirationUnlocked(session_id);
@@ -203,7 +203,7 @@ void RemoteBootstrapServiceImpl::FetchData(const FetchDataRequestPB* req,
   int64_t client_maxlen = req->max_length();
 
   const DataIdPB& data_id = req->data_id();
-  RemoteBootstrapErrorPB::Code error_code = RemoteBootstrapErrorPB::UNKNOWN_ERROR;
+  TabletCopyErrorPB::Code error_code = TabletCopyErrorPB::UNKNOWN_ERROR;
   RPC_RETURN_NOT_OK(ValidateFetchRequestDataId(data_id, &error_code, session),
                     error_code, "Invalid DataId");
 
@@ -234,60 +234,60 @@ void RemoteBootstrapServiceImpl::FetchData(const FetchDataRequestPB* req,
   context->RespondSuccess();
 }
 
-void RemoteBootstrapServiceImpl::EndRemoteBootstrapSession(
-        const EndRemoteBootstrapSessionRequestPB* req,
-        EndRemoteBootstrapSessionResponsePB* resp,
+void TabletCopyServiceImpl::EndTabletCopySession(
+        const EndTabletCopySessionRequestPB* req,
+        EndTabletCopySessionResponsePB* resp,
         rpc::RpcContext* context) {
   {
     MutexLock l(sessions_lock_);
-    RemoteBootstrapErrorPB::Code app_error;
-    LOG(INFO) << "Request end of remote bootstrap session " << req->session_id()
+    TabletCopyErrorPB::Code app_error;
+    LOG(INFO) << "Request end of tablet copy session " << req->session_id()
       << " received from " << context->requestor_string();
-    RPC_RETURN_NOT_OK(DoEndRemoteBootstrapSessionUnlocked(req->session_id(), &app_error),
+    RPC_RETURN_NOT_OK(DoEndTabletCopySessionUnlocked(req->session_id(), &app_error),
                       app_error, "No such session");
   }
   context->RespondSuccess();
 }
 
-void RemoteBootstrapServiceImpl::Shutdown() {
+void TabletCopyServiceImpl::Shutdown() {
   shutdown_latch_.CountDown();
   session_expiration_thread_->Join();
 
-  // Destroy all remote bootstrap sessions.
+  // Destroy all tablet copy sessions.
   vector<string> session_ids;
   for (const MonoTimeMap::value_type& entry : session_expirations_) {
     session_ids.push_back(entry.first);
   }
   for (const string& session_id : session_ids) {
-    LOG(INFO) << "Destroying remote bootstrap session " << session_id << " due to service shutdown";
-    RemoteBootstrapErrorPB::Code app_error;
-    CHECK_OK(DoEndRemoteBootstrapSessionUnlocked(session_id, &app_error));
+    LOG(INFO) << "Destroying tablet copy session " << session_id << " due to service shutdown";
+    TabletCopyErrorPB::Code app_error;
+    CHECK_OK(DoEndTabletCopySessionUnlocked(session_id, &app_error));
   }
 }
 
-Status RemoteBootstrapServiceImpl::FindSessionUnlocked(
+Status TabletCopyServiceImpl::FindSessionUnlocked(
         const string& session_id,
-        RemoteBootstrapErrorPB::Code* app_error,
-        scoped_refptr<RemoteBootstrapSession>* session) const {
+        TabletCopyErrorPB::Code* app_error,
+        scoped_refptr<TabletCopySession>* session) const {
   if (!FindCopy(sessions_, session_id, session)) {
-    *app_error = RemoteBootstrapErrorPB::NO_SESSION;
+    *app_error = TabletCopyErrorPB::NO_SESSION;
     return Status::NotFound(
-        Substitute("Remote bootstrap session with Session ID \"$0\" not found", session_id));
+        Substitute("Tablet Copy session with Session ID \"$0\" not found", session_id));
   }
   return Status::OK();
 }
 
-Status RemoteBootstrapServiceImpl::ValidateFetchRequestDataId(
+Status TabletCopyServiceImpl::ValidateFetchRequestDataId(
         const DataIdPB& data_id,
-        RemoteBootstrapErrorPB::Code* app_error,
-        const scoped_refptr<RemoteBootstrapSession>& session) const {
+        TabletCopyErrorPB::Code* app_error,
+        const scoped_refptr<TabletCopySession>& session) const {
   if (PREDICT_FALSE(data_id.has_block_id() && data_id.has_wal_segment_seqno())) {
-    *app_error = RemoteBootstrapErrorPB::INVALID_REMOTE_BOOTSTRAP_REQUEST;
+    *app_error = TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST;
     return Status::InvalidArgument(
         Substitute("Only one of BlockId or segment sequence number are required, "
             "but both were specified. DataTypeID: $0", data_id.ShortDebugString()));
   } else if (PREDICT_FALSE(!data_id.has_block_id() && !data_id.has_wal_segment_seqno())) {
-    *app_error = RemoteBootstrapErrorPB::INVALID_REMOTE_BOOTSTRAP_REQUEST;
+    *app_error = TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST;
     return Status::InvalidArgument(
         Substitute("Only one of BlockId or segment sequence number are required, "
             "but neither were specified. DataTypeID: $0", data_id.ShortDebugString()));
@@ -309,20 +309,20 @@ Status RemoteBootstrapServiceImpl::ValidateFetchRequestDataId(
   return Status::OK();
 }
 
-void RemoteBootstrapServiceImpl::ResetSessionExpirationUnlocked(const std::string& session_id) {
+void TabletCopyServiceImpl::ResetSessionExpirationUnlocked(const std::string& session_id) {
   MonoTime expiration(MonoTime::Now(MonoTime::FINE));
-  expiration.AddDelta(MonoDelta::FromMilliseconds(FLAGS_remote_bootstrap_idle_timeout_ms));
+  expiration.AddDelta(MonoDelta::FromMilliseconds(FLAGS_tablet_copy_idle_timeout_ms));
   InsertOrUpdate(&session_expirations_, session_id, expiration);
 }
 
-Status RemoteBootstrapServiceImpl::DoEndRemoteBootstrapSessionUnlocked(
+Status TabletCopyServiceImpl::DoEndTabletCopySessionUnlocked(
         const std::string& session_id,
-        RemoteBootstrapErrorPB::Code* app_error) {
-  scoped_refptr<RemoteBootstrapSession> session;
+        TabletCopyErrorPB::Code* app_error) {
+  scoped_refptr<TabletCopySession> session;
   RETURN_NOT_OK(FindSessionUnlocked(session_id, app_error, &session));
   // Remove the session from the map.
   // It will get destroyed once there are no outstanding refs.
-  LOG(INFO) << "Ending remote bootstrap session " << session_id << " on tablet "
+  LOG(INFO) << "Ending tablet copy session " << session_id << " on tablet "
     << session->tablet_id() << " with peer " << session->requestor_uuid();
   CHECK_EQ(1, sessions_.erase(session_id));
   CHECK_EQ(1, session_expirations_.erase(session_id));
@@ -330,7 +330,7 @@ Status RemoteBootstrapServiceImpl::DoEndRemoteBootstrapSessionUnlocked(
   return Status::OK();
 }
 
-void RemoteBootstrapServiceImpl::EndExpiredSessions() {
+void TabletCopyServiceImpl::EndExpiredSessions() {
   do {
     MutexLock l(sessions_lock_);
     MonoTime now = MonoTime::Now(MonoTime::FINE);
@@ -344,13 +344,13 @@ void RemoteBootstrapServiceImpl::EndExpiredSessions() {
       }
     }
     for (const string& session_id : expired_session_ids) {
-      LOG(INFO) << "Remote bootstrap session " << session_id
+      LOG(INFO) << "Tablet Copy session " << session_id
                 << " has expired. Terminating session.";
-      RemoteBootstrapErrorPB::Code app_error;
-      CHECK_OK(DoEndRemoteBootstrapSessionUnlocked(session_id, &app_error));
+      TabletCopyErrorPB::Code app_error;
+      CHECK_OK(DoEndTabletCopySessionUnlocked(session_id, &app_error));
     }
   } while (!shutdown_latch_.WaitFor(MonoDelta::FromMilliseconds(
-                                    FLAGS_remote_bootstrap_timeout_poll_period_ms)));
+                                    FLAGS_tablet_copy_timeout_poll_period_ms)));
 }
 
 } // namespace tserver

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_service.h b/src/kudu/tserver/remote_bootstrap_service.h
index 43c3055..a5eab8c 100644
--- a/src/kudu/tserver/remote_bootstrap_service.h
+++ b/src/kudu/tserver/remote_bootstrap_service.h
@@ -14,8 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_TSERVER_REMOTE_BOOTSTRAP_SERVICE_H_
-#define KUDU_TSERVER_REMOTE_BOOTSTRAP_SERVICE_H_
+#ifndef KUDU_TSERVER_TABLET_COPY_SERVICE_H_
+#define KUDU_TSERVER_TABLET_COPY_SERVICE_H_
 
 #include <string>
 #include <unordered_map>
@@ -39,54 +39,54 @@ class ReadableLogSegment;
 
 namespace tserver {
 
-class RemoteBootstrapSession;
+class TabletCopySession;
 class TabletPeerLookupIf;
 
-class RemoteBootstrapServiceImpl : public RemoteBootstrapServiceIf {
+class TabletCopyServiceImpl : public TabletCopyServiceIf {
  public:
-  RemoteBootstrapServiceImpl(FsManager* fs_manager,
+  TabletCopyServiceImpl(FsManager* fs_manager,
                              TabletPeerLookupIf* tablet_peer_lookup,
                              const scoped_refptr<MetricEntity>& metric_entity,
                              const scoped_refptr<rpc::ResultTracker>& result_tracker);
 
-  virtual void BeginRemoteBootstrapSession(const BeginRemoteBootstrapSessionRequestPB* req,
-                                           BeginRemoteBootstrapSessionResponsePB* resp,
+  virtual void BeginTabletCopySession(const BeginTabletCopySessionRequestPB* req,
+                                           BeginTabletCopySessionResponsePB* resp,
                                            rpc::RpcContext* context) OVERRIDE;
 
-  virtual void CheckSessionActive(const CheckRemoteBootstrapSessionActiveRequestPB* req,
-                                  CheckRemoteBootstrapSessionActiveResponsePB* resp,
+  virtual void CheckSessionActive(const CheckTabletCopySessionActiveRequestPB* req,
+                                  CheckTabletCopySessionActiveResponsePB* resp,
                                   rpc::RpcContext* context) OVERRIDE;
 
   virtual void FetchData(const FetchDataRequestPB* req,
                          FetchDataResponsePB* resp,
                          rpc::RpcContext* context) OVERRIDE;
 
-  virtual void EndRemoteBootstrapSession(const EndRemoteBootstrapSessionRequestPB* req,
-                                         EndRemoteBootstrapSessionResponsePB* resp,
+  virtual void EndTabletCopySession(const EndTabletCopySessionRequestPB* req,
+                                         EndTabletCopySessionResponsePB* resp,
                                          rpc::RpcContext* context) OVERRIDE;
 
   virtual void Shutdown() OVERRIDE;
 
  private:
-  typedef std::unordered_map<std::string, scoped_refptr<RemoteBootstrapSession> > SessionMap;
+  typedef std::unordered_map<std::string, scoped_refptr<TabletCopySession> > SessionMap;
   typedef std::unordered_map<std::string, MonoTime> MonoTimeMap;
 
   // Look up session in session map.
   Status FindSessionUnlocked(const std::string& session_id,
-                             RemoteBootstrapErrorPB::Code* app_error,
-                             scoped_refptr<RemoteBootstrapSession>* session) const;
+                             TabletCopyErrorPB::Code* app_error,
+                             scoped_refptr<TabletCopySession>* session) const;
 
   // Validate the data identifier in a FetchData request.
   Status ValidateFetchRequestDataId(const DataIdPB& data_id,
-                                    RemoteBootstrapErrorPB::Code* app_error,
-                                    const scoped_refptr<RemoteBootstrapSession>& session) const;
+                                    TabletCopyErrorPB::Code* app_error,
+                                    const scoped_refptr<TabletCopySession>& session) const;
 
   // Take note of session activity; Re-update the session timeout deadline.
   void ResetSessionExpirationUnlocked(const std::string& session_id);
 
-  // Destroy the specified remote bootstrap session.
-  Status DoEndRemoteBootstrapSessionUnlocked(const std::string& session_id,
-                                             RemoteBootstrapErrorPB::Code* app_error);
+  // Destroy the specified tablet copy session.
+  Status DoEndTabletCopySessionUnlocked(const std::string& session_id,
+                                             TabletCopyErrorPB::Code* app_error);
 
   // The timeout thread periodically checks whether sessions are expired and
   // removes them from the map.
@@ -109,4 +109,4 @@ class RemoteBootstrapServiceImpl : public RemoteBootstrapServiceIf {
 } // namespace tserver
 } // namespace kudu
 
-#endif // KUDU_TSERVER_REMOTE_BOOTSTRAP_SERVICE_H_
+#endif // KUDU_TSERVER_TABLET_COPY_SERVICE_H_

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tserver/remote_bootstrap_session-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/remote_bootstrap_session-test.cc b/src/kudu/tserver/remote_bootstrap_session-test.cc
index d354699..0f0236a 100644
--- a/src/kudu/tserver/remote_bootstrap_session-test.cc
+++ b/src/kudu/tserver/remote_bootstrap_session-test.cc
@@ -68,9 +68,9 @@ using tablet::TabletPeer;
 using tablet::TabletSuperBlockPB;
 using tablet::WriteTransactionState;
 
-class RemoteBootstrapTest : public KuduTabletTest {
+class TabletCopyTest : public KuduTabletTest {
  public:
-  RemoteBootstrapTest()
+  TabletCopyTest()
     : KuduTabletTest(Schema({ ColumnSchema("key", STRING),
                               ColumnSchema("val", INT32) }, 1)) {
     CHECK_OK(ThreadPoolBuilder("test-exec").Build(&apply_pool_));
@@ -110,7 +110,7 @@ class RemoteBootstrapTest : public KuduTabletTest {
         new TabletPeer(tablet()->metadata(),
                        config_peer,
                        apply_pool_.get(),
-                       Bind(&RemoteBootstrapTest::TabletPeerStateChangedCallback,
+                       Bind(&TabletCopyTest::TabletPeerStateChangedCallback,
                             Unretained(this),
                             tablet()->tablet_id())));
 
@@ -179,12 +179,12 @@ class RemoteBootstrapTest : public KuduTabletTest {
   }
 
   void InitSession() {
-    session_.reset(new RemoteBootstrapSession(tablet_peer_.get(), "TestSession", "FakeUUID",
+    session_.reset(new TabletCopySession(tablet_peer_.get(), "TestSession", "FakeUUID",
                    fs_manager()));
     ASSERT_OK(session_->Init());
   }
 
-  // Read the specified BlockId, via the RemoteBootstrapSession, into a file.
+  // Read the specified BlockId, via the TabletCopySession, into a file.
   // 'path' will be populated with the name of the file used.
   // 'file' will be set to point to the SequentialFile containing the data.
   void FetchBlockToFile(const BlockId& block_id,
@@ -192,7 +192,7 @@ class RemoteBootstrapTest : public KuduTabletTest {
                         gscoped_ptr<SequentialFile>* file) {
     string data;
     int64_t block_file_size = 0;
-    RemoteBootstrapErrorPB::Code error_code;
+    TabletCopyErrorPB::Code error_code;
     CHECK_OK(session_->GetBlockPiece(block_id, 0, 0, &data, &block_file_size, &error_code));
     if (block_file_size > 0) {
       CHECK_GT(data.size(), 0);
@@ -213,12 +213,12 @@ class RemoteBootstrapTest : public KuduTabletTest {
   scoped_refptr<LogAnchorRegistry> log_anchor_registry_;
   gscoped_ptr<ThreadPool> apply_pool_;
   scoped_refptr<TabletPeer> tablet_peer_;
-  scoped_refptr<RemoteBootstrapSession> session_;
+  scoped_refptr<TabletCopySession> session_;
 };
 
-// Ensure that the serialized SuperBlock included in the RemoteBootstrapSession is
+// Ensure that the serialized SuperBlock included in the TabletCopySession is
 // equal to the serialized live superblock (on a quiesced tablet).
-TEST_F(RemoteBootstrapTest, TestSuperBlocksEqual) {
+TEST_F(TabletCopyTest, TestSuperBlocksEqual) {
   // Compare content of superblocks.
   faststring session_buf;
   faststring tablet_buf;
@@ -247,7 +247,7 @@ TEST_F(RemoteBootstrapTest, TestSuperBlocksEqual) {
 
 // Test fetching all files from tablet server, ensure the checksums for each
 // chunk and the total file sizes match.
-TEST_F(RemoteBootstrapTest, TestBlocksEqual) {
+TEST_F(TabletCopyTest, TestBlocksEqual) {
   TabletSuperBlockPB tablet_superblock;
   ASSERT_OK(tablet()->metadata()->ToSuperBlock(&tablet_superblock));
   for (int i = 0; i < tablet_superblock.rowsets_size(); i++) {
@@ -290,7 +290,7 @@ TEST_F(RemoteBootstrapTest, TestBlocksEqual) {
 
 // Ensure that blocks are still readable through the open session even
 // after they've been deleted.
-TEST_F(RemoteBootstrapTest, TestBlocksAreFetchableAfterBeingDeleted) {
+TEST_F(TabletCopyTest, TestBlocksAreFetchableAfterBeingDeleted) {
   TabletSuperBlockPB tablet_superblock;
   ASSERT_OK(tablet()->metadata()->ToSuperBlock(&tablet_superblock));
 
@@ -323,7 +323,7 @@ TEST_F(RemoteBootstrapTest, TestBlocksAreFetchableAfterBeingDeleted) {
   for (const BlockId& block_id : data_blocks) {
     ASSERT_TRUE(session_->IsBlockOpenForTests(block_id));
     string data;
-    RemoteBootstrapErrorPB::Code error_code;
+    TabletCopyErrorPB::Code error_code;
     int64_t piece_size;
     ASSERT_OK(session_->GetBlockPiece(block_id, 0, 0,
                                       &data, &piece_size, &error_code));


[3/3] kudu git commit: Rename Remote Bootstrap to Tablet Copy (part 1)

Posted by to...@apache.org.
Rename Remote Bootstrap to Tablet Copy (part 1)

This does a bunch of automated substitutions of the terminology,
renaming "remote bootstrap" to "tablet copy". This commit does
not rename the files yet, so that it's easier to review without
combining a file name change with the file content change.

This is an incompatible change since some flag and metric names are
changed. Another follow-up will have to add release notes.

Change-Id: Ic017e9121ed365cb3d9c9c81430915bf5b295bf6
Reviewed-on: http://gerrit.cloudera.org:8080/3848
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: a6cb942e855a7a5a022a9f55f1b888df5dbc0d36
Parents: c8e2995
Author: Todd Lipcon <to...@apache.org>
Authored: Fri Aug 5 13:59:51 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Aug 5 21:56:48 2016 +0000

----------------------------------------------------------------------
 docs/contributing.adoc                          |   2 +-
 docs/design-docs/README.md                      |   2 +-
 docs/design-docs/raft-config-change.md          |  18 +--
 docs/design-docs/raft-remote-bootstrap.md       |  54 +++----
 docs/whitepaper/kudu.tex                        |   8 +-
 src/kudu/consensus/consensus.proto              |   6 +-
 src/kudu/consensus/consensus_peers.cc           |  40 +++---
 src/kudu/consensus/consensus_peers.h            |  28 ++--
 src/kudu/consensus/consensus_queue-test.cc      |  88 ++++++------
 src/kudu/consensus/consensus_queue.cc           |  26 ++--
 src/kudu/consensus/consensus_queue.h            |  16 +--
 src/kudu/consensus/log_util.cc                  |   2 +-
 src/kudu/consensus/raft_consensus-test.cc       |   2 +-
 .../integration-tests/client_failover-itest.cc  |   6 +-
 .../integration-tests/cluster_itest_util.cc     |   8 +-
 src/kudu/integration-tests/cluster_itest_util.h |   4 +-
 src/kudu/integration-tests/delete_table-test.cc |  34 ++---
 .../integration-tests/raft_consensus-itest.cc   |   4 +-
 .../integration-tests/remote_bootstrap-itest.cc |  90 ++++++------
 .../tablet_replacement-itest.cc                 |   2 +-
 src/kudu/master/catalog_manager.cc              |   8 +-
 src/kudu/master/catalog_manager.h               |   4 +-
 src/kudu/tablet/metadata.proto                  |   4 +-
 src/kudu/tablet/tablet_bootstrap-test.cc        |   4 +-
 src/kudu/tablet/tablet_bootstrap.cc             |   4 +-
 src/kudu/tablet/tablet_metadata.h               |   4 +-
 src/kudu/tablet/tablet_peer.cc                  |   2 +-
 src/kudu/tserver/CMakeLists.txt                 |  10 +-
 src/kudu/tserver/remote_bootstrap-test-base.h   |  12 +-
 src/kudu/tserver/remote_bootstrap.proto         |  52 +++----
 .../tserver/remote_bootstrap_client-test.cc     |  24 ++--
 src/kudu/tserver/remote_bootstrap_client.cc     | 138 +++++++++---------
 src/kudu/tserver/remote_bootstrap_client.h      |  50 +++----
 .../tserver/remote_bootstrap_service-test.cc    | 144 +++++++++----------
 src/kudu/tserver/remote_bootstrap_service.cc    | 140 +++++++++---------
 src/kudu/tserver/remote_bootstrap_service.h     |  40 +++---
 .../tserver/remote_bootstrap_session-test.cc    |  24 ++--
 src/kudu/tserver/remote_bootstrap_session.cc    |  70 ++++-----
 src/kudu/tserver/remote_bootstrap_session.h     |  26 ++--
 src/kudu/tserver/tablet_peer_lookup.h           |   4 +-
 src/kudu/tserver/tablet_server.cc               |   4 +-
 src/kudu/tserver/tablet_service.cc              |  12 +-
 src/kudu/tserver/tablet_service.h               |   4 +-
 src/kudu/tserver/ts_tablet_manager.cc           |  44 +++---
 src/kudu/tserver/ts_tablet_manager.h            |  10 +-
 src/kudu/tserver/tserver.proto                  |   2 +-
 46 files changed, 640 insertions(+), 640 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/docs/contributing.adoc
----------------------------------------------------------------------
diff --git a/docs/contributing.adoc b/docs/contributing.adoc
index 2b6b217..e74995f 100644
--- a/docs/contributing.adoc
+++ b/docs/contributing.adoc
@@ -304,7 +304,7 @@ Reference].
   used to group similar gflags together. If the gflag affects the entire
   process, it should not be prefixed.
 - If the gflag is for a quantity, the name should be suffixed with the units.
-  For example, `remote_bootstrap_idle_timeout_ms`.
+  For example, `tablet_copy_idle_timeout_ms`.
 - Where possible, use short names. This will save time for those entering
   command line options by hand.
 - The name is part of Kudu's compatibility contract, and should not change

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/docs/design-docs/README.md
----------------------------------------------------------------------
diff --git a/docs/design-docs/README.md b/docs/design-docs/README.md
index 433c2b4..429ea4f 100644
--- a/docs/design-docs/README.md
+++ b/docs/design-docs/README.md
@@ -26,7 +26,7 @@ made.
 | [Codegen API and impl. details](codegen.md) | Server | N/A |
 | [Consensus design](consensus.md) | Consensus | N/A |
 | [Raft config change design](raft-config-change.md) | Consensus | N/A |
-| [Raft remote bootstrap design](raft-remote-bootstrap.md) | Consensus | N/A |
+| [Raft tablet copy design](raft-remote-bootstrap.md) | Consensus | N/A |
 | [Master design](master.md) | Master | N/A |
 | [RPC design and impl. details](rpc.md) | RPC | N/A |
 | [Tablet design, impl. details and comparison to other systems](tablet.md) | Tablet | N/A |

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/docs/design-docs/raft-config-change.md
----------------------------------------------------------------------
diff --git a/docs/design-docs/raft-config-change.md b/docs/design-docs/raft-config-change.md
index d67c6cc..899c72d 100644
--- a/docs/design-docs/raft-config-change.md
+++ b/docs/design-docs/raft-config-change.md
@@ -68,14 +68,14 @@ in [2].
 This process is executed by a driver, which may be a client program or the
 Master. We\u2019ll say the node to be added to the cluster is named `new_node`.
 
-1. Driver initiates execution of remote bootstrap procedure of `new_node` from
+1. Driver initiates execution of tablet copy procedure of `new_node` from
    the current leader `bootstrap_source` using an RPC call to the `new_node`.
-   Remote bootstrap runs to completion, which means all data and logs at the
-   time remote bootstrap was initiated were replicated to `new_node`. Driver
-   polls `new_node` for indication that the remote bootstrap process is
+   Tablet Copy runs to completion, which means all data and logs at the
+   time tablet copy was initiated were replicated to `new_node`. Driver
+   polls `new_node` for indication that the tablet copy process is
    complete.
    <br>
-   If the `bootstrap_source` node crashes before remote bootstrap is complete,
+   If the `bootstrap_source` node crashes before tablet copy is complete,
    the bootstrap fails and the driver must start the entire process over from
    the beginning. If the driver or `new_node` crashes and the tablet never
    joins the configuration, the Master should eventually delete the abandoned
@@ -95,7 +95,7 @@ Master. We\u2019ll say the node to be added to the cluster is named `new_node`.
 3. As soon as a replica receives the ConfigChangeRequest it applies the
    configuration change in-memory. It does not wait for commitment to apply the
    change. See rationale in [2] section 4.1.
-4. The remote bootstrap session between `new_node` and `bootstrap_source` is
+4. The tablet copy session between `new_node` and `bootstrap_source` is
    closed once the config change to transition the node to `PRE_FOLLOWER` has
    been committed. This implies releasing an anchor on the log. Since
    `new_node` is already a member of the configuration receiving log updates,
@@ -167,12 +167,12 @@ another doc.
 ### Steps:
 
 1. Run a tool to determine the most up-to-date remaining replica.
-2. Remote bootstrap additional nodes from the most up-to-date remaining node.
-   Wait for remote bootstrap to complete on all the nodes.
+2. Tablet Copy additional nodes from the most up-to-date remaining node.
+   Wait for tablet copy to complete on all the nodes.
 3. Bring all tablet servers hosting the affected tablet offline (TODO: This is
    possible to implement per-tablet but not currently supported)
 4. Run tool to rewrite the ConsensusMetadata file per-tablet server to
-   forcefully update the configuration membership to add remotely bootstrapped
+   forcefully update the configuration membership to add copied
    nodes as followers. TODO: Violates Raft not to append to the log, do we also
    need to do that?
 5. Bring the affected tablets / tablet servers back online.

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/docs/design-docs/raft-remote-bootstrap.md
----------------------------------------------------------------------
diff --git a/docs/design-docs/raft-remote-bootstrap.md b/docs/design-docs/raft-remote-bootstrap.md
index e5a2a81..66c11df 100644
--- a/docs/design-docs/raft-remote-bootstrap.md
+++ b/docs/design-docs/raft-remote-bootstrap.md
@@ -12,19 +12,19 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-# Kudu remote bootstrap design
+# Kudu tablet copy design
 
-Master & remote bootstrap integration with configuration change
+Master & tablet copy integration with configuration change
 
 ## Summary
 
-This document contains information on implementing remote bootstrap in the
+This document contains information on implementing tablet copy in the
 context of Kudu's Raft implementation. Details on Raft config change in Kudu
 can be found in the [Raft config change design doc](raft-config-change.md).
 
 ## Goals
 
-1. Integrate remote bootstrap to allow for copying over "snapshotted" data and
+1. Integrate tablet copy to allow for copying over "snapshotted" data and
    logs to a tablet replica when logs from the "beginning of time" are no
    longer available for replay.
 2. The Master needs to tolerate and facilitate dynamic consensus config change.
@@ -38,29 +38,29 @@ The Master will expose the following operations to admin users:
 1. `AddReplica(tablet_id, TS to_add, Role role)`
 2. `RemoveReplica(tablet_id, TS to_remove)`
 
-### New tablet server remote bootstrap RPC
+### New tablet server tablet copy RPC
 
-Remote bootstrap allows a tablet snapshot to be moved to a new server. A
-`StartRemoteBootstrap()` RPC call will be available on each tablet server. When a
+Tablet Copy allows a tablet snapshot to be moved to a new server. A
+`StartTabletCopy()` RPC call will be available on each tablet server. When a
 leader determines that a follower needs log entries prior to what is available
 on the leader side, or when it detects that a follower does not host a given
 tablet, it sends the follower an RPC to instruct the follower to initiate
-remote bootstrap. Optionally, this callback is made idempotent by passing the
+tablet copy. Optionally, this callback is made idempotent by passing the
 latest OpId in the follower's log as an argument.
 
-### Management of remote bootstrap jobs
+### Management of tablet copy jobs
 
-Since remotely bootstrapping a tablet may involve copying many GB of data, we
-likely need to support operational visibility into ongoing remote bootstrap
+Since copying a tablet may involve copying many GB of data, we
+likely need to support operational visibility into ongoing tablet copy
 jobs, run them on their own thread pool, support cancellation, etc. TBD to
 enumerate all of this in detail.
 
-## Design & implementation of tablet remote bootstrap
+## Design & implementation of tablet tablet copy
 
 ### Tablet auto-vivification
 
 A leader could cause a tablet to auto-create / auto-vivify itself if it doesn't
-already exist by sending a StartRemoteBootstrap RPC to the tablet server. Not
+already exist by sending a StartTabletCopy RPC to the tablet server. Not
 requiring the Master to explicitly invoke a CreateTablet() RPC before adding a
 replica to a consensus config makes adding a new peer much simpler to implement
 on the Master side.
@@ -77,7 +77,7 @@ given node:
 1. Master -> Leader: `AddServer(peer=Follower_new, role=PRE_VOTER)`
 
 The leader will then take care of detecting whether the tablet is out of date
-or does not exist, in which cases it must be remotely bootstrapped, or whether
+or does not exist, in which cases it must be copied, or whether
 it can be caught-up normally.
 
 ### Note on deleted tablets
@@ -89,17 +89,17 @@ losing data that was guaranteed to be persisted according to the consensus
 protocol. That situation is described in detail in this [raft-dev mailing list
 thread](https://groups.google.com/d/msg/raft-dev/CL1qWP7a_1w/OfHqmbcbIlAJ).
 
-To safely support deletion and remote bootstrap, a tablet will have 4 states
+To safely support deletion and tablet copy, a tablet will have 4 states
 its data can be in: `DOES_NOT_EXIST` (implicit; just the non-existence of state),
 `DELETED`, `COPYING`, and `READY`. `DOES_NOT_EXIST` just means a tablet with that name
 has never been hosted on the server, `DELETED` means it's tombstoned, `COPYING`
-means it's in the process of remote bootstrapping, and `READY` means it's in a
+means it's in the process of tablet copyping, and `READY` means it's in a
 normal, consistent state. More details about Tablet Deletion is in a later
 section.
 
-### Auto-vivifying remote bootstrap protocol
+### Auto-vivifying tablet copy protocol
 
-The remote bootstrap protocol between the leader and follower is as follows.
+The tablet copy protocol between the leader and follower is as follows.
 The leader always starts attempting to heartbeat to the follower:
 
 Leader -> Follower: `AppendEntries(from, to, term, prev_idx, ops[])`
@@ -118,14 +118,14 @@ AppendEntries(from, to, term, prev_idx, ops[]):
 
 If the leader gets back a `DOES_NOT_EXIST` or `DELETED` tablet status, it will
 repeatedly attempt to "auto-vivify" the tablet on the follower by sending a
-StartRemoteBootstrap RPC to the follower.
+StartTabletCopy RPC to the follower.
 
-On the follower, the `StartRemoteBootstrap` RPC is idempotent w.r.t. repeated RPC
+On the follower, the `StartTabletCopy` RPC is idempotent w.r.t. repeated RPC
 requests from the leader and has logic to create a tablet if it doesn't yet
 exist. Roughly:
 
 ```
-StartRemoteBootstrap(from, to, tablet, current_state, last_opid_in_log = NULL):
+StartTabletCopy(from, to, tablet, current_state, last_opid_in_log = NULL):
   if (to != self.uuid): ERR_INVALID_NAME
   if (this.tablet.state == COPYING): ERR_ALREADY_BOOTSTRAPPING
   if (this.tablet.state != current_state): ERR_ILLEGAL_STATE
@@ -135,11 +135,11 @@ StartRemoteBootstrap(from, to, tablet, current_state, last_opid_in_log = NULL):
     CreateTablet(COPYING) # Create tablet in "COPYING" mode.
   if (caller.term < self.term): ERR_BAD_TERM
   if (last_opid_in_log != NULL && != this.log.last_op.id): ERR_ILLEGAL_STATE
-  RunRemoteBootstrap() # Download the tablet data.
+  RunTabletCopy() # Download the tablet data.
 ```
 
 The detailed process, on the follower side, of downloading and replacing the
-data is detailed below under "Follower Remote Bootstrap".
+data is detailed below under "Follower Tablet Copy".
 
 ### Tablet directory structure
 
@@ -199,9 +199,9 @@ We can safely implement tablet deletion using the following steps:
    not stripe the WAL. If we were to stripe or multiplex the WAL in the future,
    we could add some kind of tablet-level sequence number, like a generation
    number, that gets incremented at the SuperBlock level when we initiate
-   remote bootstrap. That should keep us pointed at the relevant entries.
+   tablet copy. That should keep us pointed at the relevant entries.
 
-### Follower remote bootstrap
+### Follower tablet copy
 
 Remotely bootstrapping a tablet copies the data from the remote; merges the new
 and old consensus metadata files (if a local one already existed; otherwise the
@@ -315,8 +315,8 @@ possible to do as long as we retain consensus metadata indefinitely, which is
 required for correctness anyway. However this is not a top priority.
 
 One scenario where a `DELETED` tablet may need to vote to make forward progress
-is if a `VOTER` replica falls behind and so starts to remote bootstrap, crashes
-in the middle of remote bootstrap, and deletes itself at startup. Once we
+is if a `VOTER` replica falls behind and so starts to tablet copy, crashes
+in the middle of tablet copy, and deletes itself at startup. Once we
 implement `PRE_VOTER`, and always catch up as a `PRE_VOTER` before becoming a
 `VOTER`, the opportunity for potential problems with `VOTER`s is reduced a lot,
 especially around the initial step of adding a server to the cluster, but still

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/docs/whitepaper/kudu.tex
----------------------------------------------------------------------
diff --git a/docs/whitepaper/kudu.tex b/docs/whitepaper/kudu.tex
index dabc052..7cf9e90 100644
--- a/docs/whitepaper/kudu.tex
+++ b/docs/whitepaper/kudu.tex
@@ -363,11 +363,11 @@ by at most one in each configuration change. In order to grow a 3-replica config
 replicas, two separate configuration changes (3$\rightarrow$4, 4$\rightarrow$5) must be proposed
 and committed.
 
-Kudu implements the addition of new servers through a process called {\em remote bootstrap}.
+Kudu implements the addition of new servers through a process called {\em tablet copy}.
 In our design, in order to add a new replica, we first add it as a new member in the
 Raft configuration, even before notifying the destination server that a new replica will
 be copied to it. When this configuration change has been committed, the current Raft leader
-replica triggers a {\tt StartRemoteBootstrap} RPC, which causes the destination server to pull a
+replica triggers a {\tt StartTabletCopy} RPC, which causes the destination server to pull a
 snapshot of the tablet data and log from the current leader. When the transfer
 is complete, the new server opens the tablet following the same process as after
 a server restart. When the tablet has opened the tablet data and replayed any necessary
@@ -378,11 +378,11 @@ In our current implementation, new servers are added immediately as {\tt VOTER}
 the disadvantage that, after moving from a 3-server configuration to a 4-server configuration, three
 out of the four servers must acknowledge each operation. Because the new server is in the process of
 copying, it is unable to acknowledge operations. If another server were to crash during the
-snapshot-transfer process, the tablet would become unavailable for writes until the remote bootstrap
+snapshot-transfer process, the tablet would become unavailable for writes until the tablet copy
 finished.
 
 To address this issue, we plan to implement a {\tt PRE\_VOTER} replica state. In this
-state, the leader will send Raft updates and trigger remote bootstrap on the
+state, the leader will send Raft updates and trigger tablet copy on the
 target replica, but not count it as a voter when calculating the size of the configuration's
 majority. Upon detecting that the {\tt PRE\_VOTER} replica has fully caught up to
 the current logs, the leader will automatically propose and commit another configuration change to

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/consensus.proto
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus.proto b/src/kudu/consensus/consensus.proto
index 0091080..3c18f84 100644
--- a/src/kudu/consensus/consensus.proto
+++ b/src/kudu/consensus/consensus.proto
@@ -451,7 +451,7 @@ message GetConsensusStateResponsePB {
   optional tserver.TabletServerErrorPB error = 2;
 }
 
-message StartRemoteBootstrapRequestPB {
+message StartTabletCopyRequestPB {
   // UUID of server this request is addressed to.
   optional bytes dest_uuid = 5;
 
@@ -467,7 +467,7 @@ message StartRemoteBootstrapRequestPB {
   optional int64 caller_term = 4 [ default = -1 ];
 }
 
-message StartRemoteBootstrapResponsePB {
+message StartTabletCopyResponsePB {
   optional tserver.TabletServerErrorPB error = 1;
 }
 
@@ -499,5 +499,5 @@ service ConsensusService {
   rpc GetConsensusState(GetConsensusStateRequestPB) returns (GetConsensusStateResponsePB);
 
   // Instruct this server to remotely bootstrap a tablet from another host.
-  rpc StartRemoteBootstrap(StartRemoteBootstrapRequestPB) returns (StartRemoteBootstrapResponsePB);
+  rpc StartTabletCopy(StartTabletCopyRequestPB) returns (StartTabletCopyResponsePB);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 5363dea..2d4fa6c 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -61,13 +61,13 @@ DEFINE_double(fault_crash_after_leader_request_fraction, 0.0,
 TAG_FLAG(fault_crash_on_leader_request_fraction, unsafe);
 
 
-// Allow for disabling remote bootstrap in unit tests where we want to test
+// Allow for disabling tablet copy in unit tests where we want to test
 // certain scenarios without triggering bootstrap of a remote peer.
-DEFINE_bool(enable_remote_bootstrap, true,
-            "Whether remote bootstrap will be initiated by the leader when it "
+DEFINE_bool(enable_tablet_copy, true,
+            "Whether tablet copy will be initiated by the leader when it "
             "detects that a follower is out of date or does not have a tablet "
             "replica. For testing purposes only.");
-TAG_FLAG(enable_remote_bootstrap, unsafe);
+TAG_FLAG(enable_tablet_copy, unsafe);
 
 namespace kudu {
 namespace consensus {
@@ -174,11 +174,11 @@ Status Peer::SignalRequest(bool even_if_queue_empty) {
 
 void Peer::SendNextRequest(bool even_if_queue_empty) {
   // The peer has no pending request nor is sending: send the request.
-  bool needs_remote_bootstrap = false;
+  bool needs_tablet_copy = false;
   int64_t commit_index_before = request_.has_committed_index() ?
       request_.committed_index().index() : kMinimumOpIdIndex;
   Status s = queue_->RequestForPeer(peer_pb_.permanent_uuid(), &request_,
-                                    &replicate_msg_refs_, &needs_remote_bootstrap);
+                                    &replicate_msg_refs_, &needs_tablet_copy);
   int64_t commit_index_after = request_.has_committed_index() ?
       request_.committed_index().index() : kMinimumOpIdIndex;
 
@@ -189,10 +189,10 @@ void Peer::SendNextRequest(bool even_if_queue_empty) {
     return;
   }
 
-  if (PREDICT_FALSE(needs_remote_bootstrap)) {
-    Status s = SendRemoteBootstrapRequest();
+  if (PREDICT_FALSE(needs_tablet_copy)) {
+    Status s = SendTabletCopyRequest();
     if (!s.ok()) {
-      LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to generate remote bootstrap request for peer: "
+      LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to generate tablet copy request for peer: "
                                         << s.ToString();
       sem_.Release();
     }
@@ -294,28 +294,28 @@ void Peer::DoProcessResponse() {
   }
 }
 
-Status Peer::SendRemoteBootstrapRequest() {
-  if (!FLAGS_enable_remote_bootstrap) {
+Status Peer::SendTabletCopyRequest() {
+  if (!FLAGS_enable_tablet_copy) {
     failed_attempts_++;
-    return Status::NotSupported("remote bootstrap is disabled");
+    return Status::NotSupported("tablet copy is disabled");
   }
 
   LOG_WITH_PREFIX_UNLOCKED(INFO) << "Sending request to remotely bootstrap";
-  RETURN_NOT_OK(queue_->GetRemoteBootstrapRequestForPeer(peer_pb_.permanent_uuid(), &rb_request_));
+  RETURN_NOT_OK(queue_->GetTabletCopyRequestForPeer(peer_pb_.permanent_uuid(), &rb_request_));
   controller_.Reset();
-  proxy_->StartRemoteBootstrap(&rb_request_, &rb_response_, &controller_,
-                               boost::bind(&Peer::ProcessRemoteBootstrapResponse, this));
+  proxy_->StartTabletCopy(&rb_request_, &rb_response_, &controller_,
+                               boost::bind(&Peer::ProcessTabletCopyResponse, this));
   return Status::OK();
 }
 
-void Peer::ProcessRemoteBootstrapResponse() {
+void Peer::ProcessTabletCopyResponse() {
   if (controller_.status().ok() && rb_response_.has_error()) {
     // ALREADY_INPROGRESS is expected, so we do not log this error.
     if (rb_response_.error().code() ==
         TabletServerErrorPB::TabletServerErrorPB::ALREADY_INPROGRESS) {
       queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());
     } else {
-      LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin remote bootstrap on peer: "
+      LOG_WITH_PREFIX_UNLOCKED(WARNING) << "Unable to begin tablet copy on peer: "
                                         << rb_response_.ShortDebugString();
     }
   }
@@ -392,11 +392,11 @@ void RpcPeerProxy::RequestConsensusVoteAsync(const VoteRequestPB* request,
   consensus_proxy_->RequestConsensusVoteAsync(*request, response, controller, callback);
 }
 
-void RpcPeerProxy::StartRemoteBootstrap(const StartRemoteBootstrapRequestPB* request,
-                                        StartRemoteBootstrapResponsePB* response,
+void RpcPeerProxy::StartTabletCopy(const StartTabletCopyRequestPB* request,
+                                        StartTabletCopyResponsePB* response,
                                         rpc::RpcController* controller,
                                         const rpc::ResponseCallback& callback) {
-  consensus_proxy_->StartRemoteBootstrapAsync(*request, response, controller, callback);
+  consensus_proxy_->StartTabletCopyAsync(*request, response, controller, callback);
 }
 
 RpcPeerProxy::~RpcPeerProxy() {}

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/consensus_peers.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.h b/src/kudu/consensus/consensus_peers.h
index e10856f..f676c4c 100644
--- a/src/kudu/consensus/consensus_peers.h
+++ b/src/kudu/consensus/consensus_peers.h
@@ -160,15 +160,15 @@ class Peer {
   // Run on 'thread_pool'. Does response handling that requires IO or may block.
   void DoProcessResponse();
 
-  // Fetch the desired remote bootstrap request from the queue and send it
-  // to the peer. The callback goes to ProcessRemoteBootstrapResponse().
+  // Fetch the desired tablet copy request from the queue and send it
+  // to the peer. The callback goes to ProcessTabletCopyResponse().
   //
-  // Returns a bad Status if remote bootstrap is disabled, or if the
+  // Returns a bad Status if tablet copy is disabled, or if the
   // request cannot be generated for some reason.
-  Status SendRemoteBootstrapRequest();
+  Status SendTabletCopyRequest();
 
-  // Handle RPC callback from initiating remote bootstrap.
-  void ProcessRemoteBootstrapResponse();
+  // Handle RPC callback from initiating tablet copy.
+  void ProcessTabletCopyResponse();
 
   // Signals there was an error sending the request to the peer.
   void ProcessResponseError(const Status& status);
@@ -191,9 +191,9 @@ class Peer {
   ConsensusRequestPB request_;
   ConsensusResponsePB response_;
 
-  // The latest remote bootstrap request and response.
-  StartRemoteBootstrapRequestPB rb_request_;
-  StartRemoteBootstrapResponsePB rb_response_;
+  // The latest tablet copy request and response.
+  StartTabletCopyRequestPB rb_request_;
+  StartTabletCopyResponsePB rb_response_;
 
   // Reference-counted pointers to any ReplicateMsgs which are in-flight to the peer. We
   // may have loaded these messages from the LogCache, in which case we are potentially
@@ -249,9 +249,9 @@ class PeerProxy {
                                          rpc::RpcController* controller,
                                          const rpc::ResponseCallback& callback) = 0;
 
-  // Instructs a peer to begin a remote bootstrap session.
-  virtual void StartRemoteBootstrap(const StartRemoteBootstrapRequestPB* request,
-                                    StartRemoteBootstrapResponsePB* response,
+  // Instructs a peer to begin a tablet copy session.
+  virtual void StartTabletCopy(const StartTabletCopyRequestPB* request,
+                                    StartTabletCopyResponsePB* response,
                                     rpc::RpcController* controller,
                                     const rpc::ResponseCallback& callback) {
     LOG(DFATAL) << "Not implemented";
@@ -287,8 +287,8 @@ class RpcPeerProxy : public PeerProxy {
                                          rpc::RpcController* controller,
                                          const rpc::ResponseCallback& callback) OVERRIDE;
 
-  virtual void StartRemoteBootstrap(const StartRemoteBootstrapRequestPB* request,
-                                    StartRemoteBootstrapResponsePB* response,
+  virtual void StartTabletCopy(const StartTabletCopyRequestPB* request,
+                                    StartTabletCopyResponsePB* response,
                                     rpc::RpcController* controller,
                                     const rpc::ResponseCallback& callback) OVERRIDE;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/consensus_queue-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue-test.cc b/src/kudu/consensus/consensus_queue-test.cc
index fe10135..204f65f 100644
--- a/src/kudu/consensus/consensus_queue-test.cc
+++ b/src/kudu/consensus/consensus_queue-test.cc
@@ -113,9 +113,9 @@ class ConsensusQueueTest : public KuduTest {
     // Ask for a request. The queue assumes the peer is up-to-date so
     // this should contain no operations.
     vector<ReplicateRefPtr> refs;
-    bool needs_remote_bootstrap;
-    ASSERT_OK(queue_->RequestForPeer(kPeerUuid, request, &refs, &needs_remote_bootstrap));
-    ASSERT_FALSE(needs_remote_bootstrap);
+    bool needs_tablet_copy;
+    ASSERT_OK(queue_->RequestForPeer(kPeerUuid, request, &refs, &needs_tablet_copy));
+    ASSERT_FALSE(needs_tablet_copy);
     ASSERT_EQ(request->ops_size(), 0);
 
     // Refuse saying that the log matching property check failed and
@@ -220,9 +220,9 @@ TEST_F(ConsensusQueueTest, TestStartTrackingAfterStart) {
 
   // Getting a new request should get all operations after 7.50
   vector<ReplicateRefPtr> refs;
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(50, request.ops_size());
 
   SetLastReceivedAndLastCommitted(&response, request.ops(49).id());
@@ -230,8 +230,8 @@ TEST_F(ConsensusQueueTest, TestStartTrackingAfterStart) {
   ASSERT_FALSE(more_pending) << "Queue still had requests pending";
 
   // if we ask for a new request, it should come back empty
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(0, request.ops_size());
 
   // extract the ops from the request to avoid double free
@@ -281,9 +281,9 @@ TEST_F(ConsensusQueueTest, TestGetPagedMessages) {
   for (int i = 0; i < 11; i++) {
     VLOG(1) << "Making request " << i;
     vector<ReplicateRefPtr> refs;
-    bool needs_remote_bootstrap;
-    ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-    ASSERT_FALSE(needs_remote_bootstrap);
+    bool needs_tablet_copy;
+    ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+    ASSERT_FALSE(needs_tablet_copy);
     ASSERT_EQ(kOpsPerRequest, request.ops_size());
     last = request.ops(request.ops_size() -1).id();
     SetLastReceivedAndLastCommitted(&response, last);
@@ -292,9 +292,9 @@ TEST_F(ConsensusQueueTest, TestGetPagedMessages) {
     ASSERT_TRUE(more_pending);
   }
   vector<ReplicateRefPtr> refs;
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(1, request.ops_size());
   last = request.ops(request.ops_size() -1).id();
   SetLastReceivedAndLastCommitted(&response, last);
@@ -337,9 +337,9 @@ TEST_F(ConsensusQueueTest, TestPeersDontAckBeyondWatermarks) {
   ASSERT_OPID_EQ(queue_->GetMajorityReplicatedOpIdForTests(), MinimumOpId());
 
   vector<ReplicateRefPtr> refs;
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(50, request.ops_size());
 
   AppendReplicateMessagesToQueue(queue_.get(), clock_, 101, 100);
@@ -354,8 +354,8 @@ TEST_F(ConsensusQueueTest, TestPeersDontAckBeyondWatermarks) {
   ASSERT_OPID_EQ(queue_->GetAllReplicatedIndexForTests(), MakeOpId(14, 100));
 
   // if we ask for a new request, it should come back with the rest of the messages
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(100, request.ops_size());
 
   OpId expected = request.ops(99).id();
@@ -499,9 +499,9 @@ TEST_F(ConsensusQueueTest, TestQueueLoadsOperationsForPeer) {
   // When we get another request for the peer the queue should load
   // the missing operations.
   vector<ReplicateRefPtr> refs;
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(request.ops_size(), 50);
 
   // The messages still belong to the queue so we have to release them.
@@ -558,9 +558,9 @@ TEST_F(ConsensusQueueTest, TestQueueHandlesOperationOverwriting) {
 
   // Ask for a request. The queue assumes the peer is up-to-date so
   // this should contain no operations.
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(request.ops_size(), 0);
   ASSERT_OPID_EQ(request.preceding_id(), MakeOpId(2, 20));
   ASSERT_OPID_EQ(request.committed_index(), committed_index);
@@ -598,8 +598,8 @@ TEST_F(ConsensusQueueTest, TestQueueHandlesOperationOverwriting) {
 
   // Generate another request for the remote peer, which should include
   // all of the ops since the peer's last-known committed index.
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_OPID_EQ(MakeOpId(1, 5), request.preceding_id());
   ASSERT_EQ(16, request.ops_size());
 
@@ -713,9 +713,9 @@ TEST_F(ConsensusQueueTest, TestOnlyAdvancesWatermarkWhenPeerHasAPrefixOfOurLog)
 
   // When we get operations for this peer we should get them starting immediately after
   // the committed index, for a total of 9 operations.
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(request.ops_size(), 9);
   ASSERT_OPID_EQ(request.ops(0).id(), MakeOpId(72, 32));
   const OpId* last_op = &request.ops(request.ops_size() - 1).id();
@@ -736,8 +736,8 @@ TEST_F(ConsensusQueueTest, TestOnlyAdvancesWatermarkWhenPeerHasAPrefixOfOurLog)
   // Another request for this peer should get another page of messages. Still not
   // on the queue's term (and thus without advancing watermarks).
   request.mutable_ops()->ExtractSubrange(0, request.ops().size(), nullptr);
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(request.ops_size(), 9);
   ASSERT_OPID_EQ(request.ops(0).id(), MakeOpId(72, 41));
   last_op = &request.ops(request.ops_size() - 1).id();
@@ -755,8 +755,8 @@ TEST_F(ConsensusQueueTest, TestOnlyAdvancesWatermarkWhenPeerHasAPrefixOfOurLog)
   // The last page of request should overwrite the peer's operations and the
   // response should finally advance the watermarks.
   request.mutable_ops()->ExtractSubrange(0, request.ops().size(), nullptr);
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
   ASSERT_EQ(request.ops_size(), 4);
   ASSERT_OPID_EQ(request.ops(0).id(), MakeOpId(73, 50));
 
@@ -774,8 +774,8 @@ TEST_F(ConsensusQueueTest, TestOnlyAdvancesWatermarkWhenPeerHasAPrefixOfOurLog)
   request.mutable_ops()->ExtractSubrange(0, request.ops().size(), nullptr);
 }
 
-// Test that remote bootstrap is triggered when a "tablet not found" error occurs.
-TEST_F(ConsensusQueueTest, TestTriggerRemoteBootstrapIfTabletNotFound) {
+// Test that tablet copy is triggered when a "tablet not found" error occurs.
+TEST_F(ConsensusQueueTest, TestTriggerTabletCopyIfTabletNotFound) {
   queue_->Init(MinimumOpId());
   queue_->SetLeaderMode(MinimumOpId(), MinimumOpId().term(), BuildRaftConfigPBForTests(3));
   AppendReplicateMessagesToQueue(queue_.get(), clock_, 1, 100);
@@ -787,9 +787,9 @@ TEST_F(ConsensusQueueTest, TestTriggerRemoteBootstrapIfTabletNotFound) {
 
   // Create request for new peer.
   vector<ReplicateRefPtr> refs;
-  bool needs_remote_bootstrap;
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_FALSE(needs_remote_bootstrap);
+  bool needs_tablet_copy;
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_FALSE(needs_tablet_copy);
 
   // Peer responds with tablet not found.
   response.mutable_error()->set_code(tserver::TabletServerErrorPB::TABLET_NOT_FOUND);
@@ -797,16 +797,16 @@ TEST_F(ConsensusQueueTest, TestTriggerRemoteBootstrapIfTabletNotFound) {
   bool more_pending = false;
   queue_->ResponseFromPeer(kPeerUuid, response, &more_pending);
 
-  // If the peer needs remote bootstrap, more_pending should be set to true.
+  // If the peer needs tablet copy, more_pending should be set to true.
   ASSERT_TRUE(more_pending);
 
   // On the next request, we should find out that the queue wants us to remotely bootstrap.
   request.Clear();
-  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_remote_bootstrap));
-  ASSERT_TRUE(needs_remote_bootstrap);
+  ASSERT_OK(queue_->RequestForPeer(kPeerUuid, &request, &refs, &needs_tablet_copy));
+  ASSERT_TRUE(needs_tablet_copy);
 
-  StartRemoteBootstrapRequestPB rb_req;
-  ASSERT_OK(queue_->GetRemoteBootstrapRequestForPeer(kPeerUuid, &rb_req));
+  StartTabletCopyRequestPB rb_req;
+  ASSERT_OK(queue_->GetTabletCopyRequestForPeer(kPeerUuid, &rb_req));
 
   ASSERT_TRUE(rb_req.IsInitialized()) << rb_req.ShortDebugString();
   ASSERT_EQ(kTestTablet, rb_req.tablet_id());

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/consensus_queue.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue.cc b/src/kudu/consensus/consensus_queue.cc
index c936b4b..83039a5 100644
--- a/src/kudu/consensus/consensus_queue.cc
+++ b/src/kudu/consensus/consensus_queue.cc
@@ -83,11 +83,11 @@ METRIC_DEFINE_gauge_int64(tablet, in_progress_ops, "Leader Operations in Progres
 std::string PeerMessageQueue::TrackedPeer::ToString() const {
   return Substitute("Peer: $0, Is new: $1, Last received: $2, Next index: $3, "
                     "Last known committed idx: $4, Last exchange result: $5, "
-                    "Needs remote bootstrap: $6",
+                    "Needs tablet copy: $6",
                     uuid, is_new, OpIdToString(last_received), next_index,
                     last_known_committed_idx,
                     is_last_exchange_successful ? "SUCCESS" : "ERROR",
-                    needs_remote_bootstrap);
+                    needs_tablet_copy);
 }
 
 #define INSTANTIATE_METRIC(x) \
@@ -275,7 +275,7 @@ Status PeerMessageQueue::AppendOperations(const vector<ReplicateRefPtr>& msgs,
 Status PeerMessageQueue::RequestForPeer(const string& uuid,
                                         ConsensusRequestPB* request,
                                         vector<ReplicateRefPtr>* msg_refs,
-                                        bool* needs_remote_bootstrap) {
+                                        bool* needs_tablet_copy) {
   TrackedPeer* peer = nullptr;
   OpId preceding_id;
   {
@@ -313,12 +313,12 @@ Status PeerMessageQueue::RequestForPeer(const string& uuid,
     }
   }
 
-  if (PREDICT_FALSE(peer->needs_remote_bootstrap)) {
-    VLOG_WITH_PREFIX_UNLOCKED(1) << "Peer needs remote bootstrap: " << peer->ToString();
-    *needs_remote_bootstrap = true;
+  if (PREDICT_FALSE(peer->needs_tablet_copy)) {
+    VLOG_WITH_PREFIX_UNLOCKED(1) << "Peer needs tablet copy: " << peer->ToString();
+    *needs_tablet_copy = true;
     return Status::OK();
   }
-  *needs_remote_bootstrap = false;
+  *needs_tablet_copy = false;
 
   // If we've never communicated with the peer, we don't know what messages to
   // send, so we'll send a status-only request. Otherwise, we grab requests
@@ -387,8 +387,8 @@ Status PeerMessageQueue::RequestForPeer(const string& uuid,
   return Status::OK();
 }
 
-Status PeerMessageQueue::GetRemoteBootstrapRequestForPeer(const string& uuid,
-                                                          StartRemoteBootstrapRequestPB* req) {
+Status PeerMessageQueue::GetTabletCopyRequestForPeer(const string& uuid,
+                                                          StartTabletCopyRequestPB* req) {
   TrackedPeer* peer = nullptr;
   {
     std::lock_guard<simple_spinlock> lock(queue_lock_);
@@ -400,7 +400,7 @@ Status PeerMessageQueue::GetRemoteBootstrapRequestForPeer(const string& uuid,
     }
   }
 
-  if (PREDICT_FALSE(!peer->needs_remote_bootstrap)) {
+  if (PREDICT_FALSE(!peer->needs_tablet_copy)) {
     return Status::IllegalState("Peer does not need to remotely bootstrap", uuid);
   }
   req->Clear();
@@ -409,7 +409,7 @@ Status PeerMessageQueue::GetRemoteBootstrapRequestForPeer(const string& uuid,
   req->set_bootstrap_peer_uuid(local_peer_pb_.permanent_uuid());
   *req->mutable_bootstrap_peer_addr() = local_peer_pb_.last_known_addr();
   req->set_caller_term(queue_state_.current_term);
-  peer->needs_remote_bootstrap = false; // Now reset the flag.
+  peer->needs_tablet_copy = false; // Now reset the flag.
   return Status::OK();
 }
 
@@ -499,8 +499,8 @@ void PeerMessageQueue::ResponseFromPeer(const std::string& peer_uuid,
       CHECK_EQ(tserver::TabletServerErrorPB::TABLET_NOT_FOUND, response.error().code())
           << response.ShortDebugString();
 
-      peer->needs_remote_bootstrap = true;
-      VLOG_WITH_PREFIX_UNLOCKED(1) << "Marked peer as needing remote bootstrap: "
+      peer->needs_tablet_copy = true;
+      VLOG_WITH_PREFIX_UNLOCKED(1) << "Marked peer as needing tablet copy: "
                                      << peer->ToString();
       *more_pending = true;
       return;

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/consensus_queue.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue.h b/src/kudu/consensus/consensus_queue.h
index 6c22f68..bba6d41 100644
--- a/src/kudu/consensus/consensus_queue.h
+++ b/src/kudu/consensus/consensus_queue.h
@@ -75,7 +75,7 @@ class PeerMessageQueue {
           last_known_committed_idx(MinimumOpId().index()),
           is_last_exchange_successful(false),
           last_successful_communication_time(MonoTime::Now(MonoTime::FINE)),
-          needs_remote_bootstrap(false),
+          needs_tablet_copy(false),
           last_seen_term_(0) {}
 
     // Check that the terms seen from a given peer only increase
@@ -112,8 +112,8 @@ class PeerMessageQueue {
     // successful communication ever took place.
     MonoTime last_successful_communication_time;
 
-    // Whether the follower was detected to need remote bootstrap.
-    bool needs_remote_bootstrap;
+    // Whether the follower was detected to need tablet copy.
+    bool needs_tablet_copy;
 
    private:
     // The last term we saw from a given peer.
@@ -193,13 +193,13 @@ class PeerMessageQueue {
   virtual Status RequestForPeer(const std::string& uuid,
                                 ConsensusRequestPB* request,
                                 std::vector<ReplicateRefPtr>* msg_refs,
-                                bool* needs_remote_bootstrap);
+                                bool* needs_tablet_copy);
 
-  // Fill in a StartRemoteBootstrapRequest for the specified peer.
+  // Fill in a StartTabletCopyRequest for the specified peer.
   // If that peer should not remotely bootstrap, returns a non-OK status.
-  // On success, also internally resets peer->needs_remote_bootstrap to false.
-  virtual Status GetRemoteBootstrapRequestForPeer(const std::string& uuid,
-                                                  StartRemoteBootstrapRequestPB* req);
+  // On success, also internally resets peer->needs_tablet_copy to false.
+  virtual 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

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/log_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_util.cc b/src/kudu/consensus/log_util.cc
index 80468e1..84043b6 100644
--- a/src/kudu/consensus/log_util.cc
+++ b/src/kudu/consensus/log_util.cc
@@ -215,7 +215,7 @@ Status LogEntryReader::HandleReadError(const Status& s) const {
   LOG(INFO) << "Ignoring log segment corruption in " << seg_->path_ << " because "
             << "there are no log entries following the corrupted one. "
             << "The server probably crashed in the middle of writing an entry "
-            << "to the write-ahead log or downloaded an active log via remote bootstrap. "
+            << "to the write-ahead log or downloaded an active log via tablet copy. "
             << "Error detail: " << corruption_status.ToString();
   return Status::EndOfFile("");
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/consensus/raft_consensus-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus-test.cc b/src/kudu/consensus/raft_consensus-test.cc
index 52dfa97..6c6819e 100644
--- a/src/kudu/consensus/raft_consensus-test.cc
+++ b/src/kudu/consensus/raft_consensus-test.cc
@@ -81,7 +81,7 @@ class MockQueue : public PeerMessageQueue {
   MOCK_METHOD4(RequestForPeer, Status(const std::string& uuid,
                                       ConsensusRequestPB* request,
                                       std::vector<ReplicateRefPtr>* msg_refs,
-                                      bool* needs_remote_bootstrap));
+                                      bool* needs_tablet_copy));
   MOCK_METHOD3(ResponseFromPeer, void(const std::string& peer_uuid,
                                       const ConsensusResponsePB& response,
                                       bool* more_pending));

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/integration-tests/client_failover-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/client_failover-itest.cc b/src/kudu/integration-tests/client_failover-itest.cc
index de32b5f..1542679 100644
--- a/src/kudu/integration-tests/client_failover-itest.cc
+++ b/src/kudu/integration-tests/client_failover-itest.cc
@@ -59,7 +59,7 @@ TEST_P(ClientFailoverParamITest, TestDeleteLeaderWhileScanning) {
   const MonoDelta kTimeout = MonoDelta::FromSeconds(30);
 
   vector<string> ts_flags = { "--enable_leader_failure_detection=false",
-                              "--enable_remote_bootstrap=false" };
+                              "--enable_tablet_copy=false" };
   vector<string> master_flags = { "--master_add_server_when_underreplicated=false",
                                   "--catalog_manager_wait_for_new_tablets_to_elect_leader=false" };
 
@@ -171,13 +171,13 @@ TEST_P(ClientFailoverParamITest, TestDeleteLeaderWhileScanning) {
                       boost::none, kTimeout));
   HostPort hp;
   ASSERT_OK(HostPortFromPB(leader->registration.rpc_addresses(0), &hp));
-  ASSERT_OK(StartRemoteBootstrap(to_add, tablet_id, leader->uuid(), hp, 1, kTimeout));
+  ASSERT_OK(StartTabletCopy(to_add, tablet_id, leader->uuid(), hp, 1, kTimeout));
 
   const string& new_ts_uuid = cluster_->tablet_server(missing_replica_index)->uuid();
   InsertOrDie(&replica_indexes, missing_replica_index);
   InsertOrDie(&active_ts_map, new_ts_uuid, ts_map_[new_ts_uuid]);
 
-  // Wait for remote bootstrap to complete. Then elect the new node.
+  // Wait for tablet copy to complete. Then elect the new node.
   ASSERT_OK(WaitForServersToAgree(kTimeout, active_ts_map, tablet_id,
                                   workload.batches_completed() + 5));
   leader_index = missing_replica_index;

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 dd8e028..e9306c6 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -774,14 +774,14 @@ Status DeleteTablet(const TServerDetails* ts,
   return Status::OK();
 }
 
-Status StartRemoteBootstrap(const TServerDetails* ts,
+Status StartTabletCopy(const TServerDetails* ts,
                             const string& tablet_id,
                             const string& bootstrap_source_uuid,
                             const HostPort& bootstrap_source_addr,
                             int64_t caller_term,
                             const MonoDelta& timeout) {
-  consensus::StartRemoteBootstrapRequestPB req;
-  consensus::StartRemoteBootstrapResponsePB resp;
+  consensus::StartTabletCopyRequestPB req;
+  consensus::StartTabletCopyResponsePB resp;
   RpcController rpc;
   rpc.set_timeout(timeout);
 
@@ -791,7 +791,7 @@ Status StartRemoteBootstrap(const TServerDetails* ts,
   RETURN_NOT_OK(HostPortToPB(bootstrap_source_addr, req.mutable_bootstrap_peer_addr()));
   req.set_caller_term(caller_term);
 
-  RETURN_NOT_OK(ts->consensus_proxy->StartRemoteBootstrap(req, &resp, &rpc));
+  RETURN_NOT_OK(ts->consensus_proxy->StartTabletCopy(req, &resp, &rpc));
   if (resp.has_error()) {
     return StatusFromPB(resp.error().status());
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 596e4a2..5ba3a8d 100644
--- a/src/kudu/integration-tests/cluster_itest_util.h
+++ b/src/kudu/integration-tests/cluster_itest_util.h
@@ -288,9 +288,9 @@ Status DeleteTablet(const TServerDetails* ts,
                     const MonoDelta& timeout,
                     tserver::TabletServerErrorPB::Code* error_code = NULL);
 
-// Cause the remote to initiate remote bootstrap using the specified host as a
+// Cause the remote to initiate tablet copy using the specified host as a
 // source.
-Status StartRemoteBootstrap(const TServerDetails* ts,
+Status StartTabletCopy(const TServerDetails* ts,
                             const std::string& tablet_id,
                             const std::string& bootstrap_source_uuid,
                             const HostPort& bootstrap_source_addr,

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/integration-tests/delete_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-test.cc b/src/kudu/integration-tests/delete_table-test.cc
index 6a0d349..272efed 100644
--- a/src/kudu/integration-tests/delete_table-test.cc
+++ b/src/kudu/integration-tests/delete_table-test.cc
@@ -413,13 +413,13 @@ TEST_F(DeleteTableTest, TestDeleteTableWithConcurrentWrites) {
 }
 
 // Test that a tablet replica is automatically tombstoned on startup if a local
-// crash occurs in the middle of remote bootstrap.
-TEST_F(DeleteTableTest, TestAutoTombstoneAfterCrashDuringRemoteBootstrap) {
+// crash occurs in the middle of tablet copy.
+TEST_F(DeleteTableTest, TestAutoTombstoneAfterCrashDuringTabletCopy) {
   NO_FATALS(StartCluster());
   const MonoDelta timeout = MonoDelta::FromSeconds(10);
   const int kTsIndex = 0; // We'll test with the first TS.
 
-  // We'll do a config change to remote bootstrap a replica here later. For
+  // We'll do a config change to tablet copy a replica here later. For
   // now, shut it down.
   LOG(INFO) << "Shutting down TS " << cluster_->tablet_server(kTsIndex)->uuid();
   cluster_->tablet_server(kTsIndex)->Shutdown();
@@ -442,12 +442,12 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterCrashDuringRemoteBootstrap) {
   }
   workload.StopAndJoin();
 
-  // Enable a fault crash when remote bootstrap occurs on TS 0.
+  // Enable a fault crash when tablet copy occurs on TS 0.
   ASSERT_OK(cluster_->tablet_server(kTsIndex)->Restart());
   const string& kFaultFlag = "fault_crash_after_rb_files_fetched";
   ASSERT_OK(cluster_->SetFlag(cluster_->tablet_server(kTsIndex), kFaultFlag, "1.0"));
 
-  // Figure out the tablet id to remote bootstrap.
+  // Figure out the tablet id to tablet copy.
   vector<string> tablets = inspect_->ListTabletsOnTS(1);
   ASSERT_EQ(1, tablets.size());
   const string& tablet_id = tablets[0];
@@ -467,7 +467,7 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterCrashDuringRemoteBootstrap) {
   cluster_->tablet_server(1)->Shutdown();
   cluster_->tablet_server(2)->Shutdown();
 
-  // Now we restart the TS. It will clean up the failed remote bootstrap and
+  // Now we restart the TS. It will clean up the failed tablet copy and
   // convert it to TABLET_DATA_TOMBSTONED. It crashed, so we have to call
   // Shutdown() then Restart() to bring it back up.
   cluster_->tablet_server(kTsIndex)->Shutdown();
@@ -476,9 +476,9 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterCrashDuringRemoteBootstrap) {
 }
 
 // Test that a tablet replica automatically tombstones itself if the remote
-// bootstrap source server fails in the middle of the remote bootstrap process.
+// bootstrap source server fails in the middle of the tablet copy process.
 // Also test that we can remotely bootstrap a tombstoned tablet.
-TEST_F(DeleteTableTest, TestAutoTombstoneAfterRemoteBootstrapRemoteFails) {
+TEST_F(DeleteTableTest, TestAutoTombstoneAfterTabletCopyRemoteFails) {
   vector<string> ts_flags = {
       "--enable_leader_failure_detection=false",  // Make test deterministic.
       "--log_segment_size_mb=1"                   // Faster log rolls.
@@ -490,7 +490,7 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterRemoteBootstrapRemoteFails) {
   const MonoDelta kTimeout = MonoDelta::FromSeconds(20);
   const int kTsIndex = 0; // We'll test with the first TS.
 
-  // We'll do a config change to remote bootstrap a replica here later. For
+  // We'll do a config change to tablet copy a replica here later. For
   // now, shut down TS-0.
   LOG(INFO) << "Shutting down TS " << cluster_->tablet_server(kTsIndex)->uuid();
   cluster_->tablet_server(kTsIndex)->Shutdown();
@@ -525,7 +525,7 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterRemoteBootstrapRemoteFails) {
     SleepFor(MonoDelta::FromMilliseconds(10));
   }
 
-  // Remote bootstrap doesn't see the active WAL segment, and we need to
+  // Tablet Copy doesn't see the active WAL segment, and we need to
   // download a file to trigger the fault in this test. Due to the log index
   // chunks, that means 3 files minimum: One in-flight WAL segment, one index
   // chunk file (these files grow much more slowly than the WAL segments), and
@@ -552,7 +552,7 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterRemoteBootstrapRemoteFails) {
   NO_FATALS(WaitForTabletTombstonedOnTS(kTsIndex, tablet_id, CMETA_NOT_EXPECTED));
 
   // Now bring the other replicas back, re-elect the previous leader (TS-1),
-  // and wait for the leader to remote bootstrap the tombstoned replica. This
+  // and wait for the leader to tablet copy the tombstoned replica. This
   // will have replaced a tablet with no consensus metadata.
   ASSERT_OK(cluster_->tablet_server(1)->Restart());
   ASSERT_OK(cluster_->tablet_server(2)->Restart());
@@ -584,7 +584,7 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterRemoteBootstrapRemoteFails) {
                             workload.rows_inserted()));
 }
 
-// Test for correct remote bootstrap merge of consensus metadata.
+// Test for correct tablet copy merge of consensus metadata.
 TEST_F(DeleteTableTest, TestMergeConsensusMetadata) {
   // Enable manual leader selection.
   vector<string> ts_flags, master_flags;
@@ -598,7 +598,7 @@ TEST_F(DeleteTableTest, TestMergeConsensusMetadata) {
   workload.Setup();
   ASSERT_OK(inspect_->WaitForReplicaCount(3));
 
-  // Figure out the tablet id to remote bootstrap.
+  // Figure out the tablet id to tablet copy.
   vector<string> tablets = inspect_->ListTabletsOnTS(1);
   ASSERT_EQ(1, tablets.size());
   const string& tablet_id = tablets[0];
@@ -658,7 +658,7 @@ TEST_F(DeleteTableTest, TestMergeConsensusMetadata) {
   ASSERT_OK(itest::WaitUntilLeader(leader, tablet_id, timeout));
 
   // Bring our special little guy back up.
-  // Wait until he gets remote bootstrapped.
+  // Wait until he gets tablet copyped.
   LOG(INFO) << "Bringing TS " << cluster_->tablet_server(kTsIndex)->uuid()
             << " back up...";
   ASSERT_OK(cluster_->tablet_server(kTsIndex)->Restart());
@@ -670,9 +670,9 @@ TEST_F(DeleteTableTest, TestMergeConsensusMetadata) {
   ASSERT_EQ(ts->uuid(), cmeta_pb.voted_for());
 
   // Now do the same thing as above, where we tombstone TS 0 then trigger a new
-  // term (term 3) on the other machines. TS 0 will get remotely bootstrapped
+  // term (term 3) on the other machines. TS 0 will get copied
   // again, but this time the vote record on TS 0 for term 2 should not be
-  // retained after remote bootstrap occurs.
+  // retained after tablet copy occurs.
   cluster_->tablet_server(1)->Shutdown();
   cluster_->tablet_server(2)->Shutdown();
 
@@ -1047,7 +1047,7 @@ TEST_P(DeleteTableTombstonedParamTest, TestTabletTombstone) {
   workload.StopAndJoin();
 
   // Shut down the master and the other tablet servers so they don't interfere
-  // by attempting to create tablets or remote bootstrap while we delete tablets.
+  // by attempting to create tablets or tablet copy while we delete tablets.
   cluster_->master()->Shutdown();
   cluster_->tablet_server(1)->Shutdown();
   cluster_->tablet_server(2)->Shutdown();

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 467d222..4a1d5f7 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -2090,7 +2090,7 @@ TEST_F(RaftConsensusITest, TestEarlyCommitDespiteMemoryPressure) {
   WaitForRowCount(replica_ts->tserver_proxy.get(), kNumOps, &rows);
 }
 
-// Test that we can create (vivify) a new tablet via remote bootstrap.
+// Test that we can create (vivify) a new tablet via tablet copy.
 TEST_F(RaftConsensusITest, TestAutoCreateReplica) {
   FLAGS_num_tablet_servers = 3;
   FLAGS_num_replicas = 2;
@@ -2481,7 +2481,7 @@ TEST_F(RaftConsensusITest, TestMasterReplacesEvictedFollowers) {
   NO_FATALS(CauseFollowerToFallBehindLogGC(&leader_uuid, &orig_term, &follower_uuid));
 
   // The follower will be evicted. Now wait for the master to cause it to be
-  // remotely bootstrapped.
+  // copied.
   ASSERT_OK(WaitForServersToAgree(timeout, tablet_servers_, tablet_id_, 2));
 
   ClusterVerifier v(cluster_.get());

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/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 9c9d0c7..0ada5fc 100644
--- a/src/kudu/integration-tests/remote_bootstrap-itest.cc
+++ b/src/kudu/integration-tests/remote_bootstrap-itest.cc
@@ -40,13 +40,13 @@
 #include "kudu/util/test_util.h"
 
 DEFINE_int32(test_delete_leader_num_iters, 3,
-             "Number of iterations to run in TestDeleteLeaderDuringRemoteBootstrapStressTest.");
+             "Number of iterations to run in TestDeleteLeaderDuringTabletCopyStressTest.");
 DEFINE_int32(test_delete_leader_min_rows_per_iter, 20,
-             "Number of writer threads in TestDeleteLeaderDuringRemoteBootstrapStressTest.");
+             "Number of writer threads in TestDeleteLeaderDuringTabletCopyStressTest.");
 DEFINE_int32(test_delete_leader_payload_bytes, 16 * 1024,
-             "Payload byte size in TestDeleteLeaderDuringRemoteBootstrapStressTest.");
+             "Payload byte size in TestDeleteLeaderDuringTabletCopyStressTest.");
 DEFINE_int32(test_delete_leader_num_writer_threads, 1,
-             "Number of writer threads in TestDeleteLeaderDuringRemoteBootstrapStressTest.");
+             "Number of writer threads in TestDeleteLeaderDuringTabletCopyStressTest.");
 
 using kudu::client::KuduClient;
 using kudu::client::KuduClientBuilder;
@@ -59,7 +59,7 @@ using kudu::itest::TServerDetails;
 using kudu::tablet::TABLET_DATA_DELETED;
 using kudu::tablet::TABLET_DATA_TOMBSTONED;
 using kudu::tserver::ListTabletsResponsePB;
-using kudu::tserver::RemoteBootstrapClient;
+using kudu::tserver::TabletCopyClient;
 using std::string;
 using std::unordered_map;
 using std::vector;
@@ -73,7 +73,7 @@ METRIC_DECLARE_counter(glog_error_messages);
 
 namespace kudu {
 
-class RemoteBootstrapITest : public KuduTest {
+class TabletCopyITest : public KuduTest {
  public:
   virtual void TearDown() OVERRIDE {
     if (HasFatalFailure()) {
@@ -106,7 +106,7 @@ class RemoteBootstrapITest : public KuduTest {
   unordered_map<string, TServerDetails*> ts_map_;
 };
 
-void RemoteBootstrapITest::StartCluster(const vector<string>& extra_tserver_flags,
+void TabletCopyITest::StartCluster(const vector<string>& extra_tserver_flags,
                                         const vector<string>& extra_master_flags,
                                         int num_tablet_servers) {
   ExternalMiniClusterOptions opts;
@@ -127,7 +127,7 @@ void RemoteBootstrapITest::StartCluster(const vector<string>& extra_tserver_flag
   ASSERT_OK(cluster_->CreateClient(builder, &client_));
 }
 
-// If a rogue (a.k.a. zombie) leader tries to remote bootstrap a tombstoned
+// If a rogue (a.k.a. zombie) leader tries to tablet copy a tombstoned
 // tablet, make sure its term isn't older than the latest term we observed.
 // If it is older, make sure we reject the request, to avoid allowing old
 // leaders to create a parallel universe. This is possible because config
@@ -135,7 +135,7 @@ void RemoteBootstrapITest::StartCluster(const vector<string>& extra_tserver_flag
 // because only one node can be elected leader for a given term.
 //
 // A leader can "go rogue" due to a VM pause, CTRL-z, partition, etc.
-TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
+TEST_F(TabletCopyITest, TestRejectRogueLeader) {
   // This test pauses for at least 10 seconds. Only run in slow-test mode.
   if (!AllowSlowTests()) {
     LOG(INFO) << "Skipping test in fast-test mode.";
@@ -180,7 +180,7 @@ TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
   // Come out of the blue and try to remotely bootstrap a running server while
   // specifying an old term. That running server should reject the request.
   // We are essentially masquerading as a rogue leader here.
-  Status s = itest::StartRemoteBootstrap(ts, tablet_id, zombie_leader_uuid,
+  Status s = itest::StartTabletCopy(ts, tablet_id, zombie_leader_uuid,
                                          HostPort(cluster_->tablet_server(1)->bound_rpc_addr()),
                                          0, // Say I'm from term 0.
                                          timeout);
@@ -201,7 +201,7 @@ TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
 
   // Wait for the NO_OP entry from the term 2 election to propagate to the
   // remaining nodes' logs so that we are guaranteed to reject the rogue
-  // leader's remote bootstrap request when we bring it back online.
+  // leader's tablet copy request when we bring it back online.
   int log_index = workload.batches_completed() + 2; // 2 terms == 2 additional NO_OP entries.
   ASSERT_OK(WaitForServersToAgree(timeout, active_ts_map, tablet_id, log_index));
   // TODO: Write more rows to the new leader once KUDU-1034 is fixed.
@@ -211,7 +211,7 @@ TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
   ASSERT_OK(itest::DeleteTablet(ts, tablet_id, TABLET_DATA_TOMBSTONED, boost::none, timeout));
 
   // Zombies!!! Resume the rogue zombie leader.
-  // He should attempt to remote bootstrap TS 0 but fail.
+  // He should attempt to tablet copy TS 0 but fail.
   ASSERT_OK(cluster_->tablet_server(zombie_leader_index)->Resume());
 
   // Loop for a few seconds to ensure that the tablet doesn't transition to READY.
@@ -225,7 +225,7 @@ TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
   }
 
   // Force the rogue leader to step down.
-  // Then, send a remote bootstrap start request from a "fake" leader that
+  // Then, send a tablet copy start request from a "fake" leader that
   // sends an up-to-date term in the RB request but the actual term stored
   // in the bootstrap source's consensus metadata would still be old.
   LOG(INFO) << "Forcing rogue leader T " << tablet_id << " P " << zombie_leader_uuid
@@ -235,12 +235,12 @@ TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
   // It's not necessarily part of the API but this could return faliure due to
   // rejecting the remote. We intend to make that part async though, so ignoring
   // this return value in this test.
-  ignore_result(itest::StartRemoteBootstrap(ts, tablet_id, zombie_leader_uuid,
+  ignore_result(itest::StartTabletCopy(ts, tablet_id, zombie_leader_uuid,
                                             HostPort(zombie_ets->bound_rpc_addr()),
                                             2, // Say I'm from term 2.
                                             timeout));
 
-  // Wait another few seconds to be sure the remote bootstrap is rejected.
+  // Wait another few seconds to be sure the tablet copy is rejected.
   deadline = MonoTime::Now(MonoTime::FINE);
   deadline.AddDelta(MonoDelta::FromSeconds(5));
   while (MonoTime::Now(MonoTime::FINE).ComesBefore(deadline)) {
@@ -251,13 +251,13 @@ TEST_F(RemoteBootstrapITest, TestRejectRogueLeader) {
   }
 }
 
-// Start remote bootstrap session and delete the tablet in the middle.
+// Start tablet copy session and delete the tablet in the middle.
 // It should actually be possible to complete bootstrap in such a case, because
-// when a remote bootstrap session is started on the "source" server, all of
+// when a tablet copy session is started on the "source" server, all of
 // the relevant files are either read or opened, meaning that an in-progress
-// remote bootstrap can complete even after a tablet is officially "deleted" on
+// tablet copy can complete even after a tablet is officially "deleted" on
 // the source server. This is also a regression test for KUDU-1009.
-TEST_F(RemoteBootstrapITest, TestDeleteTabletDuringRemoteBootstrap) {
+TEST_F(TabletCopyITest, TestDeleteTabletDuringTabletCopy) {
   MonoDelta timeout = MonoDelta::FromSeconds(10);
   const int kTsIndex = 0; // We'll test with the first TS.
   NO_FATALS(StartCluster());
@@ -280,7 +280,7 @@ TEST_F(RemoteBootstrapITest, TestDeleteTabletDuringRemoteBootstrap) {
   workload.StopAndJoin();
   ASSERT_OK(WaitForServersToAgree(timeout, ts_map_, tablet_id, workload.batches_completed()));
 
-  // Set up an FsManager to use with the RemoteBootstrapClient.
+  // Set up an FsManager to use with the TabletCopyClient.
   FsManagerOpts opts;
   string testbase = GetTestPath("fake-ts");
   ASSERT_OK(env_->CreateDir(testbase));
@@ -291,8 +291,8 @@ TEST_F(RemoteBootstrapITest, TestDeleteTabletDuringRemoteBootstrap) {
   ASSERT_OK(fs_manager->Open());
 
   {
-    // Start up a RemoteBootstrapClient and open a remote bootstrap session.
-    RemoteBootstrapClient rb_client(tablet_id, fs_manager.get(),
+    // Start up a TabletCopyClient and open a tablet copy session.
+    TabletCopyClient rb_client(tablet_id, fs_manager.get(),
                                     cluster_->messenger());
     scoped_refptr<tablet::TabletMetadata> meta;
     ASSERT_OK(rb_client.Start(cluster_->tablet_server(kTsIndex)->bound_rpc_hostport(),
@@ -317,9 +317,9 @@ TEST_F(RemoteBootstrapITest, TestDeleteTabletDuringRemoteBootstrap) {
 // This test ensures that a leader can remote-bootstrap a tombstoned replica
 // that has a higher term recorded in the replica's consensus metadata if the
 // replica's last-logged opid has the same term (or less) as the leader serving
-// as the remote bootstrap source. When a tablet is tombstoned, its last-logged
+// as the tablet copy source. When a tablet is tombstoned, its last-logged
 // opid is stored in a field its on-disk superblock.
-TEST_F(RemoteBootstrapITest, TestRemoteBootstrapFollowerWithHigherTerm) {
+TEST_F(TabletCopyITest, TestTabletCopyFollowerWithHigherTerm) {
   vector<string> ts_flags, master_flags;
   ts_flags.push_back("--enable_leader_failure_detection=false");
   master_flags.push_back("--catalog_manager_wait_for_new_tablets_to_elect_leader=false");
@@ -383,18 +383,18 @@ TEST_F(RemoteBootstrapITest, TestRemoteBootstrapFollowerWithHigherTerm) {
   ASSERT_OK(cluster_->tablet_server(kFollowerIndex)->Restart());
 
   // Now wake the leader. It should detect that the follower needs to be
-  // remotely bootstrapped and proceed to bring it back up to date.
+  // copied and proceed to bring it back up to date.
   ASSERT_OK(cluster_->tablet_server(kLeaderIndex)->Resume());
 
   // Wait for the follower to come back up.
   ASSERT_OK(WaitForServersToAgree(timeout, ts_map_, tablet_id, workload.batches_completed()));
 }
 
-// Test that multiple concurrent remote bootstraps do not cause problems.
+// Test that multiple concurrent tablet copys do not cause problems.
 // This is a regression test for KUDU-951, in which concurrent sessions on
-// multiple tablets between the same remote bootstrap client host and remote
+// multiple tablets between the same tablet copy client host and remote
 // bootstrap source host could corrupt each other.
-TEST_F(RemoteBootstrapITest, TestConcurrentRemoteBootstraps) {
+TEST_F(TabletCopyITest, TestConcurrentTabletCopys) {
   if (!AllowSlowTests()) {
     LOG(INFO) << "Skipping test in fast-test mode.";
     return;
@@ -414,7 +414,7 @@ TEST_F(RemoteBootstrapITest, TestConcurrentRemoteBootstraps) {
   const MonoDelta timeout = MonoDelta::FromSeconds(60);
 
   // Create a table with several tablets. These will all be simultaneously
-  // remotely bootstrapped to a single target node from the same leader host.
+  // copied to a single target node from the same leader host.
   const int kNumTablets = 10;
   KuduSchema client_schema(KuduSchemaFromSchema(GetSimpleTestSchema()));
   vector<const KuduPartialRow*> splits;
@@ -497,9 +497,9 @@ TEST_F(RemoteBootstrapITest, TestConcurrentRemoteBootstraps) {
 }
 
 // Test that repeatedly runs a load, tombstones a follower, then tombstones the
-// leader while the follower is remotely bootstrapping. Regression test for
+// leader while the follower is copying. Regression test for
 // KUDU-1047.
-TEST_F(RemoteBootstrapITest, TestDeleteLeaderDuringRemoteBootstrapStressTest) {
+TEST_F(TabletCopyITest, TestDeleteLeaderDuringTabletCopyStressTest) {
   // This test takes a while due to failure detection.
   if (!AllowSlowTests()) {
     LOG(INFO) << "Skipping test in fast-test mode.";
@@ -561,7 +561,7 @@ TEST_F(RemoteBootstrapITest, TestDeleteLeaderDuringRemoteBootstrapStressTest) {
     ASSERT_OK(itest::DeleteTablet(follower_ts, tablet_id, TABLET_DATA_TOMBSTONED, boost::none,
                                   timeout));
 
-    // Wait for remote bootstrap to start.
+    // Wait for tablet copy to start.
     ASSERT_OK(inspect_->WaitForTabletDataStateOnTS(
         follower_index, tablet_id,
         { tablet::TABLET_DATA_COPYING, tablet::TABLET_DATA_READY },
@@ -574,7 +574,7 @@ TEST_F(RemoteBootstrapITest, TestDeleteLeaderDuringRemoteBootstrapStressTest) {
 
     // Quiesce and rebuild to full strength. This involves electing a new
     // leader from the remaining three, which requires a unanimous vote, and
-    // that leader then remotely bootstrapping the old leader.
+    // that leader then copying the old leader.
     workload.StopAndJoin();
     ASSERT_OK(WaitForServersToAgree(timeout, ts_map_, tablet_id, 1));
   }
@@ -628,15 +628,15 @@ int64_t CountLogMessages(ExternalTabletServer* ets) {
 }
 } // anonymous namespace
 
-// Test that if remote bootstrap is disabled by a flag, we don't get into
+// Test that if tablet copy is disabled by a flag, we don't get into
 // tight loops after a tablet is deleted. This is a regression test for situation
 // similar to the bug described in KUDU-821: we were previously handling a missing
 // tablet within consensus in such a way that we'd immediately send another RPC.
-TEST_F(RemoteBootstrapITest, TestDisableRemoteBootstrap_NoTightLoopWhenTabletDeleted) {
+TEST_F(TabletCopyITest, TestDisableTabletCopy_NoTightLoopWhenTabletDeleted) {
   MonoDelta timeout = MonoDelta::FromSeconds(10);
   vector<string> ts_flags, master_flags;
   ts_flags.push_back("--enable_leader_failure_detection=false");
-  ts_flags.push_back("--enable_remote_bootstrap=false");
+  ts_flags.push_back("--enable_tablet_copy=false");
   master_flags.push_back("--catalog_manager_wait_for_new_tablets_to_elect_leader=false");
   NO_FATALS(StartCluster(ts_flags, master_flags));
 
@@ -688,13 +688,13 @@ 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,
+// Test that if a tablet copy is taking a long time but the client peer is still responsive,
 // the leader won't mark it as failed.
-TEST_F(RemoteBootstrapITest, TestSlowBootstrapDoesntFail) {
+TEST_F(TabletCopyITest, TestSlowBootstrapDoesntFail) {
   MonoDelta timeout = MonoDelta::FromSeconds(30);
   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("--tablet_copy_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));
@@ -732,7 +732,7 @@ TEST_F(RemoteBootstrapITest, TestSlowBootstrapDoesntFail) {
   ASSERT_OK(itest::DeleteTablet(replica_ts, tablet_id, TABLET_DATA_TOMBSTONED, boost::none,
                                 timeout));
 
-  // Wait for remote bootstrap to start.
+  // Wait for tablet copy to start.
   ASSERT_OK(inspect_->WaitForTabletDataStateOnTS(1, tablet_id,
                                                  { tablet::TABLET_DATA_COPYING }, timeout));
 
@@ -745,11 +745,11 @@ TEST_F(RemoteBootstrapITest, TestSlowBootstrapDoesntFail) {
                             workload.rows_inserted()));
 }
 
-// Attempting to start remote bootstrap on a tablet that was deleted with
+// Attempting to start tablet copy on a tablet that was deleted with
 // TABLET_DATA_DELETED should fail. This behavior helps avoid thrashing when
 // a follower tablet is deleted and the leader notices before it has processed
 // its own DeleteTablet RPC, thinking that it needs to bring its follower back.
-TEST_F(RemoteBootstrapITest, TestRemoteBootstrappingDeletedTabletFails) {
+TEST_F(TabletCopyITest, TestTabletCopypingDeletedTabletFails) {
   // Delete the leader with TABLET_DATA_DELETED
   // Attempt to manually bootstrap the leader from a follower
   // Should get an error saying it's illegal
@@ -783,7 +783,7 @@ TEST_F(RemoteBootstrapITest, TestRemoteBootstrappingDeletedTabletFails) {
   // bring back the leader after that until restarting the process.
   ASSERT_OK(itest::DeleteTablet(leader, tablet_id, TABLET_DATA_DELETED, boost::none, kTimeout));
 
-  Status s = itest::StartRemoteBootstrap(leader, tablet_id,
+  Status s = itest::StartTabletCopy(leader, tablet_id,
                                          cluster_->tablet_server(1)->uuid(),
                                          HostPort(cluster_->tablet_server(1)->bound_rpc_addr()),
                                          1, // We are in term 1.
@@ -792,11 +792,11 @@ TEST_F(RemoteBootstrapITest, TestRemoteBootstrappingDeletedTabletFails) {
   ASSERT_STR_CONTAINS(s.ToString(), "Cannot transition from state TABLET_DATA_DELETED");
 
   // Restart the server so that it won't remember the tablet was permanently
-  // deleted and we can remote bootstrap the server again.
+  // deleted and we can tablet copy the server again.
   cluster_->tablet_server(0)->Shutdown();
   ASSERT_OK(cluster_->tablet_server(0)->Restart());
 
-  ASSERT_OK(itest::StartRemoteBootstrap(leader, tablet_id,
+  ASSERT_OK(itest::StartTabletCopy(leader, tablet_id,
                                         cluster_->tablet_server(1)->uuid(),
                                         HostPort(cluster_->tablet_server(1)->bound_rpc_addr()),
                                         1, // We are in term 1.

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/integration-tests/tablet_replacement-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tablet_replacement-itest.cc b/src/kudu/integration-tests/tablet_replacement-itest.cc
index 7418238..e12cdef 100644
--- a/src/kudu/integration-tests/tablet_replacement-itest.cc
+++ b/src/kudu/integration-tests/tablet_replacement-itest.cc
@@ -219,7 +219,7 @@ TEST_F(TabletReplacementITest, TestEvictAndReplaceDeadFollower) {
 // bootstrap will attempt to replay committed (and applied) config change
 // operations. This is achieved by delaying application of a write at the
 // tablet level that precedes the config change operations in the WAL, then
-// initiating a remote bootstrap to a follower. The follower will not have the
+// initiating a tablet copy to a follower. The follower will not have the
 // COMMIT for the write operation, so will ignore COMMIT messages for the
 // applied config change operations. At startup time, the newly
 // remotely-bootstrapped tablet should detect that these config change

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 88382df..babe43c 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -186,7 +186,7 @@ using consensus::ConsensusStatePB;
 using consensus::GetConsensusRole;
 using consensus::OpId;
 using consensus::RaftPeerPB;
-using consensus::StartRemoteBootstrapRequestPB;
+using consensus::StartTabletCopyRequestPB;
 using rpc::RpcContext;
 using strings::Substitute;
 using tablet::TABLET_DATA_DELETED;
@@ -2088,10 +2088,10 @@ const NodeInstancePB& CatalogManager::NodeInstance() const {
   return master_->instance_pb();
 }
 
-Status CatalogManager::StartRemoteBootstrap(
-    const StartRemoteBootstrapRequestPB& req,
+Status CatalogManager::StartTabletCopy(
+    const StartTabletCopyRequestPB& req,
     boost::optional<kudu::tserver::TabletServerErrorPB::Code>* error_code) {
-  return Status::NotSupported("Remote bootstrap not yet implemented for the master tablet");
+  return Status::NotSupported("Tablet Copy not yet implemented for the master tablet");
 }
 
 // Interface used by RetryingTSRpcTask to pick the tablet server to

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 29119f5..36a8f0c 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -500,8 +500,8 @@ class CatalogManager : public tserver::TabletPeerLookupIf {
 
   bool IsInitialized() const;
 
-  virtual Status StartRemoteBootstrap(
-      const consensus::StartRemoteBootstrapRequestPB& req,
+  virtual Status StartTabletCopy(
+      const consensus::StartTabletCopyRequestPB& req,
       boost::optional<kudu::tserver::TabletServerErrorPB::Code>* error_code) OVERRIDE;
 
   // Returns this CatalogManager's role in a consensus configuration. CatalogManager

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tablet/metadata.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/metadata.proto b/src/kudu/tablet/metadata.proto
index eb80611..5fae07a 100644
--- a/src/kudu/tablet/metadata.proto
+++ b/src/kudu/tablet/metadata.proto
@@ -53,9 +53,9 @@ enum TabletDataState {
   TABLET_DATA_UNKNOWN = 999;
 
   // The tablet is set to TABLET_DATA_COPYING state when in the middle of
-  // remote bootstrap while copying data files from a remote peer. If a tablet
+  // tablet copy while copying data files from a remote peer. If a tablet
   // server crashes with a tablet in this state, the tablet must be deleted and
-  // the remote bootstrap process must be restarted for that tablet.
+  // the tablet copy process must be restarted for that tablet.
   TABLET_DATA_COPYING = 0;
 
   // Fresh empty tablets and successfully copied tablets are set to the

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tablet/tablet_bootstrap-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc
index 17de026..dd211a8 100644
--- a/src/kudu/tablet/tablet_bootstrap-test.cc
+++ b/src/kudu/tablet/tablet_bootstrap-test.cc
@@ -179,8 +179,8 @@ TEST_F(BootstrapTest, TestBootstrap) {
 }
 
 // Tests attempting a local bootstrap of a tablet that was in the middle of a
-// remote bootstrap before "crashing".
-TEST_F(BootstrapTest, TestIncompleteRemoteBootstrap) {
+// tablet copy before "crashing".
+TEST_F(BootstrapTest, TestIncompleteTabletCopy) {
   ASSERT_OK(BuildLog());
 
   ASSERT_OK(PersistTestTabletMetadataState(TABLET_DATA_COPYING));

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index 0fe9e09..300ac75 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -153,7 +153,7 @@ class FlushedStoresSnapshot {
 // has been appointed LEADER of that particular consensus configuration.
 //
 // NOTE: this does not handle pulling data from other replicas in the cluster. That
-// is handled by the 'RemoteBootstrap' classes, which copy blocks and metadata locally
+// is handled by the 'TabletCopy' classes, which copy blocks and metadata locally
 // before invoking this local bootstrap functionality.
 //
 // TODO Because the table that is being rebuilt is never flushed/compacted, consensus
@@ -468,7 +468,7 @@ Status TabletBootstrap::Bootstrap(shared_ptr<Tablet>* rebuilt_tablet,
                         "Unable to load Consensus metadata");
 
   // Make sure we don't try to locally bootstrap a tablet that was in the middle
-  // of a remote bootstrap. It's likely that not all files were copied over
+  // of a tablet copy. It's likely that not all files were copied over
   // successfully.
   TabletDataState tablet_data_state = meta_->tablet_data_state();
   if (tablet_data_state != TABLET_DATA_READY) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tablet/tablet_metadata.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_metadata.h b/src/kudu/tablet/tablet_metadata.h
index 1d373c1..eea8aa2 100644
--- a/src/kudu/tablet/tablet_metadata.h
+++ b/src/kudu/tablet/tablet_metadata.h
@@ -132,7 +132,7 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
     return partition_schema_;
   }
 
-  // Set / get the remote bootstrap / tablet data state.
+  // Set / get the tablet copy / tablet data state.
   void set_tablet_data_state(TabletDataState state);
   TabletDataState tablet_data_state() const;
 
@@ -323,7 +323,7 @@ class TabletMetadata : public RefCountedThreadSafe<TabletMetadata> {
   // Protected by 'data_lock_'.
   std::unordered_set<BlockId, BlockIdHash, BlockIdEqual> orphaned_blocks_;
 
-  // The current state of remote bootstrap for the tablet.
+  // The current state of tablet copy for the tablet.
   TabletDataState tablet_data_state_;
 
   // Record of the last opid logged by the tablet before it was last

http://git-wip-us.apache.org/repos/asf/kudu/blob/a6cb942e/src/kudu/tablet/tablet_peer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_peer.cc b/src/kudu/tablet/tablet_peer.cc
index 013653e..c8e3646 100644
--- a/src/kudu/tablet/tablet_peer.cc
+++ b/src/kudu/tablet/tablet_peer.cc
@@ -390,7 +390,7 @@ string TabletPeer::HumanReadableState() const {
     return Substitute("$0 ($1): $2", TabletStatePB_Name(state_),
                       TabletDataState_Name(data_state),
                       error_.ToString());
-  // If it's remotely bootstrapping, or tombstoned, that is the important thing
+  // If it's copying, or tombstoned, that is the important thing
   // to show.
   } else if (data_state != TABLET_DATA_READY) {
     return TabletDataState_Name(data_state);