You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/01/31 03:17:10 UTC

kudu git commit: KUDU-1853. Tablet copy: Don't orphan blocks on failure

Repository: kudu
Updated Branches:
  refs/heads/master e87fb7907 -> 72541b47e


KUDU-1853. Tablet copy: Don't orphan blocks on failure

Previously, if a tablet copy failed we would orphan data blocks. This
patch makes it so that a failed tablet copy operation that does not
involve a process crash does not orphan data blocks.

This also refactors some deletion logic out of TSTabletManager so that
TabletCopyClient will tombstone partially-copied tablets when the copy
operation fails.

Change-Id: Ifbc0720d460de8d912a241a5186ef6e8f524b830
Reviewed-on: http://gerrit.cloudera.org:8080/5799
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 72541b47eb55b2df4eab5d6050f517476ed6d370
Parents: e87fb79
Author: Mike Percy <mp...@apache.org>
Authored: Wed Jan 25 20:10:19 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Jan 31 03:16:53 2017 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/delete_table-test.cc |   4 +-
 src/kudu/tserver/tablet_copy-test-base.h        |  16 ++-
 src/kudu/tserver/tablet_copy_client-test.cc     |  73 +++++++++-
 src/kudu/tserver/tablet_copy_client.cc          | 135 ++++++++++++-------
 src/kudu/tserver/tablet_copy_client.h           |  39 ++++--
 src/kudu/tserver/tablet_copy_service-test.cc    |   6 +-
 src/kudu/tserver/tablet_service.cc              |   9 +-
 src/kudu/tserver/ts_tablet_manager.cc           |  67 ++++-----
 src/kudu/tserver/ts_tablet_manager.h            |   7 -
 9 files changed, 224 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/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 1ac83b9..15114a3 100644
