Skip to content

Commit

Permalink
metrics: replace kudu::Bind usage with lambdas in FunctionGauge
Browse files Browse the repository at this point in the history
Let's start with a little history lesson. Back in 2014, we imported
Chromium's Bind (and associated machinery) into gutil as kudu::Bind. At the
time we hadn't migrated to C++11, so we couldn't use lambdas, and the
kudu::Bind feature set was rich[1], at least as compared to boost::bind.

Now that we're on C++11 and can use lambdas, there's very little that
kudu::Bind can do that lambdas can't. The only thing that comes to mind is
passing ownership of an object to a callback via the kudu::Owned() or
kudu::Passed() wrappers, and we're not using those features. So in the
interest of modernizing the Kudu codebase a bit, I'm going to replace
kudu::Bind with lambdas. Lambdas are simpler, can be more easily inlined by
the compiler, and will result in a reduction of code at the end when we
remove kudu::Bind and friends from the repo.

This patch kicks off the conversion process by using lambdas in
FunctionGauge, which is a heavily used source of kudu::Bind.

1. See commit 66282df.

Change-Id: If60a0dabde1fde1465d73dd9e8edc244fc780c37
Reviewed-on: http://gerrit.cloudera.org:8080/15474
Reviewed-by: Bankim Bhavsar <[email protected]>
Tested-by: Adar Dembo <[email protected]>
Reviewed-by: Andrew Wong <[email protected]>
  • Loading branch information
adembo committed Mar 19, 2020
1 parent a626e27 commit 7dd10ba
Show file tree
Hide file tree
Showing 21 changed files with 131 additions and 157 deletions.
17 changes: 6 additions & 11 deletions src/kudu/clock/builtin_ntp.cc
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,6 @@
#include <glog/logging.h>

#include "kudu/clock/builtin_ntp-internal.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/port.h"
#include "kudu/gutil/strings/join.h"
#include "kudu/gutil/strings/strcat.h"
Expand Down Expand Up @@ -1108,17 +1106,14 @@ Status BuiltInNtp::CombineClocks() {

void BuiltInNtp::RegisterMetrics(const scoped_refptr<MetricEntity>& entity) {
METRIC_builtin_ntp_local_clock_delta.InstantiateFunctionGauge(
entity,
Bind(&BuiltInNtp::LocalClockDeltaForMetrics, Unretained(this)))->
AutoDetachToLastValue(&metric_detacher_);
entity, [this]() { return this->LocalClockDeltaForMetrics(); })->
AutoDetachToLastValue(&metric_detacher_);
METRIC_builtin_ntp_time.InstantiateFunctionGauge(
entity,
Bind(&BuiltInNtp::WalltimeForMetrics, Unretained(this)))->
AutoDetachToLastValue(&metric_detacher_);
entity, [this]() { return this->WalltimeForMetrics(); })->
AutoDetachToLastValue(&metric_detacher_);
METRIC_builtin_ntp_error.InstantiateFunctionGauge(
entity,
Bind(&BuiltInNtp::MaxErrorForMetrics, Unretained(this)))->
AutoDetachToLastValue(&metric_detacher_);
entity, [this]() { return this->MaxErrorForMetrics(); })->
AutoDetachToLastValue(&metric_detacher_);
max_errors_histogram_ =
METRIC_builtin_ntp_max_errors.Instantiate(entity);
}
Expand Down
8 changes: 2 additions & 6 deletions src/kudu/clock/hybrid_clock.cc
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,6 @@
#include "kudu/clock/mock_ntp.h"
#include "kudu/clock/system_ntp.h"
#include "kudu/clock/system_unsync_time.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/macros.h"
#include "kudu/gutil/port.h"
#include "kudu/gutil/strings/substitute.h"
Expand Down Expand Up @@ -605,12 +603,10 @@ Status HybridClock::InitWithTimeSource(TimeSource time_source,
// values when the object is not ready for that yet. Gauges produce special
// values for corresponding metrics, but there is no value in that anyways.
METRIC_hybrid_clock_timestamp.InstantiateFunctionGauge(
metric_entity_,
Bind(&HybridClock::NowForMetrics, Unretained(this)))->
metric_entity_, [this]() { return this->NowForMetrics(); })->
AutoDetachToLastValue(&metric_detacher_);
METRIC_hybrid_clock_error.InstantiateFunctionGauge(
metric_entity_,
Bind(&HybridClock::ErrorForMetrics, Unretained(this)))->
metric_entity_, [this]() { return this->ErrorForMetrics(); })->
AutoDetachToLastValue(&metric_detacher_);

