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

[1/3] kudu git commit: [mini-cluster] allow 18-bit PID in LOOPBACK bind mode

Repository: kudu
Updated Branches:
  refs/heads/master edae025c4 -> f2c21b4fa


[mini-cluster] allow 18-bit PID in LOOPBACK bind mode

This changelist allows to use 18-bit PIDs for binding in LOOPBACK mode.
Prior to this patch, the minicluster could not use PIDs wider
than 16 bits.

While expanding the range of acceptable PIDs, this changelist also
decreases the maximum allowed number of 'indexed servers' from 254
to 62.  As of now, that's enough for our minicluster-based tests.

The incentive to put up this patch was change of the max PID
up to 147456 in /proc/sys/kernel/pid_max after recent restart of
one of our build servers.

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


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

Branch: refs/heads/master
Commit: 0bd8800b77a5de53b5db896df20abb83ff2e1779
Parents: edae025
Author: Alexey Serbin <as...@cloudera.com>
Authored: Mon Oct 30 15:39:30 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Nov 1 04:43:07 2017 +0000

----------------------------------------------------------------------
 src/kudu/mini-cluster/mini_cluster.cc | 27 ++++++++++++++++++++-------
 1 file changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0bd8800b/src/kudu/mini-cluster/mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/mini_cluster.cc b/src/kudu/mini-cluster/mini_cluster.cc
