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/06/02 02:27:04 UTC

[2/2] kudu git commit: tablet: Get rid of TabletStatusListener

tablet: Get rid of TabletStatusListener

There is no need for this interface. We don't have a circular ownership
dependency related to TabletReplica.

Change-Id: I92204ec9d6f5db218deb51c83e44f821e2ed2442
Reviewed-on: http://gerrit.cloudera.org:8080/7039
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 037f72b375ecfa3b0cf4e7b393f1093332c2dfbc
Parents: 6ea3acd
Author: Mike Percy <mp...@apache.org>
Authored: Wed May 31 21:42:35 2017 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Fri Jun 2 02:26:41 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/sys_catalog.cc         |  3 +-
 src/kudu/tablet/tablet_bootstrap.cc    | 54 ++++++++++++++---------------
 src/kudu/tablet/tablet_bootstrap.h     |  4 +--
 src/kudu/tablet/tablet_replica.cc      | 10 +++---
 src/kudu/tablet/tablet_replica.h       | 19 +++-------
 src/kudu/tserver/tablet_copy_client.cc | 28 +++++++--------
 src/kudu/tserver/tablet_copy_client.h  |  9 +++--
 src/kudu/tserver/ts_tablet_manager.cc  |  8 ++---
 src/kudu/tserver/ts_tablet_manager.h   |  1 -
 9 files changed, 59 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/src/kudu/master/sys_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 3d75e98..c25aae7 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -73,7 +73,6 @@ using kudu::log::Log;
 using kudu::tablet::LatchTransactionCompletionCallback;
 using kudu::tablet::Tablet;
 using kudu::tablet::TabletReplica;
-using kudu::tablet::TabletStatusListener;
 using kudu::tserver::WriteRequestPB;
 using kudu::tserver::WriteResponsePB;
 using std::function;