return Status::OK();
Expand Down
6 changes: 2 additions & 4 deletions src/kudu/clock/logical_clock.cc
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,11 @@
#include <memory>
#include <ostream>
#include <string>
#include <utility>

#include <glog/logging.h>

#include "kudu/gutil/atomicops.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/port.h"
#include "kudu/gutil/strings/substitute.h"
#include "kudu/util/metrics.h"
Expand All @@ -51,8 +50,7 @@ LogicalClock::LogicalClock(const Timestamp& timestamp,
: now_(timestamp.value() - 1) {
if (metric_entity) {
METRIC_logical_clock_timestamp.InstantiateFunctionGauge(
metric_entity,
Bind(&LogicalClock::GetCurrentTime, Unretained(this)))->
metric_entity, [this]() { return this->GetCurrentTime(); })->
AutoDetachToLastValue(&metric_detacher_);
}
}
Expand Down
15 changes: 4 additions & 11 deletions src/kudu/codegen/compilation_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,6 @@
#include "kudu/codegen/jit_wrapper.h"
#include "kudu/codegen/row_projector.h"
#include "kudu/common/schema.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/callback.h"
#include "kudu/gutil/casts.h"
#include "kudu/gutil/ref_counted.h"
#include "kudu/util/faststring.h"
Expand Down Expand Up @@ -162,16 +159,12 @@ Status CompilationManager::StartInstrumentation(const scoped_refptr<MetricEntity
// register the same metric in multiple registries. Using a gauge which loads
// an atomic int is a suitable workaround: each TS's registry ends up with a
// unique gauge which reads the value of the singleton's integer.
Callback<int64_t(void)> hits = Bind(&AtomicInt<int64_t>::Load,
Unretained(&hit_counter_),
kMemOrderNoBarrier);
Callback<int64_t(void)> queries = Bind(&AtomicInt<int64_t>::Load,
Unretained(&query_counter_),
kMemOrderNoBarrier);
metric_entity->NeverRetire(
METRIC_code_cache_hits.InstantiateFunctionGauge(metric_entity, hits));
METRIC_code_cache_hits.InstantiateFunctionGauge(
metric_entity, [this]() { return this->hit_counter_.Load(kMemOrderNoBarrier); }));
metric_entity->NeverRetire(
METRIC_code_cache_queries.InstantiateFunctionGauge(metric_entity, queries));
METRIC_code_cache_queries.InstantiateFunctionGauge(
metric_entity, [this]() { return this->query_counter_.Load(kMemOrderNoBarrier); }));
return Status::OK();
}

Expand Down
1 change: 1 addition & 0 deletions src/kudu/consensus/consensus-test-util.h
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
#include "kudu/consensus/log.h"
#include "kudu/consensus/opid_util.h"
#include "kudu/consensus/raft_consensus.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/map-util.h"
#include "kudu/gutil/strings/substitute.h"
#include "kudu/rpc/messenger.h"
Expand Down
6 changes: 2 additions & 4 deletions src/kudu/consensus/raft_consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@
#include "kudu/consensus/quorum_util.h"
#include "kudu/consensus/time_manager.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/macros.h"
#include "kudu/gutil/map-util.h"
#include "kudu/gutil/port.h"
Expand Down Expand Up @@ -249,10 +248,9 @@ Status RaftConsensus::Start(const ConsensusBootstrapInfo& info,
failed_elections_since_stable_leader_);

METRIC_time_since_last_leader_heartbeat.InstantiateFunctionGauge(
metric_entity,
Bind(&RaftConsensus::GetMillisSinceLastLeaderHeartbeat, Unretained(this)),
metric_entity, [this]() { return this->GetMillisSinceLastLeaderHeartbeat(); },
MergeType::kMax)
->AutoDetach(&metric_detacher_);
->AutoDetach(&metric_detacher_);