index 1da43e5..b477e79 100644
--- a/src/kudu/mini-cluster/mini_cluster.cc
+++ b/src/kudu/mini-cluster/mini_cluster.cc
@@ -34,15 +34,28 @@ namespace kudu {
 namespace cluster {
 
 string MiniCluster::GetBindIpForDaemon(DaemonType type, int index, BindMode bind_mode) {
+  static const int kPidBits = 18;
+  static const int kServerIdxBits = 24 - kPidBits;
+  static const int kServersMaxNum = (1 << kServerIdxBits) - 2;
+  CHECK(0 <= index && index < kServersMaxNum) << Substitute(
+      "server index $0 is not in range [$1, $2)", index, 0, kServersMaxNum);
+
   switch (bind_mode) {
     case UNIQUE_LOOPBACK: {
-      // IP address last octet range: [1 - 254].
-      uint8_t last_octet = (type == TSERVER) ? index + 1 : UINT8_MAX - 1 - index;
-      CHECK_GE(last_octet, 1);
-      CHECK_LE(last_octet, 254);
-      pid_t p = getpid();
-      CHECK_LE(p, UINT16_MAX) << "Cannot run on systems with >16-bit pid";
-      return Substitute("127.$0.$1.$2", p >> 8, p & 0xff, last_octet);
+      uint32_t pid = getpid();
+      CHECK_LT(pid, 1 << kPidBits) << Substitute(
+          "PID $0 is more than $1 bits wide", pid, kPidBits);
+      int idx = (type == TSERVER) ? index + 1 : kServersMaxNum - index;
+      uint32_t ip = (pid << kServerIdxBits) | static_cast<uint32_t>(idx);
+      uint8_t octets[] = {
+          static_cast<uint8_t>((ip >> 16) & 0xff),
+          static_cast<uint8_t>((ip >>  8) & 0xff),
+          static_cast<uint8_t>((ip >>  0) & 0xff),
+      };
+      // Range for the last octet of a valid unicast IP address is (0, 255).
+      CHECK(0 < octets[2] && octets[2] < UINT8_MAX) << Substitute(
+          "last IP octet $0 is not in range ($1, $2)", octets[2], 0, UINT8_MAX);
+      return Substitute("127.$0.$1.$2", octets[0], octets[1], octets[2]);
     }
     case WILDCARD:
       return kWildcardIpAddr;


[2/3] kudu git commit: KUDU-2193 (part 1): switch to a waiting mutex in TSTabletManager

Posted by to...@apache.org.
KUDU-2193 (part 1): switch to a waiting mutex in TSTabletManager

The TSTabletManager map is sometimes held for a long time. Given that, a
sleeping mutex is more appropriate than a spinlock.

Change-Id: I763abddd74d8b1dabb618318dc84256b533077e3
Reviewed-on: http://gerrit.cloudera.org:8080/8345
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/94bf6996
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/94bf6996
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/94bf6996

Branch: refs/heads/master
Commit: 94bf699645120bbe2a0e9f3d3a33ff65b8214dce
Parents: 0bd8800
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Oct 19 16:46:36 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Nov 1 04:43:50 2017 +0000

----------------------------------------------------------------------
 src/kudu/tserver/ts_tablet_manager.cc | 40 +++++++++++++++---------------
 src/kudu/tserver/ts_tablet_manager.h  |  9 ++++---
 2 files changed, 25 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/94bf6996/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 56a9478..5ff61d5 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -299,7 +299,7 @@ Status TSTabletManager::Init() {
   for (const scoped_refptr<TabletMetadata>& meta : metas) {
     scoped_refptr<TransitionInProgressDeleter> deleter;
     {
-      std::lock_guard<rw_spinlock> lock(lock_);
+      std::lock_guard<RWMutex> lock(lock_);
       CHECK_OK(StartTabletStateTransitionUnlocked(meta->tablet_id(), "opening tablet", &deleter));
     }
 
@@ -310,7 +310,7 @@ Status TSTabletManager::Init() {
   }
 
   {
-    std::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<RWMutex> lock(lock_);
     state_ = MANAGER_RUNNING;
   }
 
@@ -322,7 +322,7 @@ Status TSTabletManager::WaitForAllBootstrapsToFinish() {
 
   open_tablet_pool_->Wait();
 
-  shared_lock<rw_spinlock> l(lock_);
+  shared_lock<RWMutex> l(lock_);
   for (const TabletMap::value_type& entry : tablet_map_) {
     if (entry.second->state() == tablet::FAILED) {
       return entry.second->error();
@@ -350,7 +350,7 @@ Status TSTabletManager::CreateNewTablet(const string& table_id,
   {
     // acquire the lock in exclusive mode as we'll add a entry to the
     // transition_in_progress_ set if the lookup fails.
-    std::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<RWMutex> lock(lock_);
     TRACE("Acquired tablet manager lock");
 
     // Sanity check that the tablet isn't already registered.
@@ -475,7 +475,7 @@ void TSTabletManager::StartTabletCopy(
   boost::optional<string> transition;
   {
     // Lock must be dropped before executing callbacks.
-    shared_lock<rw_spinlock> lock(lock_);
+    shared_lock<RWMutex> lock(lock_);
     auto* t = FindOrNull(transition_in_progress_, tablet_id);
     if (t) {
       transition = *t;
@@ -539,7 +539,7 @@ void TSTabletManager::RunTabletCopy(
   bool replacing_tablet = false;
   scoped_refptr<TransitionInProgressDeleter> deleter;
   {
-    std::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<RWMutex> lock(lock_);
     if (LookupTabletUnlocked(tablet_id, &old_replica)) {
       meta = old_replica->tablet_metadata();
       replacing_tablet = true;
@@ -757,7 +757,7 @@ Status TSTabletManager::DeleteTablet(
   {
     // Acquire the lock in exclusive mode as we'll add a entry to the
     // transition_in_progress_ map.
-    std::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<RWMutex> lock(lock_);
     TRACE("Acquired tablet manager lock");
     RETURN_NOT_OK(CheckRunningUnlocked(error_code));
 
@@ -833,7 +833,7 @@ Status TSTabletManager::DeleteTablet(
   // Only DELETED tablets are fully shut down and removed from the tablet map.
   if (delete_type == TABLET_DATA_DELETED) {
     replica->Shutdown();
-    std::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<RWMutex> lock(lock_);
     RETURN_NOT_OK(CheckRunningUnlocked(error_code));
     CHECK_EQ(1, tablet_map_.erase(tablet_id)) << tablet_id;
     InsertOrDie(&perm_deleted_tablet_ids_, tablet_id);
@@ -863,7 +863,7 @@ Status TSTabletManager::StartTabletStateTransitionUnlocked(
     const string& tablet_id,
     const string& reason,
     scoped_refptr<TransitionInProgressDeleter>* deleter) {
-  DCHECK(lock_.is_write_locked());
+  lock_.AssertAcquiredForWriting();
   if (ContainsKey(perm_deleted_tablet_ids_, tablet_id)) {
     // When a table is deleted, the master sends a DeleteTablet() RPC to every
     // replica of every tablet with the TABLET_DATA_DELETED parameter, which
@@ -1000,7 +1000,7 @@ void TSTabletManager::OpenTablet(const scoped_refptr<TabletReplica>& replica,
 
 void TSTabletManager::Shutdown() {
   {
-    std::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<RWMutex> lock(lock_);
     switch (state_) {
       case MANAGER_QUIESCING: {
         VLOG(1) << "Tablet manager shut down already in progress..";
@@ -1040,7 +1040,7 @@ void TSTabletManager::Shutdown() {
   }
 
   {
-    std::lock_guard<rw_spinlock> l(lock_);
+    std::lock_guard<RWMutex> l(lock_);
     // We don't expect anyone else to be modifying the map after we start the
     // shut down process.
     CHECK_EQ(tablet_map_.size(), replicas_to_shutdown.size())
@@ -1054,7 +1054,7 @@ void TSTabletManager::Shutdown() {
 void TSTabletManager::RegisterTablet(const std::string& tablet_id,
                                      const scoped_refptr<TabletReplica>& replica,
                                      RegisterTabletReplicaMode mode) {
-  std::lock_guard<rw_spinlock> lock(lock_);
+  std::lock_guard<RWMutex> lock(lock_);
   // If we are replacing a tablet replica, we delete the existing one first.
   if (mode == REPLACEMENT_REPLICA && tablet_map_.erase(tablet_id) != 1) {
     LOG(FATAL) << "Unable to remove previous tablet replica " << tablet_id << ": not registered!";
@@ -1070,7 +1070,7 @@ void TSTabletManager::RegisterTablet(const std::string& tablet_id,
 
 bool TSTabletManager::LookupTablet(const string& tablet_id,
                                    scoped_refptr<TabletReplica>* replica) const {
-  shared_lock<rw_spinlock> l(lock_);
+  shared_lock<RWMutex> l(lock_);
   return LookupTabletUnlocked(tablet_id, replica);
 }
 
@@ -1097,7 +1097,7 @@ const NodeInstancePB& TSTabletManager::NodeInstance() const {
 }
 
 void TSTabletManager::GetTabletReplicas(vector<scoped_refptr<TabletReplica> >* replicas) const {
-  shared_lock<rw_spinlock> l(lock_);
+  shared_lock<RWMutex> l(lock_);
   AppendValuesFromMap(tablet_map_, replicas);
 }
 
@@ -1111,7 +1111,7 @@ void TSTabletManager::MarkTabletDirty(const std::string& tablet_id, const std::s
 
 int TSTabletManager::GetNumLiveTablets() const {
   int count = 0;
-  shared_lock<rw_spinlock> l(lock_);
+  shared_lock<RWMutex> l(lock_);
   for (const auto& entry : tablet_map_) {
     tablet::TabletStatePB state = entry.second->state();
     if (state == tablet::BOOTSTRAPPING ||
@@ -1151,7 +1151,7 @@ void TSTabletManager::CreateReportedTabletPB(const string& tablet_id,
 }
 
 void TSTabletManager::PopulateFullTabletReport(TabletReportPB* report) const {
-  shared_lock<rw_spinlock> shared_lock(lock_);
+  shared_lock<RWMutex> shared_lock(lock_);
   for (const auto& e : tablet_map_) {
     CreateReportedTabletPB(e.first, e.second, report->add_updated_tablets());
   }
@@ -1159,7 +1159,7 @@ void TSTabletManager::PopulateFullTabletReport(TabletReportPB* report) const {
 
 void TSTabletManager::PopulateIncrementalTabletReport(TabletReportPB* report,
                                                       const vector<string>& tablet_ids) const {
-  shared_lock<rw_spinlock> shared_lock(lock_);
+  shared_lock<RWMutex> shared_lock(lock_);
   for (const auto& id : tablet_ids) {
     const scoped_refptr<tablet::TabletReplica>* replica =
         FindOrNull(tablet_map_, id);
@@ -1291,7 +1291,7 @@ void TSTabletManager::FailTabletsInDataDir(const string& uuid) {
 
 int TSTabletManager::RefreshTabletStateCacheAndReturnCount(tablet::TabletStatePB st) {
   MonoDelta period = MonoDelta::FromMilliseconds(FLAGS_tablet_state_walk_min_period_ms);
-  std::lock_guard<rw_spinlock> lock(lock_);
+  std::lock_guard<RWMutex> lock(lock_);
   if (last_walked_ + period < MonoTime::Now()) {
     // Old cache: regenerate counts.
     tablet_state_counts_.clear();
@@ -1304,11 +1304,11 @@ int TSTabletManager::RefreshTabletStateCacheAndReturnCount(tablet::TabletStatePB
 }
 
 TransitionInProgressDeleter::TransitionInProgressDeleter(
-    TransitionInProgressMap* map, rw_spinlock* lock, string entry)
+    TransitionInProgressMap* map, RWMutex* lock, string entry)
     : in_progress_(map), lock_(lock), entry_(std::move(entry)) {}
 
 TransitionInProgressDeleter::~TransitionInProgressDeleter() {
-  std::lock_guard<rw_spinlock> lock(*lock_);
+  std::lock_guard<RWMutex> lock(*lock_);
   CHECK(in_progress_->erase(entry_));
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/94bf6996/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 1b5f24d..6c9cd28 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -39,6 +39,7 @@
 #include "kudu/util/locks.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/rw_mutex.h"
 #include "kudu/util/status.h"
 
 namespace boost {
@@ -292,7 +293,7 @@ class TSTabletManager : public tserver::TabletReplicaLookupIf {
                                  int64_t last_logged_term);
 
   TSTabletManagerStatePB state() const {
-    shared_lock<rw_spinlock> l(lock_);
+    shared_lock<RWMutex> l(lock_);
     return state_;
   }
 
@@ -315,7 +316,7 @@ class TSTabletManager : public tserver::TabletReplicaLookupIf {
   // Lock protecting tablet_map_, dirty_tablets_, state_,
   // transition_in_progress_, perm_deleted_tablet_ids_,
   // tablet_state_counts_, and last_walked_.
-  mutable rw_spinlock lock_;
+  mutable RWMutex lock_;
 
   // Map from tablet ID to tablet
   TabletMap tablet_map_;
@@ -357,7 +358,7 @@ class TSTabletManager : public tserver::TabletReplicaLookupIf {
 // when tablet bootstrap, create, and delete operations complete.
 class TransitionInProgressDeleter : public RefCountedThreadSafe<TransitionInProgressDeleter> {
  public:
-  TransitionInProgressDeleter(TransitionInProgressMap* map, rw_spinlock* lock,
+  TransitionInProgressDeleter(TransitionInProgressMap* map, RWMutex* lock,
                               std::string entry);
 
  private:
@@ -365,7 +366,7 @@ class TransitionInProgressDeleter : public RefCountedThreadSafe<TransitionInProg
   ~TransitionInProgressDeleter();
 
   TransitionInProgressMap* const in_progress_;
-  rw_spinlock* const lock_;
+  RWMutex* const lock_;
   const std::string entry_;
 };
 


[3/3] kudu git commit: KUDU-2193 (part 2): avoid holding TSTabletManager::lock_ for a long time

Posted by to...@apache.org.
KUDU-2193 (part 2): avoid holding TSTabletManager::lock_ for a long time

This changes tablet report generation to only hold the TSTabletManager lock
long enough to copy a list of refs to the relevant tablets.

Even though the lock is a reader-writer lock, a long read-side critical
section can end up blocking other readers as long as any writer shows up.
I saw the following in a cluster:

- election storm ongoing
- T1 generating a tablet report (thus holding the reader lock)
  - blocks for a long time getting ConsensusState from some tablets currently
    mid-fsync.
- T2 handling a DeleteTablet or CreateTablet call (waiting for writer lock)
- T3 through T20: blocking on reader lock in LookupTablet()

The effect here is that all other threads end up blocked until T1 finishes
its tablet report generation, which in this case can be tens of seconds due
to all of the fsync traffic. These blocked threads then contribute to the
ongoing election storm since they may delay timely responses to vote requests
from other tablets.

I tested this and the previous patch on a cluster that was previously
experiencing the issue. I triggered some elections by kill -STOPping some
servers and resuming them a few seconds later. Without these patches,
other servers started doing lots of context switches (due to the spinlock
used prior to this patch series) and I saw lots of blocked threads in
pstack. With the patch, things seem cleaner.

The following screenshot shows before/after voluntary_context_switch rate:

https://i.imgur.com/7zcbImw.png

(the missing data around 5pm is my restarting the servers with this patch)

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


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

Branch: refs/heads/master
Commit: f2c21b4fac15fcaa5f53a6c7960ecd19a0664c45
Parents: 94bf699
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Oct 19 17:29:41 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Nov 1 04:44:02 2017 +0000

----------------------------------------------------------------------
 src/kudu/tserver/ts_tablet_manager.cc | 46 +++++++++++++++++++-----------
 src/kudu/tserver/ts_tablet_manager.h  |  3 +-
 2 files changed, 31 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f2c21b4f/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 5ff61d5..47eae78 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -1131,10 +1131,9 @@ void TSTabletManager::InitLocalRaftPeerPB() {
   CHECK_OK(HostPortToPB(hp, local_peer_pb_.mutable_last_known_addr()));
 }
 
-void TSTabletManager::CreateReportedTabletPB(const string& tablet_id,
-                                             const scoped_refptr<TabletReplica>& replica,
+void TSTabletManager::CreateReportedTabletPB(const scoped_refptr<TabletReplica>& replica,
                                              ReportedTabletPB* reported_tablet) const {
-  reported_tablet->set_tablet_id(tablet_id);
+  reported_tablet->set_tablet_id(replica->tablet_id());
   reported_tablet->set_state(replica->state());
   reported_tablet->set_tablet_data_state(replica->tablet_metadata()->tablet_data_state());
   const Status& error = replica->error();
@@ -1151,26 +1150,41 @@ void TSTabletManager::CreateReportedTabletPB(const string& tablet_id,
 }
 
 void TSTabletManager::PopulateFullTabletReport(TabletReportPB* report) const {
-  shared_lock<RWMutex> shared_lock(lock_);
-  for (const auto& e : tablet_map_) {
-    CreateReportedTabletPB(e.first, e.second, report->add_updated_tablets());
+  // Creating the tablet report can be slow in the case that it is in the
+  // middle of flushing its consensus metadata. We don't want to hold
+  // lock_ for too long, even in read mode, since it can cause other readers
+  // to block if there is a waiting writer (see KUDU-2193). So, we just make
+  // a local copy of the set of replicas.
+  vector<scoped_refptr<tablet::TabletReplica>> to_report;
+  GetTabletReplicas(&to_report);
+  for (const auto& replica : to_report) {
+    CreateReportedTabletPB(replica, report->add_updated_tablets());
   }
 }
 
 void TSTabletManager::PopulateIncrementalTabletReport(TabletReportPB* report,
                                                       const vector<string>& tablet_ids) const {
-  shared_lock<RWMutex> shared_lock(lock_);
-  for (const auto& id : tablet_ids) {
-    const scoped_refptr<tablet::TabletReplica>* replica =
-        FindOrNull(tablet_map_, id);
-    if (replica) {
-      // Dirty entry, report on it.
-      CreateReportedTabletPB(id, *replica, report->add_updated_tablets());
-    } else {
-      // Removed.
-      report->add_removed_tablet_ids(id);
+  // See comment in PopulateFullTabletReport for rationale on making a local
+  // copy of the set of tablets to report.
+  vector<scoped_refptr<tablet::TabletReplica>> to_report;
+  to_report.reserve(tablet_ids.size());
+  {
+    shared_lock<RWMutex> shared_lock(lock_);
+    for (const auto& id : tablet_ids) {
+      const scoped_refptr<tablet::TabletReplica>* replica =
+          FindOrNull(tablet_map_, id);
+      if (replica) {
+        // Dirty entry, report on it.
+        to_report.push_back(*replica);
+      } else {
+        // Removed.
+        report->add_removed_tablet_ids(id);
+      }
     }
   }
+  for (const auto& replica : to_report) {
+    CreateReportedTabletPB(replica, report->add_updated_tablets());
+  }
 }
 
 Status TSTabletManager::HandleNonReadyTabletOnStartup(const scoped_refptr<TabletMetadata>& meta) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/f2c21b4f/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 6c9cd28..317ab9f 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -278,8 +278,7 @@ class TSTabletManager : public tserver::TabletReplicaLookupIf {
                                         scoped_refptr<tablet::TabletReplica>* replica_out);
 
   // Helper to generate the report for a single tablet.
-  void CreateReportedTabletPB(const std::string& tablet_id,
-                              const scoped_refptr<tablet::TabletReplica>& replica,
+  void CreateReportedTabletPB(const scoped_refptr<tablet::TabletReplica>& replica,
                               master::ReportedTabletPB* reported_tablet) const;
 
   // Handle the case on startup where we find a tablet that is not in