@@ -321,7 +320,7 @@ Status SysCatalogTable::SetupTablet(const scoped_refptr<tablet::TabletMetadata>&
                                 master_->mem_tracker(),
                                 scoped_refptr<rpc::ResultTracker>(),
                                 metric_registry_,
-                                implicit_cast<TabletStatusListener*>(tablet_replica_.get()),
+                                tablet_replica_,
                                 &tablet,
                                 &log,
                                 tablet_replica_->log_anchor_registry(),

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index c5b8bd1..f182ed5 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -35,7 +35,6 @@
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/consensus/opid_util.h"
 #include "kudu/fs/fs_manager.h"
-#include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/strcat.h"
@@ -48,7 +47,6 @@
 #include "kudu/tablet/lock_manager.h"
 #include "kudu/tablet/row_op.h"
 #include "kudu/tablet/tablet.h"
-#include "kudu/tablet/tablet_replica.h"
 #include "kudu/tablet/transactions/alter_schema_transaction.h"
 #include "kudu/tablet/transactions/write_transaction.h"
 #include "kudu/util/debug/trace_event.h"
@@ -169,9 +167,9 @@ class TabletBootstrap {
   TabletBootstrap(const scoped_refptr<TabletMetadata>& meta,
                   const scoped_refptr<Clock>& clock,
                   shared_ptr<MemTracker> mem_tracker,
-                  const scoped_refptr<ResultTracker> result_tracker,
+                  const scoped_refptr<ResultTracker>& result_tracker,
                   MetricRegistry* metric_registry,
-                  TabletStatusListener* listener,
+                  const scoped_refptr<TabletReplica>& tablet_replica,
                   const scoped_refptr<LogAnchorRegistry>& log_anchor_registry);
 
   // Plays the log segments, rebuilding the portion of the Tablet's soft
@@ -334,16 +332,15 @@ class TabletBootstrap {
   // Return a log prefix string in the standard "T xxx P yyy" format.
   string LogPrefix() const;
 
-  // Report a status message in the WAL as well as update the TabletReplica's
-  // status.
-  void StatusMessage(const string& status);
+  // Log a status message and set the TabletReplica's status as well.
+  void SetStatusMessage(const string& status);
 
   scoped_refptr<TabletMetadata> meta_;
   scoped_refptr<Clock> clock_;
   shared_ptr<MemTracker> mem_tracker_;
   scoped_refptr<rpc::ResultTracker> result_tracker_;
   MetricRegistry* metric_registry_;
-  TabletStatusListener* listener_;
+  scoped_refptr<TabletReplica> tablet_replica_;
   gscoped_ptr<tablet::Tablet> tablet_;
   const scoped_refptr<log::LogAnchorRegistry> log_anchor_registry_;
   scoped_refptr<log::Log> log_;
@@ -402,10 +399,10 @@ class TabletBootstrap {
   DISALLOW_COPY_AND_ASSIGN(TabletBootstrap);
 };
 
-void TabletBootstrap::StatusMessage(const string& status) {
+void TabletBootstrap::SetStatusMessage(const string& status) {
   LOG(INFO) << "T " << meta_->tablet_id() << " P " << meta_->fs_manager()->uuid() << ": "
             << status;
-  if (listener_) listener_->StatusMessage(status);
+  if (tablet_replica_) tablet_replica_->SetStatusMessage(status);
 }
 
 Status BootstrapTablet(const scoped_refptr<TabletMetadata>& meta,
@@ -413,7 +410,7 @@ Status BootstrapTablet(const scoped_refptr<TabletMetadata>& meta,
                        const shared_ptr<MemTracker>& mem_tracker,
                        const scoped_refptr<ResultTracker>& result_tracker,
                        MetricRegistry* metric_registry,
-                       TabletStatusListener* listener,
+                       const scoped_refptr<TabletReplica>& tablet_replica,
                        shared_ptr<tablet::Tablet>* rebuilt_tablet,
                        scoped_refptr<log::Log>* rebuilt_log,
                        const scoped_refptr<log::LogAnchorRegistry>& log_anchor_registry,
@@ -421,7 +418,7 @@ Status BootstrapTablet(const scoped_refptr<TabletMetadata>& meta,
   TRACE_EVENT1("tablet", "BootstrapTablet",
                "tablet_id", meta->tablet_id());
   TabletBootstrap bootstrap(meta, clock, mem_tracker, result_tracker,
-                            metric_registry, listener, log_anchor_registry);
+                            metric_registry, tablet_replica, log_anchor_registry);
   RETURN_NOT_OK(bootstrap.Bootstrap(rebuilt_tablet, rebuilt_log, consensus_info));
   // This is necessary since OpenNewLog() initially disables sync.
   RETURN_NOT_OK((*rebuilt_log)->ReEnableSyncIfRequired());
@@ -449,15 +446,16 @@ static string DebugInfo(const string& tablet_id,
 TabletBootstrap::TabletBootstrap(
     const scoped_refptr<TabletMetadata>& meta,
     const scoped_refptr<Clock>& clock, shared_ptr<MemTracker> mem_tracker,
-    const scoped_refptr<ResultTracker> result_tracker,
-    MetricRegistry* metric_registry, TabletStatusListener* listener,
+    const scoped_refptr<ResultTracker>& result_tracker,
+    MetricRegistry* metric_registry,
+    const scoped_refptr<TabletReplica>& tablet_replica,
     const scoped_refptr<LogAnchorRegistry>& log_anchor_registry)
     : meta_(meta),
       clock_(clock),
       mem_tracker_(std::move(mem_tracker)),
       result_tracker_(result_tracker),
       metric_registry_(metric_registry),
-      listener_(listener),
+      tablet_replica_(tablet_replica),
       log_anchor_registry_(log_anchor_registry) {}
 
 Status TabletBootstrap::Bootstrap(shared_ptr<Tablet>* rebuilt_tablet,
@@ -519,7 +517,7 @@ Status TabletBootstrap::RunBootstrap(shared_ptr<Tablet>* rebuilt_tablet,
                               TabletDataState_Name(tablet_data_state));
   }
 
-  StatusMessage("Bootstrap starting.");
+  SetStatusMessage("Bootstrap starting.");
 
   if (VLOG_IS_ON(1)) {
     TabletSuperBlockPB super_block;
@@ -572,7 +570,7 @@ void TabletBootstrap::FinishBootstrap(const string& message,
                                       scoped_refptr<log::Log>* rebuilt_log,
                                       shared_ptr<Tablet>* rebuilt_tablet) {
   tablet_->MarkFinishedBootstrapping();
-  StatusMessage(message);
+  SetStatusMessage(message);
   rebuilt_tablet->reset(tablet_.release());
   rebuilt_log->swap(log_);
 }
@@ -1159,21 +1157,21 @@ Status TabletBootstrap::PlaySegments(ConsensusBootstrapInfo* consensus_info) {
 
       auto now = MonoTime::Now();
       if (now - last_status_update > kStatusUpdateInterval) {
-        StatusMessage(Substitute("Bootstrap replaying log segment $0/$1 "
-                                 "($2/$3 this segment, stats: $4)",
-                                 segment_count + 1, log_reader_->num_segments(),
-                                 HumanReadableNumBytes::ToString(reader.offset()),
-                                 HumanReadableNumBytes::ToString(reader.read_up_to_offset()),
-                                 stats_.ToString()));
+        SetStatusMessage(Substitute("Bootstrap replaying log segment $0/$1 "
+                                    "($2/$3 this segment, stats: $4)",
+                                    segment_count + 1, log_reader_->num_segments(),
+                                    HumanReadableNumBytes::ToString(reader.offset()),
+                                    HumanReadableNumBytes::ToString(reader.read_up_to_offset()),
+                                    stats_.ToString()));
         last_status_update = now;
       }
     }
 
-    StatusMessage(Substitute("Bootstrap replayed $0/$1 log segments. "
-                             "Stats: $2. Pending: $3 replicates",
-                             segment_count + 1, log_reader_->num_segments(),
-                             stats_.ToString(),
-                             state.pending_replicates.size()));
+    SetStatusMessage(Substitute("Bootstrap replayed $0/$1 log segments. "
+                                "Stats: $2. Pending: $3 replicates",
+                                segment_count + 1, log_reader_->num_segments(),
+                                stats_.ToString(),
+                                state.pending_replicates.size()));
     segment_count++;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/src/kudu/tablet/tablet_bootstrap.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.h b/src/kudu/tablet/tablet_bootstrap.h
index 5dc72a3..905989f 100644
--- a/src/kudu/tablet/tablet_bootstrap.h
+++ b/src/kudu/tablet/tablet_bootstrap.h
@@ -25,6 +25,7 @@
 #include "kudu/consensus/log.pb.h"
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/ref_counted.h"
+#include "kudu/tablet/tablet_replica.h"
 #include "kudu/util/rw_mutex.h"
 #include "kudu/util/status.h"
 
@@ -54,7 +55,6 @@ class Clock;
 namespace tablet {
 class Tablet;
 class TabletMetadata;
-class TabletStatusListener;
 
 extern const char* kLogRecoveryDir;
 
@@ -69,7 +69,7 @@ Status BootstrapTablet(const scoped_refptr<TabletMetadata>& meta,
                        const std::shared_ptr<MemTracker>& mem_tracker,
                        const scoped_refptr<rpc::ResultTracker>& result_tracker,
                        MetricRegistry* metric_registry,
-                       TabletStatusListener* status_listener,
+                       const scoped_refptr<TabletReplica>& tablet_replica,
                        std::shared_ptr<Tablet>* rebuilt_tablet,
                        scoped_refptr<log::Log>* rebuilt_log,
                        const scoped_refptr<log::LogAnchorRegistry>& log_anchor_registry,

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/src/kudu/tablet/tablet_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_replica.cc b/src/kudu/tablet/tablet_replica.cc
index 259ac05..67c408b 100644
--- a/src/kudu/tablet/tablet_replica.cc
+++ b/src/kudu/tablet/tablet_replica.cc
@@ -28,9 +28,9 @@
 #include "kudu/consensus/consensus.h"
 #include "kudu/consensus/consensus_meta.h"
 #include "kudu/consensus/log.h"
+#include "kudu/consensus/log_anchor_registry.h"
 #include "kudu/consensus/log_util.h"
 #include "kudu/consensus/opid_util.h"
-#include "kudu/consensus/log_anchor_registry.h"
 #include "kudu/consensus/quorum_util.h"
 #include "kudu/consensus/raft_consensus.h"
 #include "kudu/gutil/mathlimits.h"
@@ -41,13 +41,13 @@
 #include "kudu/rpc/remote_method.h"
 #include "kudu/rpc/rpc_service.h"
 #include "kudu/rpc/service_pool.h"
-#include "kudu/tablet/transactions/transaction_driver.h"
 #include "kudu/tablet/transactions/alter_schema_transaction.h"
+#include "kudu/tablet/transactions/transaction_driver.h"
 #include "kudu/tablet/transactions/write_transaction.h"
+#include "kudu/tablet/tablet.pb.h"
 #include "kudu/tablet/tablet_bootstrap.h"
 #include "kudu/tablet/tablet_metrics.h"
 #include "kudu/tablet/tablet_replica_mm_ops.h"
-#include "kudu/tablet/tablet.pb.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/pb_util.h"
@@ -321,7 +321,7 @@ Status TabletReplica::WaitUntilConsensusRunning(const MonoDelta& timeout) {
       return Status::TimedOut(Substitute("Consensus is not running after waiting for $0. State; $1",
                                          elapsed.ToString(), TabletStatePB_Name(cached_state)));
     }
-    SleepFor(MonoDelta::FromMilliseconds(1 << backoff_exp));
+    SleepFor(MonoDelta::FromMilliseconds(1L << backoff_exp));
     backoff_exp = std::min(backoff_exp + 1, kMaxBackoffExp);
   }
   return Status::OK();
@@ -377,7 +377,7 @@ Status TabletReplica::RunLogGC() {
   return Status::OK();
 }
 
-void TabletReplica::StatusMessage(const std::string& status) {
+void TabletReplica::SetStatusMessage(const std::string& status) {
   std::lock_guard<simple_spinlock> lock(lock_);
   last_status_ = status;
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/src/kudu/tablet/tablet_replica.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_replica.h b/src/kudu/tablet/tablet_replica.h
index 0dfc5f2..51cd387 100644
--- a/src/kudu/tablet/tablet_replica.h
+++ b/src/kudu/tablet/tablet_replica.h
@@ -58,27 +58,15 @@ class LeaderTransactionDriver;
 class ReplicaTransactionDriver;
 class TabletReplica;
 class TabletStatusPB;
-class TabletStatusListener;
 class TransactionDriver;
 
-// Interface by which various tablet-related processes can report back their status
-// to TabletReplica without having to have a circular class dependency, and so that
-// those other classes can be easily tested without constructing a TabletReplica.
-class TabletStatusListener {
- public:
-  virtual ~TabletStatusListener() {}
-
-  virtual void StatusMessage(const std::string& status) = 0;
-};
-
 // A replica in a tablet consensus configuration, which coordinates writes to tablets.
 // Each time Write() is called this class appends a new entry to a replicated
 // state machine through a consensus algorithm, which makes sure that other
 // peers see the same updates in the same order. In addition to this, this
 // class also splits the work and coordinates multi-threaded execution.
 class TabletReplica : public RefCountedThreadSafe<TabletReplica>,
-                      public consensus::ReplicaTransactionFactory,
-                      public TabletStatusListener {
+                      public consensus::ReplicaTransactionFactory {
  public:
   TabletReplica(const scoped_refptr<TabletMetadata>& meta,
                 const consensus::RaftPeerPB& local_peer_pb, ThreadPool* apply_pool,
@@ -178,8 +166,9 @@ class TabletReplica : public RefCountedThreadSafe<TabletReplica>,
     state_ = BOOTSTRAPPING;
   }
 
-  // Implementation of TabletStatusListener::StatusMessage().
-  void StatusMessage(const std::string& status) override;
+  // Set a user-readable status message about the tablet. This may appear on
+  // the Web UI, for example.
+  void SetStatusMessage(const std::string& status);
 
   // Retrieve the last human-readable status of this tablet replica.
   std::string last_status() const;

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/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 3cb2936..5968de6 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -92,7 +92,7 @@ using tablet::RowSetDataPB;
 using tablet::TabletDataState;
 using tablet::TabletDataState_Name;
 using tablet::TabletMetadata;
-using tablet::TabletStatusListener;
+using tablet::TabletReplica;
 using tablet::TabletSuperBlockPB;
 
 TabletCopyClient::TabletCopyClient(std::string tablet_id,
@@ -103,7 +103,7 @@ TabletCopyClient::TabletCopyClient(std::string tablet_id,
       messenger_(std::move(messenger)),
       state_(kInitialized),
       replace_tombstoned_tablet_(false),
-      status_listener_(nullptr),
+      tablet_replica_(nullptr),
       session_idle_timeout_millis_(0),
       start_time_micros_(0) {}
 
@@ -267,10 +267,10 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
   return Status::OK();
 }
 
-Status TabletCopyClient::FetchAll(TabletStatusListener* status_listener) {
+Status TabletCopyClient::FetchAll(const scoped_refptr<TabletReplica>& tablet_replica) {
   CHECK_EQ(kStarted, state_);
 
-  status_listener_ = status_listener;
+  tablet_replica_ = tablet_replica;
 
   // Download all the files (serially, for now, but in parallel in the future).
   RETURN_NOT_OK(DownloadBlocks());
@@ -290,7 +290,7 @@ Status TabletCopyClient::Finish() {
   // to TABLET_DATA_READY, since we checked above that the response
   // superblock is in a valid state to bootstrap from.
   LOG_WITH_PREFIX(INFO) << "Tablet Copy complete. Replacing tablet superblock.";
-  UpdateStatusMessage("Replacing tablet superblock");
+  SetStatusMessage("Replacing tablet superblock");
   superblock_->set_tablet_data_state(tablet::TABLET_DATA_READY);
   RETURN_NOT_OK(meta_->ReplaceSuperBlock(*superblock_));
 
@@ -322,7 +322,7 @@ Status TabletCopyClient::Abort() {
       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_));
+  SetStatusMessage(Substitute("Tombstoned tablet $0: Tablet copy aborted", tablet_id_));
   return Status::OK();
 }
 
@@ -343,9 +343,9 @@ Status TabletCopyClient::UnwindRemoteError(const Status& status,
                           StatusFromPB(error.status()).ToString()));
 }
 
-void TabletCopyClient::UpdateStatusMessage(const string& message) {
-  if (status_listener_ != nullptr) {
-    status_listener_->StatusMessage(Substitute("Tablet Copy: $0", message));
+void TabletCopyClient::SetStatusMessage(const string& message) {
+  if (tablet_replica_ != nullptr) {
+    tablet_replica_->SetStatusMessage(Substitute("Tablet Copy: $0", message));
   }
 }
 
@@ -385,8 +385,8 @@ Status TabletCopyClient::DownloadWALs() {
   LOG_WITH_PREFIX(INFO) << "Starting download of " << num_segments << " WAL segments...";
   uint64_t counter = 0;
   for (uint64_t seg_seqno : wal_seqnos_) {
-    UpdateStatusMessage(Substitute("Downloading WAL segment with seq. number $0 ($1/$2)",
-                                   seg_seqno, counter + 1, num_segments));
+    SetStatusMessage(Substitute("Downloading WAL segment with seq. number $0 ($1/$2)",
+                                seg_seqno, counter + 1, num_segments));
     RETURN_NOT_OK(DownloadWAL(seg_seqno));
     ++counter;
   }
@@ -527,9 +527,9 @@ Status TabletCopyClient::DownloadAndRewriteBlock(const BlockIdPB& src_block_id,
                                                  int* block_count,
                                                  BlockIdPB* dest_block_id) {
   BlockId old_block_id(BlockId::FromPB(src_block_id));
-  UpdateStatusMessage(Substitute("Downloading block $0 ($1/$2)",
-                                 old_block_id.ToString(),
-                                 *block_count + 1, num_blocks));
+  SetStatusMessage(Substitute("Downloading block $0 ($1/$2)",
+                              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());

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/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 0340e6f..33ffa20 100644
--- a/src/kudu/tserver/tablet_copy_client.h
+++ b/src/kudu/tserver/tablet_copy_client.h
@@ -50,7 +50,6 @@ class RpcController;
 namespace tablet {
 class TabletMetadata;
 class TabletReplica;
-class TabletStatusListener;
 class TabletSuperBlockPB;
 } // namespace tablet
 
@@ -98,7 +97,7 @@ class TabletCopyClient {
 
   // 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);
+  Status FetchAll(const scoped_refptr<tablet::TabletReplica>& tablet_replica);
 
   // After downloading all files successfully, write out the completed
   // replacement superblock. Must be called after Start() and FetchAll().
@@ -126,9 +125,9 @@ class TabletCopyClient {
 
   static Status UnwindRemoteError(const Status& status, const rpc::RpcController& controller);
 
-  // Update the bootstrap StatusListener with a message.
+  // Set a new status message on the TabletReplica.
   // The string "TabletCopy: " will be prepended to each message.
-  void UpdateStatusMessage(const std::string& message);
+  void SetStatusMessage(const std::string& message);
 
   // End the tablet copy session.
   Status EndRemoteSession();
@@ -205,7 +204,7 @@ class TabletCopyClient {
   // bootstrapping a new replica (rather than replacing an old one).
   std::unique_ptr<consensus::ConsensusMetadata> cmeta_;
 
-  tablet::TabletStatusListener* status_listener_;
+  scoped_refptr<tablet::TabletReplica> tablet_replica_;
   std::shared_ptr<TabletCopyServiceProxy> proxy_;
   std::string session_id_;
   uint64_t session_idle_timeout_millis_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/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 773e645..db4c51d 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -148,8 +148,6 @@ using tablet::TABLET_DATA_TOMBSTONED;
 using tablet::TabletDataState;
 using tablet::TabletMetadata;
 using tablet::TabletReplica;
-using tablet::TabletStatusListener;
-using tablet::TabletStatusPB;
 using tserver::TabletCopyClient;
 
 TSTabletManager::TSTabletManager(FsManager* fs_manager,
@@ -564,7 +562,7 @@ void TSTabletManager::RunTabletCopy(
   // From this point onward, we do not notify the caller about progress or success.
 
   // Download all of the remote files.
-  Status s = tc_client.FetchAll(implicit_cast<TabletStatusListener*>(replica.get()));
+  Status s = tc_client.FetchAll(replica);
   if (!s.ok()) {
     LOG(WARNING) << LogPrefix(tablet_id) << "Tablet Copy: Unable to fetch data from remote peer "
                                          << kSrcPeerInfo << ": " << s.ToString();
@@ -680,7 +678,7 @@ Status TSTabletManager::DeleteTablet(
     return s;
   }
 
-  replica->StatusMessage("Deleted tablet blocks from disk");
+  replica->SetStatusMessage("Deleted tablet blocks from disk");
 
   // We only remove DELETED tablets from the tablet map.
   if (delete_type == TABLET_DATA_DELETED) {
@@ -782,7 +780,7 @@ void TSTabletManager::OpenTablet(const scoped_refptr<TabletMetadata>& meta,
                         server_->mem_tracker(),
                         server_->result_tracker(),
                         metric_registry_,
-                        implicit_cast<TabletStatusListener*>(replica.get()),
+                        replica,
                         &tablet,
                         &log,
                         replica->log_anchor_registry(),

http://git-wip-us.apache.org/repos/asf/kudu/blob/037f72b3/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 5a0516a..13fd87f 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -60,7 +60,6 @@ namespace tablet {
 class TabletMetadata;
 class TabletReplica;
 class TabletStatusPB;
-class TabletStatusListener;
 }
 
 namespace tserver {