// A single Raft thread pool token is shared between RaftConsensus and
// PeerManager. Because PeerManager is owned by RaftConsensus, it receives a
Expand Down
7 changes: 2 additions & 5 deletions src/kudu/master/ts_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,6 @@

#include "kudu/common/common.pb.h"
#include "kudu/common/wire_protocol.pb.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/map-util.h"
#include "kudu/gutil/port.h"
#include "kudu/gutil/strings/substitute.h"
Expand Down Expand Up @@ -97,9 +95,8 @@ TSManager::TSManager(LocationCache* location_cache,
: ts_state_lock_(RWMutex::Priority::PREFER_READING),
location_cache_(location_cache) {
METRIC_cluster_replica_skew.InstantiateFunctionGauge(
metric_entity,
Bind(&TSManager::ClusterSkew, Unretained(this)))
->AutoDetach(&metric_detacher_);
metric_entity, [this]() { return this->ClusterSkew(); })
->AutoDetach(&metric_detacher_);
}

TSManager::~TSManager() {
Expand Down
1 change: 1 addition & 0 deletions src/kudu/rpc/retriable_rpc.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

#include <boost/bind.hpp>

#include "kudu/gutil/bind.h"
#include "kudu/gutil/ref_counted.h"
#include "kudu/gutil/strings/substitute.h"
#include "kudu/rpc/messenger.h"
Expand Down
28 changes: 19 additions & 9 deletions src/kudu/server/tcmalloc_metrics.cc
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,13 @@
#include <cstddef>
#include <cstdint>
#include <ostream>
#include <utility>

#include <glog/logging.h>
#ifdef TCMALLOC_ENABLED
#include <gperftools/malloc_extension.h>
#endif

#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/util/metrics.h"

#ifndef TCMALLOC_ENABLED
Expand Down Expand Up @@ -96,23 +95,34 @@ static uint64_t GetTCMallocPropValue(const char* prop) {
void RegisterMetrics(const scoped_refptr<MetricEntity>& entity) {
entity->NeverRetire(
METRIC_generic_current_allocated_bytes.InstantiateFunctionGauge(
entity, Bind(GetTCMallocPropValue, Unretained("generic.current_allocated_bytes"))));
entity, []() {
return GetTCMallocPropValue("generic.current_allocated_bytes");
}));
entity->NeverRetire(
METRIC_generic_heap_size.InstantiateFunctionGauge(
entity, Bind(GetTCMallocPropValue, Unretained("generic.heap_size"))));
entity, []() {
return GetTCMallocPropValue("generic.heap_size");
}));
entity->NeverRetire(
METRIC_tcmalloc_pageheap_free_bytes.InstantiateFunctionGauge(
entity, Bind(GetTCMallocPropValue, Unretained("tcmalloc.pageheap_free_bytes"))));
entity, []() {
return GetTCMallocPropValue("tcmalloc.pageheap_free_bytes");
}));
entity->NeverRetire(
METRIC_tcmalloc_pageheap_unmapped_bytes.InstantiateFunctionGauge(
entity, Bind(GetTCMallocPropValue, Unretained("tcmalloc.pageheap_unmapped_bytes"))));
entity, []() {
return GetTCMallocPropValue("tcmalloc.pageheap_unmapped_bytes");
}));
entity->NeverRetire(
METRIC_tcmalloc_max_total_thread_cache_bytes.InstantiateFunctionGauge(
entity, Bind(GetTCMallocPropValue, Unretained("tcmalloc.max_total_thread_cache_bytes"))));
entity, []() {
return GetTCMallocPropValue("tcmalloc.max_total_thread_cache_bytes");
}));
entity->NeverRetire(
METRIC_tcmalloc_current_total_thread_cache_bytes.InstantiateFunctionGauge(
entity, Bind(GetTCMallocPropValue,
Unretained("tcmalloc.current_total_thread_cache_bytes"))));
entity, []() {
return GetTCMallocPropValue("tcmalloc.current_total_thread_cache_bytes");
}));
}

} // namespace tcmalloc
Expand Down
23 changes: 12 additions & 11 deletions src/kudu/tablet/tablet.cc
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@
#include "kudu/fs/fs_manager.h"
#include "kudu/fs/io_context.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/casts.h"
#include "kudu/gutil/stl_util.h"
#include "kudu/gutil/strings/human_readable.h"
Expand Down Expand Up @@ -240,20 +239,22 @@ Tablet::Tablet(scoped_refptr<TabletMetadata> metadata,
metric_entity_ = METRIC_ENTITY_tablet.Instantiate(metric_registry, tablet_id(), attrs);
metrics_.reset(new TabletMetrics(metric_entity_));
METRIC_memrowset_size.InstantiateFunctionGauge(
metric_entity_, Bind(&Tablet::MemRowSetSize, Unretained(this)))
->AutoDetach(&metric_detacher_);
metric_entity_, [this]() { return this->MemRowSetSize(); })
->AutoDetach(&metric_detacher_);
METRIC_on_disk_data_size.InstantiateFunctionGauge(
metric_entity_, Bind(&Tablet::OnDiskDataSize, Unretained(this)))
->AutoDetach(&metric_detacher_);
metric_entity_, [this]() { return this->OnDiskDataSize(); })
->AutoDetach(&metric_detacher_);
METRIC_num_rowsets_on_disk.InstantiateFunctionGauge(
metric_entity_, Bind(&Tablet::num_rowsets, Unretained(this)))
->AutoDetach(&metric_detacher_);
metric_entity_, [this]() { return this->num_rowsets(); })
->AutoDetach(&metric_detacher_);
METRIC_last_read_elapsed_time.InstantiateFunctionGauge(
metric_entity_, Bind(&Tablet::LastReadElapsedSeconds, Unretained(this)), MergeType::kMin)
->AutoDetach(&metric_detacher_);
metric_entity_, [this]() { return this->LastReadElapsedSeconds(); },
MergeType::kMin)
->AutoDetach(&metric_detacher_);
METRIC_last_write_elapsed_time.InstantiateFunctionGauge(
metric_entity_, Bind(&Tablet::LastWriteElapsedSeconds, Unretained(this)), MergeType::kMin)
->AutoDetach(&metric_detacher_);
metric_entity_, [this]() { return this->LastWriteElapsedSeconds(); },
MergeType::kMin)
->AutoDetach(&metric_detacher_);
}