--- a/src/kudu/integration-tests/delete_table-test.cc
+++ b/src/kudu/integration-tests/delete_table-test.cc
@@ -649,8 +649,8 @@ TEST_F(DeleteTableTest, TestAutoTombstoneAfterTabletCopyRemoteFails) {
   {
     vector<ListTabletsResponsePB::StatusAndSchemaPB> status_pbs;
     ASSERT_OK(WaitForNumTabletsOnTS(ts, 1, kTimeout, &status_pbs));
-    ASSERT_STR_CONTAINS(status_pbs[0].tablet_status().last_status(),
-                        "Tombstoned tablet: Tablet Copy: Unable to fetch data from remote peer");
+    ASSERT_STR_MATCHES(status_pbs[0].tablet_status().last_status(),
+                       "Tablet Copy: Tombstoned tablet .*: Tablet copy aborted");
   }
 
   // Now bring the other replicas back, re-elect the previous leader (TS-1),

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/tablet_copy-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy-test-base.h b/src/kudu/tserver/tablet_copy-test-base.h
index bdd4a42..0a634f6 100644
--- a/src/kudu/tserver/tablet_copy-test-base.h
+++ b/src/kudu/tserver/tablet_copy-test-base.h
@@ -60,11 +60,17 @@ class TabletCopyTest : public TabletServerTestBase {
 
  protected:
   // Grab the first column block we find in the SuperBlock.
-  static BlockId FirstColumnBlockId(const tablet::TabletSuperBlockPB& superblock) {
-    const tablet::RowSetDataPB& rowset = superblock.rowsets(0);
-    const tablet::ColumnDataPB& column = rowset.columns(0);
-    const BlockIdPB& block_id_pb = column.block();
-    return BlockId::FromPB(block_id_pb);
+  static BlockIdPB* FirstColumnBlockIdPB(tablet::TabletSuperBlockPB* superblock) {
+    DCHECK(superblock);
+    tablet::RowSetDataPB* rowset = DCHECK_NOTNULL(superblock->mutable_rowsets(0));
+    tablet::ColumnDataPB* column = DCHECK_NOTNULL(rowset->mutable_columns(0));
+    BlockIdPB* block_id_pb = DCHECK_NOTNULL(column->mutable_block());
+    return block_id_pb;
+  }
+
+  // Grab the first column block we find in the SuperBlock.
+  static BlockId FirstColumnBlockId(tablet::TabletSuperBlockPB* superblock) {
+    return BlockId::FromPB(*FirstColumnBlockIdPB(superblock));
   }
 
   // Check that the contents and CRC32C of a DataChunkPB are equal to a local buffer.

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/tablet_copy_client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client-test.cc b/src/kudu/tserver/tablet_copy_client-test.cc
index 454833b..213e754 100644
--- a/src/kudu/tserver/tablet_copy_client-test.cc
+++ b/src/kudu/tserver/tablet_copy_client-test.cc
@@ -43,8 +43,8 @@ class TabletCopyClientTest : public TabletCopyTest {
     tablet_peer_->WaitUntilConsensusRunning(MonoDelta::FromSeconds(10.0));
     rpc::MessengerBuilder(CURRENT_TEST_NAME()).Build(&messenger_);
     client_.reset(new TabletCopyClient(GetTabletId(),
-                                            fs_manager_.get(),
-                                            messenger_));
+                                       fs_manager_.get(),
+                                       messenger_));
     ASSERT_OK(GetRaftConfigLeader(tablet_peer_->consensus()
         ->ConsensusState(consensus::CONSENSUS_CONFIG_COMMITTED), &leader_));
 
@@ -97,7 +97,7 @@ TEST_F(TabletCopyClientTest, TestBeginEndSession) {
 
 // Basic data block download unit test.
 TEST_F(TabletCopyClientTest, TestDownloadBlock) {
-  BlockId block_id = FirstColumnBlockId(*client_->superblock_);
+  BlockId block_id = FirstColumnBlockId(client_->superblock_.get());
   Slice slice;
   faststring scratch;
 
@@ -210,8 +210,8 @@ TEST_F(TabletCopyClientTest, TestDownloadAllBlocks) {
   // As long as block IDs are generated with UUIDs or something equally
   // unique, there's no danger of a block in the new superblock somehow
   // being assigned the same ID as a block in the existing superblock.
-  vector<BlockId> old_data_blocks = GetAllSortedBlocks(*client_->superblock_.get());
-  vector<BlockId> new_data_blocks = GetAllSortedBlocks(*client_->new_superblock_.get());
+  vector<BlockId> old_data_blocks = GetAllSortedBlocks(*client_->old_superblock_);
+  vector<BlockId> new_data_blocks = GetAllSortedBlocks(*client_->superblock_);
   vector<BlockId> result;
   std::set_intersection(old_data_blocks.begin(), old_data_blocks.end(),
                         new_data_blocks.begin(), new_data_blocks.end(),
@@ -234,5 +234,68 @@ TEST_F(TabletCopyClientTest, TestDownloadAllBlocks) {
   }
 }
 
+enum DeleteTrigger {
+  kAbortMethod, // Delete blocks via Abort().
+  kDestructor,  // Delete blocks via destructor.
+  kNoDelete     // Don't delete blocks.
+};
+
+class TabletCopyClientAbortTest : public TabletCopyClientTest,
+                                  public ::testing::WithParamInterface<DeleteTrigger> {
+};
+
+// Test that we can clean up our downloaded blocks either explicitly using
+// Abort() or implicitly by destroying the TabletCopyClient instance before
+// calling Finish().
+TEST_P(TabletCopyClientAbortTest, TestAbort) {
+  // Download a block.
+  BlockIdPB* block_id_pb = FirstColumnBlockIdPB(client_->superblock_.get());
+  int block_id_count = 0;
+  int num_blocks = client_->CountBlocks();
+  ASSERT_OK(client_->DownloadAndRewriteBlock(block_id_pb, &block_id_count, num_blocks));
+  BlockId new_block_id = BlockId::FromPB(*block_id_pb);
+  ASSERT_TRUE(fs_manager_->BlockExists(new_block_id));
+
+  // Download a WAL segment.
+  ASSERT_OK(fs_manager_->CreateDirIfMissing(fs_manager_->GetTabletWalDir(GetTabletId())));
+  uint64_t seqno = client_->wal_seqnos_[0];
+  ASSERT_OK(client_->DownloadWAL(seqno));
+  string wal_path = fs_manager_->GetWalSegmentFileName(GetTabletId(), seqno);
+  ASSERT_TRUE(fs_manager_->Exists(wal_path));
+
+  scoped_refptr<TabletMetadata> meta = client_->meta_;
+
+  DeleteTrigger trigger = GetParam();
+  switch (trigger) {
+    case kAbortMethod:
+      ASSERT_OK(client_->Abort());
+      break;
+    case kDestructor:
+      client_.reset();
+      break;
+    case kNoDelete:
+      // Call Finish() and then destroy the object.
+      // It should not delete its downloaded blocks.
+      ASSERT_OK(client_->Finish());
+      client_.reset();
+      break;
+    default:
+      FAIL();
+  }
+
+  if (trigger == kNoDelete) {
+    ASSERT_TRUE(fs_manager_->BlockExists(new_block_id));
+    ASSERT_TRUE(fs_manager_->Exists(wal_path));
+  } else {
+    ASSERT_EQ(tablet::TABLET_DATA_TOMBSTONED, meta->tablet_data_state());
+    ASSERT_FALSE(fs_manager_->BlockExists(new_block_id));
+    ASSERT_FALSE(fs_manager_->Exists(wal_path));
+  }
+}
+
+INSTANTIATE_TEST_CASE_P(BlockDeleteTriggers,
+                        TabletCopyClientAbortTest,
+                        ::testing::Values(kAbortMethod, kDestructor, kNoDelete));
+
 } // namespace tserver
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/tablet_copy_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client.cc b/src/kudu/tserver/tablet_copy_client.cc
index e0e0e97..3d759e3 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -27,6 +27,7 @@
 #include "kudu/fs/block_id.h"
 #include "kudu/fs/block_manager.h"
 #include "kudu/fs/fs_manager.h"
+#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/gutil/walltime.h"
@@ -38,6 +39,7 @@
 #include "kudu/tserver/tablet_copy.pb.h"
 #include "kudu/tserver/tablet_copy.proxy.h"
 #include "kudu/tserver/tablet_server.h"
+#include "kudu/tserver/ts_tablet_manager.h"
 #include "kudu/util/crc.h"
 #include "kudu/util/env.h"
 #include "kudu/util/env_util.h"
@@ -95,14 +97,12 @@ using tablet::TabletStatusListener;
 using tablet::TabletSuperBlockPB;
 
 TabletCopyClient::TabletCopyClient(std::string tablet_id,
-                                             FsManager* fs_manager,
-                                             shared_ptr<Messenger> messenger)
+                                   FsManager* fs_manager,
+                                   shared_ptr<Messenger> messenger)
     : tablet_id_(std::move(tablet_id)),
       fs_manager_(fs_manager),
       messenger_(std::move(messenger)),
-      started_(false),
-      downloaded_wal_(false),
-      downloaded_blocks_(false),
+      state_(kInitialized),
       replace_tombstoned_tablet_(false),
       status_listener_(nullptr),
       session_idle_timeout_millis_(0),
@@ -110,11 +110,14 @@ TabletCopyClient::TabletCopyClient(std::string tablet_id,
 
 TabletCopyClient::~TabletCopyClient() {
   // Note: Ending the tablet copy session releases anchors on the remote.
-  WARN_NOT_OK(EndRemoteSession(), "Unable to close tablet copy session");
+  WARN_NOT_OK(EndRemoteSession(), Substitute("$0Unable to close tablet copy session",
+                                             LogPrefix()));
+  WARN_NOT_OK(Abort(), Substitute("$0Failed to fully clean up tablet after aborted copy",
+                                  LogPrefix()));
 }
 
 Status TabletCopyClient::SetTabletToReplace(const scoped_refptr<TabletMetadata>& meta,
-                                                 int64_t caller_term) {
+                                            int64_t caller_term) {
   CHECK_EQ(tablet_id_, meta->tablet_id());
   TabletDataState data_state = meta->tablet_data_state();
   if (data_state != tablet::TABLET_DATA_TOMBSTONED) {
@@ -150,8 +153,8 @@ Status TabletCopyClient::SetTabletToReplace(const scoped_refptr<TabletMetadata>&
 }
 
 Status TabletCopyClient::Start(const HostPort& copy_source_addr,
-                                    scoped_refptr<TabletMetadata>* meta) {
-  CHECK(!started_);
+                               scoped_refptr<TabletMetadata>* meta) {
+  CHECK_EQ(kInitialized, state_);
   start_time_micros_ = GetCurrentTimeMicros();
 
   Sockaddr addr;
@@ -192,8 +195,22 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
 
   session_id_ = resp.session_id();
   session_idle_timeout_millis_ = resp.session_idle_timeout_millis();
-  superblock_.reset(resp.release_superblock());
+
+  // Store a copy of the remote (old) superblock for testing purposes.
+  old_superblock_.reset(resp.release_superblock());
+
+  // Make a copy of the remote superblock. We will modify this
+  // TabletSuperBlockPB in place as we download blocks and assign local block
+  // ids to replace their remote ids.
+  superblock_.reset(new TabletSuperBlockPB(*old_superblock_));
+
+  // The orphaned physical block ids at the remote have no meaning to us.
+  superblock_->clear_orphaned_blocks();
+
+  // Set the data state to COPYING to indicate that, on crash, this replica
+  // should be discarded.
   superblock_->set_tablet_data_state(tablet::TABLET_DATA_COPYING);
+
   wal_seqnos_.assign(resp.wal_segment_seqnos().begin(), resp.wal_segment_seqnos().end());
   remote_committed_cstate_.reset(resp.release_initial_committed_cstate());
 
@@ -212,19 +229,19 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
       return Status::InvalidArgument(
           Substitute("Tablet $0: source peer has term $1 but "
                      "tombstoned replica has last-logged opid with higher term $2. "
-                      "Refusing tablet copy from source peer $3",
-                      tablet_id_,
-                      remote_committed_cstate_->current_term(),
-                      last_logged_term,
-                      copy_peer_uuid));
+                     "Refusing tablet copy from source peer $3",
+                     tablet_id_,
+                     remote_committed_cstate_->current_term(),
+                     last_logged_term,
+                     copy_peer_uuid));
     }
 
-    // Remove any existing orphaned blocks from the tablet, and
+    // Remove any existing orphaned blocks and WALs from the tablet, and
     // set the data state to 'COPYING'.
-    RETURN_NOT_OK_PREPEND(meta_->DeleteTabletData(tablet::TABLET_DATA_COPYING, boost::none),
-                          "Couldn't replace superblock with COPYING data state");
+    RETURN_NOT_OK_PREPEND(
+        TSTabletManager::DeleteTabletData(meta_, tablet::TABLET_DATA_COPYING, boost::none),
+        "Could not replace superblock with COPYING data state");
   } else {
-
     Partition partition;
     Partition::FromPB(superblock_->partition(), &partition);
     PartitionSchema partition_schema;
@@ -242,7 +259,7 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
                                             &meta_));
   }
 
-  started_ = true;
+  state_ = kStarted;
   if (meta) {
     *meta = meta_;
   }
@@ -250,7 +267,8 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
 }
 
 Status TabletCopyClient::FetchAll(TabletStatusListener* status_listener) {
-  CHECK(started_);
+  CHECK_EQ(kStarted, state_);
+
   status_listener_ = status_listener;
 
   // Download all the files (serially, for now, but in parallel in the future).
@@ -262,9 +280,8 @@ Status TabletCopyClient::FetchAll(TabletStatusListener* status_listener) {
 
 Status TabletCopyClient::Finish() {
   CHECK(meta_);
-  CHECK(started_);
-  CHECK(downloaded_wal_);
-  CHECK(downloaded_blocks_);
+  CHECK_EQ(kStarted, state_);
+  state_ = kFinished;
 
   RETURN_NOT_OK(WriteConsensusMetadata());
 
@@ -273,8 +290,8 @@ Status TabletCopyClient::Finish() {
   // superblock is in a valid state to bootstrap from.
   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_));
+  superblock_->set_tablet_data_state(tablet::TABLET_DATA_READY);
+  RETURN_NOT_OK(meta_->ReplaceSuperBlock(*superblock_));
 
   if (FLAGS_tablet_copy_save_downloaded_metadata) {
     string meta_path = fs_manager_->GetTabletMetadataPath(tablet_id_);
@@ -287,6 +304,27 @@ Status TabletCopyClient::Finish() {
   return Status::OK();
 }
 
+Status TabletCopyClient::Abort() {
+  if (state_ != kStarted) {
+    return Status::OK();
+  }
+  state_ = kFinished;
+  CHECK(meta_);
+
+  // Write the in-progress superblock to disk so that when we delete the tablet
+  // data all the partial blocks we have persisted will be deleted.
+  DCHECK_EQ(tablet::TABLET_DATA_COPYING, superblock_->tablet_data_state());
+  RETURN_NOT_OK(meta_->ReplaceSuperBlock(*superblock_));
+
+  // Delete all of the tablet data, including blocks and WALs.
+  RETURN_NOT_OK_PREPEND(
+      TSTabletManager::DeleteTabletData(meta_, tablet::TABLET_DATA_TOMBSTONED, boost::none),
+      LogPrefix() + "Failed to tombstone tablet after aborting tablet copy");
+
+  UpdateStatusMessage(Substitute("Tombstoned tablet $0: Tablet copy aborted", tablet_id_));
+  return Status::OK();
+}
+
 // Decode the remote error into a human-readable Status object.
 Status TabletCopyClient::ExtractRemoteError(const rpc::ErrorStatusPB& remote_error) {
   if (PREDICT_TRUE(remote_error.HasExtension(TabletCopyErrorPB::tablet_copy_error_ext))) {
@@ -302,7 +340,7 @@ Status TabletCopyClient::ExtractRemoteError(const rpc::ErrorStatusPB& remote_err
 
 // Enhance a RemoteError Status message with additional details from the remote.
 Status TabletCopyClient::UnwindRemoteError(const Status& status,
-                                                const rpc::RpcController& controller) {
+                                           const rpc::RpcController& controller) {
   if (!status.IsRemoteError()) {
     return status;
   }
@@ -312,18 +350,17 @@ Status TabletCopyClient::UnwindRemoteError(const Status& status,
 
 void TabletCopyClient::UpdateStatusMessage(const string& message) {
   if (status_listener_ != nullptr) {
-    status_listener_->StatusMessage("TabletCopy: " + message);
+    status_listener_->StatusMessage(Substitute("Tablet Copy: $0", message));
   }
 }
 
 Status TabletCopyClient::EndRemoteSession() {
-  if (!started_) {
+  if (state_ == kInitialized) {
     return Status::OK();
   }
 
   rpc::RpcController controller;
-  controller.set_timeout(MonoDelta::FromMilliseconds(
-        FLAGS_tablet_copy_begin_session_timeout_ms));
+  controller.set_timeout(MonoDelta::FromMilliseconds(FLAGS_tablet_copy_begin_session_timeout_ms));
 
   EndTabletCopySessionRequestPB req;
   req.set_session_id(session_id_);
@@ -337,7 +374,7 @@ Status TabletCopyClient::EndRemoteSession() {
 }
 
 Status TabletCopyClient::DownloadWALs() {
-  CHECK(started_);
+  CHECK_EQ(kStarted, state_);
 
   // Delete and recreate WAL dir if it already exists, to ensure stray files are
   // not kept from previous copies and runs.
@@ -359,14 +396,10 @@ Status TabletCopyClient::DownloadWALs() {
     ++counter;
   }
 
-  downloaded_wal_ = true;
   return Status::OK();
 }
 
-Status TabletCopyClient::DownloadBlocks() {
-  CHECK(started_);
-
-  // Count up the total number of blocks to download.
+int TabletCopyClient::CountBlocks() const {
   int num_blocks = 0;
   for (const RowSetDataPB& rowset : superblock_->rowsets()) {
     num_blocks += rowset.columns_size();
@@ -379,14 +412,20 @@ Status TabletCopyClient::DownloadBlocks() {
       num_blocks++;
     }
   }
+  return num_blocks;
+}
+
+Status TabletCopyClient::DownloadBlocks() {
+  CHECK_EQ(kStarted, state_);
+
+  // Count up the total number of blocks to download.
+  int num_blocks = CountBlocks();
 
   // Download each block, writing the new block IDs into the new superblock
   // as each block downloads.
-  gscoped_ptr<TabletSuperBlockPB> new_sb(new TabletSuperBlockPB());
-  new_sb->CopyFrom(*superblock_);
   int block_count = 0;
   LOG_WITH_PREFIX(INFO) << "Starting download of " << num_blocks << " data blocks...";
-  for (RowSetDataPB& rowset : *new_sb->mutable_rowsets()) {
+  for (RowSetDataPB& rowset : *superblock_->mutable_rowsets()) {
     for (ColumnDataPB& col : *rowset.mutable_columns()) {
       RETURN_NOT_OK(DownloadAndRewriteBlock(col.mutable_block(),
                                             &block_count, num_blocks));
@@ -409,12 +448,6 @@ Status TabletCopyClient::DownloadBlocks() {
     }
   }
 
-  // The orphaned physical block ids at the remote have no meaning to us.
-  new_sb->clear_orphaned_blocks();
-  new_superblock_.swap(new_sb);
-
-  downloaded_blocks_ = true;
-
   return Status::OK();
 }
 
@@ -463,11 +496,11 @@ Status TabletCopyClient::WriteConsensusMetadata() {
 }
 
 Status TabletCopyClient::DownloadAndRewriteBlock(BlockIdPB* block_id,
-                                                      int* block_count, int num_blocks) {
+                                                 int* block_count, int num_blocks) {
   BlockId old_block_id(BlockId::FromPB(*block_id));
   UpdateStatusMessage(Substitute("Downloading block $0 ($1/$2)",
-                                 old_block_id.ToString(), *block_count,
-                                 num_blocks));
+                                 old_block_id.ToString(),
+                                 *block_count + 1, num_blocks));
   BlockId new_block_id;
   RETURN_NOT_OK_PREPEND(DownloadBlock(old_block_id, &new_block_id),
       "Unable to download block with id " + old_block_id.ToString());
@@ -478,7 +511,7 @@ Status TabletCopyClient::DownloadAndRewriteBlock(BlockIdPB* block_id,
 }
 
 Status TabletCopyClient::DownloadBlock(const BlockId& old_block_id,
-                                            BlockId* new_block_id) {
+                                       BlockId* new_block_id) {
   VLOG_WITH_PREFIX(1) << "Downloading block with block_id " << old_block_id.ToString();
 
   gscoped_ptr<WritableBlock> block;
@@ -499,7 +532,7 @@ Status TabletCopyClient::DownloadBlock(const BlockId& old_block_id,
 
 template<class Appendable>
 Status TabletCopyClient::DownloadFile(const DataIdPB& data_id,
-                                           Appendable* appendable) {
+                                      Appendable* appendable) {
   uint64_t offset = 0;
   rpc::RpcController controller;
   controller.set_timeout(MonoDelta::FromMilliseconds(session_idle_timeout_millis_));

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/tablet_copy_client.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client.h b/src/kudu/tserver/tablet_copy_client.h
index de2a730..a9e195e 100644
--- a/src/kudu/tserver/tablet_copy_client.h
+++ b/src/kudu/tserver/tablet_copy_client.h
@@ -23,14 +23,13 @@
 
 #include <gtest/gtest_prod.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/fs/block_id.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
 
-class BlockId;
 class BlockIdPB;
 class FsManager;
 class HostPort;
@@ -95,22 +94,34 @@ class TabletCopyClient {
   // otherwise the TabletMetadata object resulting from the initial remote
   // bootstrap response is returned.
   Status Start(const HostPort& copy_source_addr,
-               scoped_refptr<tablet::TabletMetadata>* metadata);
+               scoped_refptr<tablet::TabletMetadata>* meta);
 
   // 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);
 
   // After downloading all files successfully, write out the completed
-  // replacement superblock.
+  // replacement superblock. Must be called after Start() and FetchAll().
+  // Must not be called after Abort().
   Status Finish();
 
+  // Abort an in-progress transfer and immediately delete the data blocks and
+  // WALs downloaded so far. Does nothing if called after Finish().
+  Status Abort();
+
  private:
   FRIEND_TEST(TabletCopyClientTest, TestBeginEndSession);
   FRIEND_TEST(TabletCopyClientTest, TestDownloadBlock);
   FRIEND_TEST(TabletCopyClientTest, TestVerifyData);
   FRIEND_TEST(TabletCopyClientTest, TestDownloadWalSegment);
   FRIEND_TEST(TabletCopyClientTest, TestDownloadAllBlocks);
+  FRIEND_TEST(TabletCopyClientAbortTest, TestAbort);
+
+  enum State {
+    kInitialized,
+    kStarted,
+    kFinished,
+  };
 
   // Extract the embedded Status message from the given ErrorStatusPB.
   // The given ErrorStatusPB must extend TabletCopyErrorPB.
@@ -137,11 +148,13 @@ class TabletCopyClient {
   // downloaded as part of initiating the tablet copy session.
   Status WriteConsensusMetadata();
 
+  // Count the number of blocks contained in 'superblock_'.
+  int CountBlocks() const;
+
   // Download all blocks belonging to a tablet sequentially.
   //
-  // Blocks are given new IDs upon creation. On success, 'new_superblock_'
-  // is populated to reflect the new block IDs and should be used in lieu
-  // of 'superblock_' henceforth.
+  // Blocks are given new IDs upon creation. On success, 'superblock_'
+  // is populated to reflect the new block IDs.
   Status DownloadBlocks();
 
   // Download the block specified by 'block_id'.
@@ -177,10 +190,8 @@ class TabletCopyClient {
   FsManager* const fs_manager_;
   const std::shared_ptr<rpc::Messenger> messenger_;
 
-  // 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.
+  // State of the progress of the tablet copy operation.
+  State state_;
 
   // Session-specific data items.
   bool replace_tombstoned_tablet_;
@@ -196,9 +207,9 @@ class TabletCopyClient {
   std::shared_ptr<TabletCopyServiceProxy> proxy_;
   std::string session_id_;
   uint64_t session_idle_timeout_millis_;
-  gscoped_ptr<tablet::TabletSuperBlockPB> superblock_;
-  gscoped_ptr<tablet::TabletSuperBlockPB> new_superblock_;
-  gscoped_ptr<consensus::ConsensusStatePB> remote_committed_cstate_;
+  std::unique_ptr<tablet::TabletSuperBlockPB> old_superblock_;
+  std::unique_ptr<tablet::TabletSuperBlockPB> superblock_;
+  std::unique_ptr<consensus::ConsensusStatePB> remote_committed_cstate_;
   std::vector<uint64_t> wal_seqnos_;
   int64_t start_time_micros_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/tablet_copy_service-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service-test.cc b/src/kudu/tserver/tablet_copy_service-test.cc
index fc88155..f389751 100644
--- a/src/kudu/tserver/tablet_copy_service-test.cc
+++ b/src/kudu/tserver/tablet_copy_service-test.cc
@@ -365,7 +365,7 @@ TEST_F(TabletCopyServiceTest, TestFetchInvalidBlockOffset) {
   RpcController controller;
   // Impossible offset.
   uint64_t offset = std::numeric_limits<uint64_t>::max();
-  Status status = DoFetchData(session_id, AsDataTypeId(FirstColumnBlockId(superblock)),
+  Status status = DoFetchData(session_id, AsDataTypeId(FirstColumnBlockId(&superblock)),
                               &offset, nullptr, &resp, &controller);
   ASSERT_REMOTE_ERROR(status, controller.error_response(),
                       TabletCopyErrorPB::INVALID_TABLET_COPY_REQUEST,
@@ -379,7 +379,7 @@ TEST_F(TabletCopyServiceTest, TestFetchBlockAtOnce) {
   ASSERT_OK(DoBeginValidTabletCopySession(&session_id, &superblock));
 
   // Local.
-  BlockId block_id = FirstColumnBlockId(superblock);
+  BlockId block_id = FirstColumnBlockId(&superblock);
   Slice local_data;
   faststring scratch;
   ASSERT_OK(ReadLocalBlockFile(mini_server_->server()->fs_manager(), block_id,
@@ -399,7 +399,7 @@ TEST_F(TabletCopyServiceTest, TestFetchBlockIncrementally) {
   tablet::TabletSuperBlockPB superblock;
   ASSERT_OK(DoBeginValidTabletCopySession(&session_id, &superblock));
 
-  BlockId block_id = FirstColumnBlockId(superblock);
+  BlockId block_id = FirstColumnBlockId(&superblock);
   Slice local_data;
   faststring scratch;
   ASSERT_OK(ReadLocalBlockFile(mini_server_->server()->fs_manager(), block_id,

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index 3aea832..477d8ea 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -1017,11 +1017,10 @@ void TabletServiceImpl::ScannerKeepAlive(const ScannerKeepAliveRequestPB *req,
   DCHECK(req->has_scanner_id());
   SharedScanner scanner;
   if (!server_->scanner_manager()->LookupScanner(req->scanner_id(), &scanner)) {
-      resp->mutable_error()->set_code(TabletServerErrorPB::SCANNER_EXPIRED);
-      StatusToPB(Status::NotFound("Scanner not found"),
-                 resp->mutable_error()->mutable_status());
-      context->RespondSuccess();
-      return;
+    resp->mutable_error()->set_code(TabletServerErrorPB::SCANNER_EXPIRED);
+    StatusToPB(Status::NotFound("Scanner not found"), resp->mutable_error()->mutable_status());
+    context->RespondSuccess();
+    return;
   }
   scanner->UpdateAccessTime();
   context->RespondSuccess();

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/src/kudu/tserver/ts_tablet_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index 8214655..77369bb 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -317,16 +317,6 @@ Status TSTabletManager::CreateNewTablet(const string& table_id,
   return Status::OK();
 }
 
-// If 'expr' fails, log a message, tombstone the given tablet, and return.
-#define TOMBSTONE_NOT_OK(expr, peer, msg) \
-  do { \
-    const Status& _s = (expr); \
-    if (PREDICT_FALSE(!_s.ok())) { \
-      LogAndTombstone((peer), (msg), _s); \
-      return; \
-    } \
-  } while (0)
-
 Status TSTabletManager::CheckLeaderTermNotLower(const string& tablet_id,
                                                 int64_t leader_term,
                                                 int64_t last_logged_term) {
@@ -424,8 +414,9 @@ void TSTabletManager::RunTabletCopy(
 
   TabletServerErrorPB::Code error_code = TabletServerErrorPB::UNKNOWN_ERROR;
 
-  const string& tablet_id = req->tablet_id();
-  const string& copy_source_uuid = req->copy_peer_uuid();
+  // Copy these strings so they stay valid even after responding to the request.
+  string tablet_id = req->tablet_id(); // NOLINT(*)
+  string copy_source_uuid = req->copy_peer_uuid(); // NOLINT(*)
   HostPort copy_source_addr;
   CALLBACK_RETURN_NOT_OK(HostPortFromPB(req->copy_peer_addr(), &copy_source_addr));
   int64_t leader_term = req->caller_term();
@@ -454,7 +445,7 @@ void TSTabletManager::RunTabletCopy(
     switch (data_state) {
       case TABLET_DATA_COPYING:
         // This should not be possible due to the transition_in_progress_ "lock".
-        LOG(FATAL) << LogPrefix(tablet_id) << " Tablet Copy: "
+        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();
@@ -476,8 +467,8 @@ void TSTabletManager::RunTabletCopy(
         old_tablet_peer->Shutdown();
         // TODO: Because we begin shutdown of the tablet after we check our
         // last-logged term against the leader's term, there may be operations
-        // 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
+        // in flight and it may be possible for the same check in the tablet
+        // copy 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 tablet copy this replica again. We could try to
         // check again after calling Shutdown(), and if the check fails, try to
@@ -531,15 +522,22 @@ void TSTabletManager::RunTabletCopy(
 
   // Download all of the remote files.
   Status s = tc_client.FetchAll(implicit_cast<TabletStatusListener*>(tablet_peer.get()));
-  TOMBSTONE_NOT_OK(s, tablet_peer,
-                   Substitute("Tablet Copy: Unable to fetch data from remote peer $0",
-                              kSrcPeerInfo));
+  if (!s.ok()) {
+    LOG(WARNING) << LogPrefix(tablet_id) << "Tablet Copy: Unable to fetch data from remote peer "
+                                         << kSrcPeerInfo << ": " << s.ToString();
+    return;
+  }
 
   MAYBE_FAULT(FLAGS_fault_crash_after_tc_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(tc_client.Finish(), tablet_peer, "Tablet Copy: Failure calling Finish()");
+  s = tc_client.Finish();
+  if (!s.ok()) {
+    LOG(WARNING) << LogPrefix(tablet_id) << "Tablet Copy: Failure calling Finish(): "
+                                         << s.ToString();
+    return;
+  }
 
   // We don't tombstone the tablet if opening the tablet fails, because on next
   // startup it's still in a valid, fully-copied state.
@@ -698,7 +696,7 @@ Status TSTabletManager::StartTabletStateTransitionUnlocked(
 
 Status TSTabletManager::OpenTabletMeta(const string& tablet_id,
                                        scoped_refptr<TabletMetadata>* metadata) {
-  LOG(INFO) << "Loading metadata for tablet " << tablet_id;
+  LOG(INFO) << LogPrefix(tablet_id) << "Loading tablet metadata";
   TRACE("Loading metadata...");
   scoped_refptr<TabletMetadata> meta;
   RETURN_NOT_OK_PREPEND(TabletMetadata::Load(fs_manager_, tablet_id, &meta),
@@ -860,7 +858,9 @@ void TSTabletManager::RegisterTablet(const std::string& tablet_id,
     LOG(FATAL) << "Unable to register tablet peer " << tablet_id << ": already registered!";
   }
 
-  LOG(INFO) << "Registered tablet " << tablet_id;
+  TabletDataState data_state = tablet_peer->tablet_metadata()->tablet_data_state();
+  LOG(INFO) << LogPrefix(tablet_id) << Substitute("Registered tablet (data state: $0)",
+                                                  TabletDataState_Name(data_state));
 }
 
 bool TSTabletManager::LookupTablet(const string& tablet_id,
@@ -1021,7 +1021,8 @@ Status TSTabletManager::DeleteTabletData(const scoped_refptr<TabletMetadata>& me
             << "Deleting tablet data with delete state "
             << TabletDataState_Name(data_state);
   CHECK(data_state == TABLET_DATA_DELETED ||
-        data_state == TABLET_DATA_TOMBSTONED)
+        data_state == TABLET_DATA_TOMBSTONED ||
+        data_state == TABLET_DATA_COPYING)
       << "Unexpected data_state to delete tablet " << meta->tablet_id() << ": "
       << TabletDataState_Name(data_state) << " (" << data_state << ")";
 
@@ -1037,33 +1038,19 @@ Status TSTabletManager::DeleteTabletData(const scoped_refptr<TabletMetadata>& me
   MAYBE_FAULT(FLAGS_fault_crash_after_wal_deleted);
 
   // We do not delete the superblock or the consensus metadata when tombstoning
-  // a tablet.
-  if (data_state == TABLET_DATA_TOMBSTONED) {
+  // a tablet or marking it as entering the tablet copy process.
+  if (data_state == TABLET_DATA_COPYING ||
+      data_state == TABLET_DATA_TOMBSTONED) {
     return Status::OK();
   }
 
   // Only TABLET_DATA_DELETED tablets get this far.
+  DCHECK_EQ(TABLET_DATA_DELETED, data_state);
   RETURN_NOT_OK(ConsensusMetadata::DeleteOnDiskData(meta->fs_manager(), meta->tablet_id()));
   MAYBE_FAULT(FLAGS_fault_crash_after_cmeta_deleted);
   return meta->DeleteSuperBlock();
 }
 
-void TSTabletManager::LogAndTombstone(const scoped_refptr<TabletPeer>& peer,
-                                      const std::string& msg,
-                                      const Status& s) {
-  const string& tablet_id = peer->tablet_id();
-  LOG(WARNING) << LogPrefix(tablet_id) << msg << ": " << s.ToString();
-
-  Status delete_status = DeleteTabletData(
-      peer->tablet_metadata(), 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) << LogPrefix(tablet_id) << "Failed to tombstone tablet after tablet copy: "
-               << delete_status.ToString();
-  }
-  peer->StatusMessage(Substitute("Tombstoned tablet: $0 ($1)", msg, s.ToString()));
-}
-
 TransitionInProgressDeleter::TransitionInProgressDeleter(
     TransitionInProgressMap* map, rw_spinlock* lock, string entry)
     : in_progress_(map), lock_(lock), entry_(std::move(entry)) {}

http://git-wip-us.apache.org/repos/asf/kudu/blob/72541b47/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 43f5396..6f437be 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -270,13 +270,6 @@ class TSTabletManager : public tserver::TabletPeerLookupIf {
                                  int64_t leader_term,
                                  int64_t last_logged_term);
 
-  // Print a log message using the given info and tombstone the specified
-  // tablet. If tombstoning the tablet fails, a FATAL error is logged, resulting
-  // in a crash.
-  void LogAndTombstone(const scoped_refptr<tablet::TabletPeer>& peer,
-                       const std::string& msg,
-                       const Status& s);
-
   TSTabletManagerStatePB state() const {
     shared_lock<rw_spinlock> l(lock_);
     return state_;