summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--metrics/src/vespa/metrics/metricmanager.cpp47
-rw-r--r--metrics/src/vespa/metrics/metricmanager.h34
-rw-r--r--metrics/src/vespa/metrics/updatehook.h10
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton.h4
-rw-r--r--storage/src/tests/bucketdb/bucketmanagertest.cpp3
-rw-r--r--storage/src/tests/distributor/distributortest.cpp13
-rw-r--r--storageframework/src/vespa/storageframework/defaultimplementation/component/componentregisterimpl.cpp3
-rw-r--r--storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h8
9 files changed, 55 insertions, 69 deletions
diff --git a/metrics/src/vespa/metrics/metricmanager.cpp b/metrics/src/vespa/metrics/metricmanager.cpp
index 7db9686d5c8..c3539452e81 100644
--- a/metrics/src/vespa/metrics/metricmanager.cpp
+++ b/metrics/src/vespa/metrics/metricmanager.cpp
@@ -32,9 +32,8 @@ MetricManager::Timer::getTime() const {
void
MetricManager::assertMetricLockLocked(const MetricLockGuard& g) const {
- if (!g.monitors(_waiter)) {
- throw vespalib::IllegalArgumentException(
- "Given lock does not lock the metric lock.", VESPA_STRLOC);
+ if ((g.mutex() != &_waiter) || !g.owns_lock()) {
+ throw vespalib::IllegalArgumentException("Given lock does not lock the metric lock.", VESPA_STRLOC);
}
}
@@ -99,10 +98,7 @@ MetricManager::stop()
return; // Let stop() be idempotent.
}
Runnable::stop();
- {
- vespalib::MonitorGuard sync(_waiter);
- sync.signal();
- }
+ _cond.notify_all();
join();
}
@@ -110,7 +106,7 @@ void
MetricManager::addMetricUpdateHook(UpdateHook& hook, uint32_t period)
{
hook._period = period;
- vespalib::MonitorGuard sync(_waiter);
+ std::lock_guard sync(_waiter);
// If we've already initialized manager, log period has been set.
// In this case. Call first time after period
hook._nextCall = _timer->getTime() + period;
@@ -136,7 +132,7 @@ MetricManager::addMetricUpdateHook(UpdateHook& hook, uint32_t period)
void
MetricManager::removeMetricUpdateHook(UpdateHook& hook)
{
- vespalib::MonitorGuard sync(_waiter);
+ std::lock_guard sync(_waiter);
if (hook._period == 0) {
for (auto it = _snapshotUpdateHooks.begin(); it != _snapshotUpdateHooks.end(); it++) {
if (*it == &hook) {
@@ -170,7 +166,7 @@ MetricManager::init(const config::ConfigUri & uri, FastOS_ThreadPool& pool,
"It can only be initialized once.", VESPA_STRLOC);
}
LOG(debug, "Initializing metric manager.");
- _configSubscriber.reset(new config::ConfigSubscriber(uri.getContext()));
+ _configSubscriber = std::make_unique<config::ConfigSubscriber>(uri.getContext());
_configHandle = _configSubscriber->subscribe<Config>(uri.getConfigId());
_configSubscriber->nextConfig();
configure(getMetricLock(), _configHandle->getConfig());
@@ -179,12 +175,12 @@ MetricManager::init(const config::ConfigUri & uri, FastOS_ThreadPool& pool,
Runnable::start(pool);
// Wait for first iteration to have completed, such that it is safe
// to access snapshots afterwards.
- vespalib::MonitorGuard sync(_waiter);
+ MetricLockGuard sync(_waiter);
while (_lastProcessedTime.load(std::memory_order_relaxed) == 0) {
- sync.wait(1);
+ _cond.wait_for(sync, 1ms);
}
} else {
- _configSubscriber.reset(0);
+ _configSubscriber.reset();
}
LOG(debug, "Metric manager completed initialization.");
}
@@ -647,8 +643,7 @@ MetricManager::getMetricSnapshotSet(const MetricLockGuard& l,
void
MetricManager::timeChangedNotification() const
{
- vespalib::MonitorGuard sync(_waiter);
- sync.broadcast();
+ _cond.notify_all();
}
void
@@ -657,13 +652,11 @@ MetricManager::updateMetrics(bool includeSnapshotOnlyHooks)
LOG(debug, "Calling metric update hooks%s.",
includeSnapshotOnlyHooks ? ", including snapshot hooks" : "");
// Ensure we're not in the way of the background thread
- vespalib::MonitorGuard sync(_waiter);
- LOG(debug, "Giving %zu periodic update hooks.",
- _periodicUpdateHooks.size());
+ MetricLockGuard sync(_waiter);
+ LOG(debug, "Giving %zu periodic update hooks.", _periodicUpdateHooks.size());
updatePeriodicMetrics(sync, 0, true);
if (includeSnapshotOnlyHooks) {
- LOG(debug, "Giving %zu snapshot update hooks.",
- _snapshotUpdateHooks.size());
+ LOG(debug, "Giving %zu snapshot update hooks.", _snapshotUpdateHooks.size());
updateSnapshotMetrics(sync);
}
}
@@ -720,9 +713,8 @@ void
MetricManager::forceEventLogging()
{
LOG(debug, "Forcing event logging to happen.");
- // Ensure background thread is not in a current cycle during change.
- vespalib::MonitorGuard sync(_waiter);
- sync.signal();
+ // Ensure background thread is not in a current cycle during change.
+ _cond.notify_all();
}
void
@@ -731,7 +723,7 @@ MetricManager::reset(time_t currentTime)
time_t preTime = _timer->getTimeInMilliSecs();
// Resetting implies visiting metrics, which needs to grab metric lock
// to avoid conflict with adding/removal of metrics
- vespalib::LockGuard waiterLock(_waiter);
+ std::lock_guard waiterLock(_waiter);
_activeMetrics.reset(currentTime);
for (uint32_t i=0; i<_snapshots.size(); ++i) {
_snapshots[i]->reset(currentTime);
@@ -744,7 +736,7 @@ MetricManager::reset(time_t currentTime)
void
MetricManager::run()
{
- vespalib::MonitorGuard sync(_waiter);
+ MetricLockGuard sync(_waiter);
// For a slow system to still be doing metrics tasks each n'th
// second, rather than each n'th + time to do something seconds,
// we constantly add next time to do something from the last timer.
@@ -763,8 +755,9 @@ MetricManager::run()
currentTime = _timer->getTime();
time_t next = tick(sync, currentTime);
if (currentTime < next) {
- sync.wait((next - currentTime) * 1000);
- _sleepTimes.addValue((next - currentTime) * 1000);
+ size_t ms = (next - currentTime) * 1000;
+ _cond.wait_for(sync, std::chrono::milliseconds(ms));
+ _sleepTimes.addValue(ms);
} else {
_sleepTimes.addValue(0);
}
diff --git a/metrics/src/vespa/metrics/metricmanager.h b/metrics/src/vespa/metrics/metricmanager.h
index 6d1f8f4dcf7..feedf2c1515 100644
--- a/metrics/src/vespa/metrics/metricmanager.h
+++ b/metrics/src/vespa/metrics/metricmanager.h
@@ -43,33 +43,31 @@
*/
#pragma once
-#include <string>
-#include <vespa/vespalib/util/document_runnable.h>
-#include <vespa/metrics/config-metricsmanager.h>
-#include <vespa/metrics/metricset.h>
-#include <vespa/metrics/metricsnapshot.h>
-#include <vespa/metrics/memoryconsumption.h>
-#include <vespa/metrics/valuemetric.h>
-#include <vespa/metrics/updatehook.h>
+#include "config-metricsmanager.h"
+#include "metricset.h"
+#include "metricsnapshot.h"
+#include "memoryconsumption.h"
+#include "valuemetric.h"
+#include "updatehook.h"
#include <vespa/vespalib/stllike/hash_set.h>
-#include <vespa/vespalib/util/sync.h>
-#include <map>
-#include <list>
+#include <vespa/vespalib/util/document_runnable.h>
#include <vespa/vespalib/util/jsonwriter.h>
#include <vespa/config/config.h>
+#include <map>
+#include <list>
template class vespalib::hash_set<metrics::Metric::String>;
namespace metrics {
-typedef vespalib::MonitorGuard MetricLockGuard;
+using MetricLockGuard = UpdateHook::MetricLockGuard;
class MetricManager : private document::Runnable
{
public:
struct Timer {
- virtual ~Timer() {}
+ virtual ~Timer() = default;
virtual time_t getTime() const;
virtual time_t getTimeInMilliSecs() const { return getTime() * 1000; }
};
@@ -82,8 +80,8 @@ public:
typedef std::shared_ptr<ConsumerSpec> SP;
vespalib::hash_set<Metric::String> includedMetrics;
- ConsumerSpec(ConsumerSpec &&) = default;
- ConsumerSpec & operator= (ConsumerSpec &&) = default;
+ ConsumerSpec(ConsumerSpec &&) noexcept = default;
+ ConsumerSpec & operator= (ConsumerSpec &&) noexcept = default;
ConsumerSpec();
~ConsumerSpec() override;
@@ -105,7 +103,8 @@ private:
std::map<Metric::String, ConsumerSpec::SP> _consumerConfig;
std::list<UpdateHook*> _periodicUpdateHooks;
std::list<UpdateHook*> _snapshotUpdateHooks;
- vespalib::Monitor _waiter;
+ mutable std::mutex _waiter;
+ mutable std::condition_variable _cond;
typedef std::pair<uint32_t, time_t> PeriodTimePair;
std::vector<MetricSnapshotSet::SP> _snapshots;
MetricSnapshot::SP _totalMetrics;
@@ -217,8 +216,7 @@ public:
* snapshots while you are accessing them.
*/
MetricLockGuard getMetricLock() const {
- MetricLockGuard m(_waiter);
- return m;
+ return MetricLockGuard(_waiter);
}
/** While accessing the active metrics you should have the metric lock. */
diff --git a/metrics/src/vespa/metrics/updatehook.h b/metrics/src/vespa/metrics/updatehook.h
index 031b36a1e5a..65f34c65664 100644
--- a/metrics/src/vespa/metrics/updatehook.h
+++ b/metrics/src/vespa/metrics/updatehook.h
@@ -1,7 +1,7 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#pragma once
-#include <vespa/vespalib/util/sync.h>
+#include <mutex>
namespace metrics {
@@ -9,16 +9,16 @@ class MetricManager;
class UpdateHook {
const char* _name;
- time_t _nextCall;
- uint32_t _period;
+ time_t _nextCall;
+ uint32_t _period;
friend class MetricManager;
public:
using UP = std::unique_ptr<UpdateHook>;
- using MetricLockGuard = vespalib::MonitorGuard;
+ using MetricLockGuard = std::unique_lock<std::mutex>;
UpdateHook(const char* name) : _name(name), _nextCall(0), _period(0) {}
- virtual ~UpdateHook() {}
+ virtual ~UpdateHook() = default;
virtual void updateMetrics(const MetricLockGuard & guard) = 0;
const char* getName() const { return _name; }
};
diff --git a/searchcore/src/vespa/searchcore/proton/server/proton.cpp b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
index 321662ea404..5817e729950 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
@@ -713,7 +713,7 @@ updateExecutorMetrics(ExecutorMetrics &metrics,
}
void
-Proton::updateMetrics(const vespalib::MonitorGuard &)
+Proton::updateMetrics(const metrics::UpdateHook::MetricLockGuard &)
{
{
ContentProtonMetrics &metrics = _metricsEngine->root();
diff --git a/searchcore/src/vespa/searchcore/proton/server/proton.h b/searchcore/src/vespa/searchcore/proton/server/proton.h
index 45556319e78..f88cd7bf0cd 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton.h
+++ b/searchcore/src/vespa/searchcore/proton/server/proton.h
@@ -66,7 +66,7 @@ private:
MetricsUpdateHook(Proton &s)
: metrics::UpdateHook("proton-hook"),
self(s) {}
- void updateMetrics(const vespalib::MonitorGuard &guard) override { self.updateMetrics(guard); }
+ void updateMetrics(const MetricLockGuard &guard) override { self.updateMetrics(guard); }
};
friend struct MetricsUpdateHook;
@@ -143,7 +143,7 @@ private:
* the metric manager). Do not call this function in multiple
* threads at once.
**/
- void updateMetrics(const vespalib::MonitorGuard &guard);
+ void updateMetrics(const metrics::UpdateHook::MetricLockGuard &guard);
void waitForInitDone();
void waitForOnlineState();
uint32_t getDistributionKey() const override { return _distributionKey; }
diff --git a/storage/src/tests/bucketdb/bucketmanagertest.cpp b/storage/src/tests/bucketdb/bucketmanagertest.cpp
index 678e3aa621c..9668eec404d 100644
--- a/storage/src/tests/bucketdb/bucketmanagertest.cpp
+++ b/storage/src/tests/bucketdb/bucketmanagertest.cpp
@@ -19,7 +19,6 @@
#include <tests/common/testhelper.h>
#include <vespa/vdslib/state/random.h>
#include <vespa/vespalib/io/fileutil.h>
-#include <vespa/vespalib/util/stringfmt.h>
#include <vespa/vespalib/gtest/gtest.h>
#include <future>
@@ -99,7 +98,7 @@ protected:
_manager->updateMinUsedBits();
}
void trigger_metric_manager_update() {
- vespalib::Monitor l;
+ std::mutex l;
_manager->updateMetrics(BucketManager::MetricLockGuard(l));
}
diff --git a/storage/src/tests/distributor/distributortest.cpp b/storage/src/tests/distributor/distributortest.cpp
index 8f308ec7984..268a58a140e 100644
--- a/storage/src/tests/distributor/distributortest.cpp
+++ b/storage/src/tests/distributor/distributortest.cpp
@@ -15,7 +15,6 @@
#include <vespa/storage/distributor/distributor.h>
#include <vespa/storage/distributor/distributormetricsset.h>
#include <vespa/vespalib/text/stringtokenizer.h>
-#include <vespa/vespalib/util/time.h>
#include <thread>
#include <vespa/vespalib/gtest/gtest.h>
#include <gmock/gmock.h>
@@ -446,8 +445,8 @@ TEST_F(DistributorTest, metric_update_hook_updates_pending_maintenance_metrics)
}
// Force trigger update hook
- vespalib::Monitor l;
- distributor_metric_update_hook().updateMetrics(vespalib::MonitorGuard(l));
+ std::mutex l;
+ distributor_metric_update_hook().updateMetrics(std::unique_lock(l));
// Metrics should now be updated to the last complete working state
{
const IdealStateMetricSet& metrics(getIdealStateManager().getMetrics());
@@ -475,8 +474,8 @@ TEST_F(DistributorTest, bucket_db_memory_usage_metrics_only_updated_at_fixed_tim
addNodesToBucketDB(document::BucketId(16, 1), "0=1/1/1/t/a,1=2/2/2");
tickDistributorNTimes(10);
- vespalib::Monitor l;
- distributor_metric_update_hook().updateMetrics(vespalib::MonitorGuard(l));
+ std::mutex l;
+ distributor_metric_update_hook().updateMetrics(std::unique_lock(l));
auto* m = getDistributor().getMetrics().mutable_dbs.memory_usage.getMetric("used_bytes");
ASSERT_TRUE(m != nullptr);
auto last_used = m->getLongValue("last");
@@ -490,7 +489,7 @@ TEST_F(DistributorTest, bucket_db_memory_usage_metrics_only_updated_at_fixed_tim
const auto sample_interval_sec = db_sample_interval_sec(getDistributor());
getClock().setAbsoluteTimeInSeconds(1000 + sample_interval_sec - 1); // Not there yet.
tickDistributorNTimes(50);
- distributor_metric_update_hook().updateMetrics(vespalib::MonitorGuard(l));
+ distributor_metric_update_hook().updateMetrics(std::unique_lock(l));
m = getDistributor().getMetrics().mutable_dbs.memory_usage.getMetric("used_bytes");
auto now_used = m->getLongValue("last");
@@ -498,7 +497,7 @@ TEST_F(DistributorTest, bucket_db_memory_usage_metrics_only_updated_at_fixed_tim
getClock().setAbsoluteTimeInSeconds(1000 + sample_interval_sec + 1);
tickDistributorNTimes(10);
- distributor_metric_update_hook().updateMetrics(vespalib::MonitorGuard(l));
+ distributor_metric_update_hook().updateMetrics(std::unique_lock(l));
m = getDistributor().getMetrics().mutable_dbs.memory_usage.getMetric("used_bytes");
now_used = m->getLongValue("last");
diff --git a/storageframework/src/vespa/storageframework/defaultimplementation/component/componentregisterimpl.cpp b/storageframework/src/vespa/storageframework/defaultimplementation/component/componentregisterimpl.cpp
index e0f441089ff..ec73150b3f2 100644
--- a/storageframework/src/vespa/storageframework/defaultimplementation/component/componentregisterimpl.cpp
+++ b/storageframework/src/vespa/storageframework/defaultimplementation/component/componentregisterimpl.cpp
@@ -137,8 +137,7 @@ namespace {
struct MetricHookWrapper : public metrics::UpdateHook {
MetricUpdateHook& _hook;
- MetricHookWrapper(vespalib::stringref name,
- MetricUpdateHook& hook)
+ MetricHookWrapper(vespalib::stringref name, MetricUpdateHook& hook)
: metrics::UpdateHook(name.data()), // Expected to point to static name
_hook(hook)
{
diff --git a/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h b/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h
index 2fade6f1f56..fb9606f33e0 100644
--- a/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h
+++ b/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h
@@ -7,15 +7,13 @@
*/
#pragma once
-namespace vespalib {
- class MonitorGuard;
-}
+#include <mutex>
namespace storage::framework {
struct MetricUpdateHook {
- using MetricLockGuard = vespalib::MonitorGuard;
- virtual ~MetricUpdateHook() {}
+ using MetricLockGuard = std::unique_lock<std::mutex>;
+ virtual ~MetricUpdateHook() = default;
virtual void updateMetrics(const MetricLockGuard &) = 0;
};