if (FLAGS_tablet_throttler_rpc_per_sec > 0 || FLAGS_tablet_throttler_bytes_per_sec > 0) {
Expand Down
4 changes: 2 additions & 2 deletions src/kudu/tablet/tablet_replica-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@
#include "kudu/fs/fs_manager.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/callback.h"
#include "kudu/gutil/macros.h"
#include "kudu/gutil/ref_counted.h"
#include "kudu/rpc/messenger.h"
Expand Down Expand Up @@ -781,8 +780,9 @@ TEST_F(TabletReplicaTest, TestLiveRowCountMetric) {
ConsensusBootstrapInfo info;
ASSERT_OK(StartReplicaAndWaitUntilLeader(info));

// We don't care what the function is, since the metric is already instantiated.
auto live_row_count = METRIC_live_row_count.InstantiateFunctionGauge(
tablet_replica_->tablet()->GetMetricEntity(), Callback<uint64_t(void)>());
tablet_replica_->tablet()->GetMetricEntity(), [](){ return 0; });
ASSERT_EQ(0, live_row_count->value());

// Insert some rows.
Expand Down
9 changes: 3 additions & 6 deletions src/kudu/tablet/tablet_replica.cc
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,6 @@
#include "kudu/consensus/time_manager.h"
#include "kudu/fs/data_dirs.h"
#include "kudu/gutil/basictypes.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/port.h"
#include "kudu/gutil/stl_util.h"
#include "kudu/gutil/strings/substitute.h"
Expand Down Expand Up @@ -208,15 +206,14 @@ Status TabletReplica::Start(const ConsensusBootstrapInfo& bootstrap_info,
txn_tracker_.StartInstrumentation(tablet_->GetMetricEntity());

METRIC_on_disk_size.InstantiateFunctionGauge(
tablet_->GetMetricEntity(), Bind(&TabletReplica::OnDiskSize, Unretained(this)))
tablet_->GetMetricEntity(), [this]() { return this->OnDiskSize(); })
->AutoDetach(&metric_detacher_);
METRIC_state.InstantiateFunctionGauge(
tablet_->GetMetricEntity(), Bind(&TabletReplica::StateName, Unretained(this)))
tablet_->GetMetricEntity(), [this]() { return this->StateName(); })
->AutoDetach(&metric_detacher_);
if (tablet_->metadata()->supports_live_row_count()) {
METRIC_live_row_count.InstantiateFunctionGauge(
tablet_->GetMetricEntity(),
Bind(&TabletReplica::CountLiveRowsNoFail, Unretained(this)))
tablet_->GetMetricEntity(), [this]() { return this->CountLiveRowsNoFail(); })
->AutoDetach(&metric_detacher_);
} else {
METRIC_live_row_count.InstantiateInvalid(tablet_->GetMetricEntity(), 0);
Expand Down
7 changes: 2 additions & 5 deletions src/kudu/tserver/scanners.cc
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,6 @@
#include "kudu/common/iterator.h"
#include "kudu/common/scan_spec.h"
#include "kudu/common/schema.h"
#include "kudu/gutil/bind.h"
#include "kudu/gutil/bind_helpers.h"
#include "kudu/gutil/hash/string_hash.h"
#include "kudu/gutil/map-util.h"
#include "kudu/gutil/stl_util.h"
Expand Down Expand Up @@ -86,9 +84,8 @@ ScannerManager::ScannerManager(const scoped_refptr<MetricEntity>& metric_entity)
if (metric_entity) {
metrics_.reset(new ScannerMetrics(metric_entity));
METRIC_active_scanners.InstantiateFunctionGauge(
metric_entity, Bind(&ScannerManager::CountActiveScanners,
Unretained(this)))
->AutoDetach(&metric_detacher_);
metric_entity, [this]() { return this->CountActiveScanners(); })
->AutoDetach(&metric_detacher_);
}
for (size_t i = 0; i < kNumScannerMapStripes; i++) {
scanner_maps_.push_back(new ScannerMapStripe());
Expand Down
5 changes: 2 additions & 3 deletions src/kudu/tserver/tablet_server-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@
#include "kudu/fs/fs.pb.h"
#include "kudu/fs/fs_manager.h"
#include "kudu/gutil/basictypes.h"
#include "kudu/gutil/callback.h"
#include "kudu/gutil/casts.h"
#include "kudu/gutil/map-util.h"
#include "kudu/gutil/port.h"
Expand Down Expand Up @@ -1859,7 +1858,7 @@ TEST_F(TabletServerTest, TestReadLatest) {
ASSERT_TRUE(mini_server_->server()->metric_entity());
// We don't care what the function is, since the metric is already instantiated.
auto active_scanners = METRIC_active_scanners.InstantiateFunctionGauge(
mini_server_->server()->metric_entity(), Callback<size_t(void)>());
mini_server_->server()->metric_entity(), []() {return 0; });
scoped_refptr<TabletReplica> tablet;
ASSERT_TRUE(mini_server_->server()->tablet_manager()->LookupTablet(kTabletId, &tablet));
ASSERT_TRUE(tablet->tablet()->GetMetricEntity());
Expand Down Expand Up @@ -3991,7 +3990,7 @@ TEST_F(TabletServerTest, TestTabletNumberOfDiskRowSetsMetric) {

// We don't care what the function is, since the metric is already instantiated.
auto num_diskrowsets = METRIC_num_rowsets_on_disk.InstantiateFunctionGauge(
tablet->tablet()->GetMetricEntity(), Callback<size_t(void)>());
tablet->tablet()->GetMetricEntity(), []() { return 0; });

// No data, no diskrowsets.
ASSERT_EQ(0, num_diskrowsets->value());
Expand Down
Loading

0 comments on commit 7dd10ba

Please sign in to comment.