summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--metrics/src/vespa/metrics/CMakeLists.txt1
-rw-r--r--metrics/src/vespa/metrics/metricmanager.cpp4
-rw-r--r--metrics/src/vespa/metrics/metricmanager.h2
-rw-r--r--metrics/src/vespa/metrics/updatehook.cpp18
-rw-r--r--metrics/src/vespa/metrics/updatehook.h19
-rw-r--r--searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.cpp3
-rw-r--r--searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb.cpp29
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb.h43
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.cpp8
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.h2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton.cpp23
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton.h38
-rw-r--r--storage/src/tests/bucketdb/bucketmanagertest.cpp1
-rw-r--r--storage/src/tests/distributor/distributortest.cpp9
-rw-r--r--storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h4
16 files changed, 122 insertions, 86 deletions
diff --git a/metrics/src/vespa/metrics/CMakeLists.txt b/metrics/src/vespa/metrics/CMakeLists.txt
index 6bb39191791..13a3f09449e 100644
--- a/metrics/src/vespa/metrics/CMakeLists.txt
+++ b/metrics/src/vespa/metrics/CMakeLists.txt
@@ -15,6 +15,7 @@ vespa_add_library(metrics
state_api_adapter.cpp
summetric.cpp
textwriter.cpp
+ updatehook.cpp
valuemetric.cpp
valuemetricvalues.cpp
xmlwriter.cpp
diff --git a/metrics/src/vespa/metrics/metricmanager.cpp b/metrics/src/vespa/metrics/metricmanager.cpp
index 51149cf67c3..6447e12a0d2 100644
--- a/metrics/src/vespa/metrics/metricmanager.cpp
+++ b/metrics/src/vespa/metrics/metricmanager.cpp
@@ -20,7 +20,7 @@ LOG_SETUP(".metrics.manager");
namespace metrics {
-typedef MetricsmanagerConfig Config;
+using Config = MetricsmanagerConfig;
MetricManager::ConsumerSpec::ConsumerSpec() = default;
MetricManager::ConsumerSpec::~ConsumerSpec() = default;
@@ -32,7 +32,7 @@ MetricManager::Timer::getTime() const {
void
MetricManager::assertMetricLockLocked(const MetricLockGuard& g) const {
- if ((g.mutex() != &_waiter) || !g.owns_lock()) {
+ if (g.owns(_waiter)) {
throw vespalib::IllegalArgumentException("Given lock does not lock the metric lock.", VESPA_STRLOC);
}
}
diff --git a/metrics/src/vespa/metrics/metricmanager.h b/metrics/src/vespa/metrics/metricmanager.h
index feedf2c1515..f5ad5c5eea3 100644
--- a/metrics/src/vespa/metrics/metricmanager.h
+++ b/metrics/src/vespa/metrics/metricmanager.h
@@ -60,8 +60,6 @@ template class vespalib::hash_set<metrics::Metric::String>;
namespace metrics {
-using MetricLockGuard = UpdateHook::MetricLockGuard;
-
class MetricManager : private document::Runnable
{
public:
diff --git a/metrics/src/vespa/metrics/updatehook.cpp b/metrics/src/vespa/metrics/updatehook.cpp
new file mode 100644
index 00000000000..99051e8df64
--- /dev/null
+++ b/metrics/src/vespa/metrics/updatehook.cpp
@@ -0,0 +1,18 @@
+// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "updatehook.h"
+
+namespace metrics {
+
+MetricLockGuard::MetricLockGuard(std::mutex & mutex)
+ : _guard(mutex)
+{}
+
+bool
+MetricLockGuard::owns(const std::mutex & mutex) const {
+ return (_guard.mutex() == &mutex) && !_guard.owns_lock();
+}
+
+MetricLockGuard::~MetricLockGuard() = default;
+
+}
diff --git a/metrics/src/vespa/metrics/updatehook.h b/metrics/src/vespa/metrics/updatehook.h
index 65f34c65664..0a0f218d61a 100644
--- a/metrics/src/vespa/metrics/updatehook.h
+++ b/metrics/src/vespa/metrics/updatehook.h
@@ -5,6 +5,21 @@
namespace metrics {
+class MetricLockGuard {
+public:
+ MetricLockGuard(std::mutex & mutex);
+ MetricLockGuard(const MetricLockGuard &) = delete;
+ MetricLockGuard & operator =(const MetricLockGuard &) = delete;
+ MetricLockGuard(MetricLockGuard &&) = default;
+ MetricLockGuard & operator =(MetricLockGuard &&) = default;
+ ~MetricLockGuard();
+
+ bool owns(const std::mutex &) const;
+ operator std::unique_lock<std::mutex> & () { return _guard; }
+private:
+ std::unique_lock<std::mutex> _guard;
+};
+
class MetricManager;
class UpdateHook {
@@ -14,9 +29,7 @@ class UpdateHook {
friend class MetricManager;
public:
- using UP = std::unique_ptr<UpdateHook>;
- using MetricLockGuard = std::unique_lock<std::mutex>;
-
+ using MetricLockGuard = metrics::MetricLockGuard;
UpdateHook(const char* name) : _name(name), _nextCall(0), _period(0) {}
virtual ~UpdateHook() = default;
virtual void updateMetrics(const MetricLockGuard & guard) = 0;
diff --git a/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.cpp b/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.cpp
index 3b1a21a7e2f..7b7fcc9e45d 100644
--- a/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.cpp
+++ b/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.cpp
@@ -194,7 +194,8 @@ DocumentDBTaggedMetrics::MatchingMetrics::RankProfileMetrics::DocIdPartition::up
}
void
-DocumentDBTaggedMetrics::MatchingMetrics::RankProfileMetrics::update(const MatchingStats &stats)
+DocumentDBTaggedMetrics::MatchingMetrics::RankProfileMetrics::update(const metrics::MetricLockGuard &,
+ const MatchingStats &stats)
{
docsMatched.inc(stats.docsMatched());
docsRanked.inc(stats.docsRanked());
diff --git a/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.h b/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.h
index 26dd52a8577..133df81a9e6 100644
--- a/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.h
+++ b/searchcore/src/vespa/searchcore/proton/metrics/documentdb_tagged_metrics.h
@@ -9,6 +9,8 @@
#include <vespa/metrics/valuemetric.h>
#include <vespa/searchcore/proton/matching/matching_stats.h>
+namespace metrics { class MetricLockGuard; }
+
namespace proton {
/**
@@ -154,7 +156,7 @@ struct DocumentDBTaggedMetrics : metrics::MetricSet
size_t numDocIdPartitions,
metrics::MetricSet *parent);
~RankProfileMetrics() override;
- void update(const matching::MatchingStats &stats);
+ void update(const metrics::MetricLockGuard & guard, const matching::MatchingStats &stats);
};
using RankProfileMap = std::map<vespalib::string, RankProfileMetrics::UP>;
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp b/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
index c7b5aaafbfc..2e1ed4e449c 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
@@ -35,6 +35,7 @@
#include <vespa/searchlib/common/gatecallback.h>
#include <vespa/vespalib/util/closuretask.h>
#include <vespa/vespalib/util/exceptions.h>
+#include <vespa/metrics/updatehook.h>
#include <vespa/log/log.h>
#include <vespa/searchcorespi/index/warmupconfig.h>
@@ -102,6 +103,18 @@ findDocumentDB(const ProtonConfig::DocumentdbVector & documentDBs, const vespali
return &_G_defaultProtonDocumentDBConfig;
}
+class MetricsUpdateHook : public metrics::UpdateHook {
+ DocumentDB &_db;
+public:
+ MetricsUpdateHook(DocumentDB &s)
+ : metrics::UpdateHook("documentdb-hook"),
+ _db(s)
+ {}
+ void updateMetrics(const MetricLockGuard & guard) override {
+ _db.updateMetrics(guard);
+ }
+};
+
}
template <typename FunctionType>
@@ -147,7 +160,6 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
_configCV(),
_activeConfigSnapshot(),
_activeConfigSnapshotGeneration(0),
- _activeConfigSnapshotSerialNum(0u),
_validateAndSanitizeDocStore(protonCfg.validateAndSanitizeDocstore == vespa::config::search::core::ProtonConfig::ValidateAndSanitizeDocstore::YES),
_initGate(),
_clusterStateHandler(_writeService.master()),
@@ -156,7 +168,8 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
_config_store(std::move(config_store)),
_sessionManager(std::make_shared<matching::SessionManager>(protonCfg.grouping.sessionmanager.maxentries)),
_metricsWireService(metricsWireService),
- _metricsHook(*this, _docTypeName.getName(), protonCfg.numthreadspersearch),
+ _metrics(_docTypeName.getName(), protonCfg.numthreadspersearch),
+ _metricsHook(std::make_unique<MetricsUpdateHook>(*this)),
_feedView(),
_refCount(),
_syncFeedViewEnabled(false),
@@ -222,8 +235,7 @@ void DocumentDB::registerReference()
}
}
-void DocumentDB::setActiveConfig(const DocumentDBConfig::SP &config,
- SerialNum serialNum, int64_t generation) {
+void DocumentDB::setActiveConfig(const DocumentDBConfig::SP &config, int64_t generation) {
lock_guard guard(_configMutex);
registerReference();
_activeConfigSnapshot = config;
@@ -231,7 +243,6 @@ void DocumentDB::setActiveConfig(const DocumentDBConfig::SP &config,
if (_activeConfigSnapshotGeneration < generation) {
_activeConfigSnapshotGeneration = generation;
}
- _activeConfigSnapshotSerialNum = serialNum;
_configCV.notify_all();
}
@@ -297,7 +308,7 @@ DocumentDB::initFinish(DocumentDBConfig::SP configSnapshot)
syncFeedView();
// Check that feed view has been activated.
assert(_feedView.get());
- setActiveConfig(configSnapshot, _initConfigSerialNum, configSnapshot->getGeneration());
+ setActiveConfig(configSnapshot, configSnapshot->getGeneration());
startTransactionLogReplay();
}
@@ -469,7 +480,7 @@ DocumentDB::applyConfig(DocumentDBConfig::SP configSnapshot, SerialNum serialNum
}
_state.clearDelayedConfig();
}
- setActiveConfig(configSnapshot, serialNum, generation);
+ setActiveConfig(configSnapshot, generation);
if (params.shouldMaintenanceControllerChange()) {
forwardMaintenanceConfig();
}
@@ -1042,12 +1053,12 @@ DocumentDB::notifyAllBucketsChanged()
}
void
-DocumentDB::updateMetrics(DocumentDBTaggedMetrics &metrics)
+DocumentDB::updateMetrics(const metrics::MetricLockGuard & guard)
{
if (_state.getState() < DDBState::State::REPLAY_TRANSACTION_LOG) {
return;
}
- _metricsUpdater.updateMetrics(metrics);
+ _metricsUpdater.updateMetrics(guard, _metrics);
}
void
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb.h b/searchcore/src/vespa/searchcore/proton/server/documentdb.h
index a171861b590..9ea88360b07 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb.h
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb.h
@@ -18,7 +18,6 @@
#include "ireplayconfig.h"
#include "maintenancecontroller.h"
#include "threading_service_config.h"
-#include <vespa/metrics/updatehook.h>
#include <vespa/searchcore/proton/attribute/attribute_usage_filter.h>
#include <vespa/searchcore/proton/common/doctypename.h>
#include <vespa/searchcore/proton/common/monitored_refcount.h>
@@ -42,6 +41,10 @@ namespace search {
}
namespace vespa::config::search::core::internal { class InternalProtonType; }
+namespace metrics {
+ class UpdateHook;
+ class MetricLockGuard;
+}
namespace proton {
class AttributeConfigInspector;
@@ -68,26 +71,6 @@ class DocumentDB : public DocumentDBConfigOwner,
public search::transactionlog::SyncProxy
{
private:
- class MetricsUpdateHook : public metrics::UpdateHook {
- DocumentDBTaggedMetrics _metrics;
- DocumentDB &_db;
- public:
- MetricsUpdateHook(DocumentDB &s, const std::string &doc_type, size_t maxNumThreads)
- : metrics::UpdateHook("documentdb-hook"),
- _metrics(doc_type, maxNumThreads),
- _db(s) {}
- void updateMetrics(const MetricLockGuard & ) override { _db.updateMetrics(_metrics); }
- DocumentDBTaggedMetrics &getMetrics() { return _metrics; }
- };
-
- struct DocumentStoreCacheStats {
- search::CacheStats total;
- search::CacheStats readySubDb;
- search::CacheStats notReadySubDb;
- search::CacheStats removedSubDb;
- DocumentStoreCacheStats() : total(), readySubDb(), notReadySubDb(), removedSubDb() {}
- };
-
using InitializeThreads = std::shared_ptr<vespalib::SyncableThreadExecutor>;
using IFlushTargetList = std::vector<std::shared_ptr<searchcorespi::IFlushTarget>>;
using StatusReportUP = std::unique_ptr<StatusReport>;
@@ -114,7 +97,6 @@ private:
mutable std::condition_variable _configCV;
DocumentDBConfig::SP _activeConfigSnapshot;
int64_t _activeConfigSnapshotGeneration;
- SerialNum _activeConfigSnapshotSerialNum;
const bool _validateAndSanitizeDocStore;
vespalib::Gate _initGate;
@@ -126,9 +108,10 @@ private:
index::IndexConfig _indexCfg;
ConfigStore::UP _config_store;
std::shared_ptr<matching::SessionManager> _sessionManager; // TODO: This should not have to be a shared pointer.
- MetricsWireService &_metricsWireService;
- MetricsUpdateHook _metricsHook;
- vespalib::VarHolder<IFeedView::SP> _feedView;
+ MetricsWireService &_metricsWireService;
+ DocumentDBTaggedMetrics _metrics;
+ std::unique_ptr<metrics::UpdateHook> _metricsHook;
+ vespalib::VarHolder<IFeedView::SP> _feedView;
MonitoredRefCount _refCount;
bool _syncFeedViewEnabled;
IDocumentDBOwner &_owner;
@@ -145,7 +128,7 @@ private:
DocumentDBMetricsUpdater _metricsUpdater;
void registerReference();
- void setActiveConfig(const DocumentDBConfig::SP &config, SerialNum serialNum, int64_t generation);
+ void setActiveConfig(const DocumentDBConfig::SP &config, int64_t generation);
DocumentDBConfig::SP getActiveConfig() const;
void internalInit();
void initManagers();
@@ -294,7 +277,9 @@ public:
*
* @return document db metrics
**/
- DocumentDBTaggedMetrics &getMetrics() { return _metricsHook.getMetrics(); }
+ DocumentDBTaggedMetrics &getMetrics() {
+ return _metrics;
+ }
/**
* Obtain the metrics update hook for this document db.
@@ -302,7 +287,7 @@ public:
* @return metrics update hook
**/
metrics::UpdateHook & getMetricsUpdateHook() {
- return _metricsHook;
+ return *_metricsHook;
}
/**
@@ -414,7 +399,7 @@ public:
* the metric manager). Do not call this function in multiple
* threads at once.
**/
- void updateMetrics(DocumentDBTaggedMetrics &metrics);
+ void updateMetrics(const metrics::MetricLockGuard & guard);
/**
* Implement search::transactionlog::SyncProxy API.
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.cpp b/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.cpp
index 3a086046a27..69600cb3b50 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.cpp
@@ -170,12 +170,12 @@ updateAttributeMetrics(DocumentDBTaggedMetrics &metrics, const DocumentSubDBColl
}
void
-updateMatchingMetrics(DocumentDBTaggedMetrics &metrics, const IDocumentSubDB &ready)
+updateMatchingMetrics(const metrics::MetricLockGuard & guard, DocumentDBTaggedMetrics &metrics, const IDocumentSubDB &ready)
{
MatchingStats totalStats;
for (const auto &rankProfile : metrics.matching.rank_profiles) {
MatchingStats matchingStats = ready.getMatcherStats(rankProfile.first);
- rankProfile.second->update(matchingStats);
+ rankProfile.second->update(guard, matchingStats);
totalStats.add(matchingStats);
}
@@ -284,13 +284,13 @@ updateLidSpaceMetrics(MetricSetType &metrics, const search::IDocumentMetaStore &
}
void
-DocumentDBMetricsUpdater::updateMetrics(DocumentDBTaggedMetrics &metrics)
+DocumentDBMetricsUpdater::updateMetrics(const metrics::MetricLockGuard & guard, DocumentDBTaggedMetrics &metrics)
{
TotalStats totalStats;
ExecutorThreadingServiceStats threadingServiceStats = _writeService.getStats();
updateIndexMetrics(metrics, _subDBs.getReadySubDB()->getSearchableStats(), totalStats);
updateAttributeMetrics(metrics, _subDBs, totalStats);
- updateMatchingMetrics(metrics, *_subDBs.getReadySubDB());
+ updateMatchingMetrics(guard, metrics, *_subDBs.getReadySubDB());
updateSessionCacheMetrics(metrics, _sessionManager);
updateDocumentsMetrics(metrics, _subDBs);
updateDocumentStoreMetrics(metrics, _subDBs, _lastDocStoreCacheStats, totalStats);
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.h b/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.h
index dbf4c45007f..425be71cfe5 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.h
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb_metrics_updater.h
@@ -49,7 +49,7 @@ public:
const DDBState &state);
~DocumentDBMetricsUpdater();
- void updateMetrics(DocumentDBTaggedMetrics &metrics);
+ void updateMetrics(const metrics::MetricLockGuard & guard, DocumentDBTaggedMetrics &metrics);
};
diff --git a/searchcore/src/vespa/searchcore/proton/server/proton.cpp b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
index dddca6a9ddd..864e2480ca8 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
@@ -34,6 +34,7 @@
#include <vespa/vespalib/util/host_name.h>
#include <vespa/vespalib/util/lambdatask.h>
#include <vespa/vespalib/util/random.h>
+#include <vespa/metrics/updatehook.h>
#include <vespa/searchlib/aggregation/forcelink.hpp>
#include <vespa/searchlib/expression/forcelink.hpp>
@@ -112,6 +113,18 @@ derive_shared_threads(const ProtonConfig &proton,
return std::max(scaledCores, proton.documentdb.size() + proton.flush.maxconcurrent + 1);
}
+struct MetricsUpdateHook : metrics::UpdateHook
+{
+ Proton &self;
+ MetricsUpdateHook(Proton &s)
+ : metrics::UpdateHook("proton-hook"),
+ self(s)
+ {}
+ void updateMetrics(const MetricLockGuard &guard) override {
+ self.updateMetrics(guard);
+ }
+};
+
const vespalib::string CUSTOM_COMPONENT_API_PATH = "/state/v1/custom/component";
VESPA_THREAD_STACK_TAG(proton_shared_executor)
@@ -186,7 +199,7 @@ Proton::Proton(const config::ConfigUri & configUri,
ComponentConfigProducer(),
_configUri(configUri),
_mutex(),
- _metricsHook(*this),
+ _metricsHook(std::make_unique<MetricsUpdateHook>(*this)),
_metricsEngine(std::make_unique<MetricsEngine>()),
_fileHeaderContext(*this, progName),
_tls(),
@@ -219,7 +232,6 @@ Proton::Proton(const config::ConfigUri & configUri,
_threadPool(128 * 1024),
_distributionKey(-1),
_isInitializing(true),
- _isReplayDone(false),
_abortInit(false),
_initStarted(false),
_initComplete(false),
@@ -260,7 +272,7 @@ Proton::init(const BootstrapConfig::SP & configSnapshot)
diskMemUsageSamplerConfig(protonConfig, hwInfo));
_tls = std::make_unique<TLS>(_configUri.createWithNewId(protonConfig.tlsconfigid), _fileHeaderContext);
- _metricsEngine->addMetricsHook(_metricsHook);
+ _metricsEngine->addMetricsHook(*_metricsHook);
_fileHeaderContext.setClusterName(protonConfig.clustername, protonConfig.basedir);
_matchEngine = std::make_unique<MatchEngine>(protonConfig.numsearcherthreads,
protonConfig.numthreadspersearch,
@@ -331,7 +343,6 @@ Proton::init(const BootstrapConfig::SP & configSnapshot)
_executor.sync();
waitForOnlineState();
- _isReplayDone = true;
_rpcHooks->set_online();
_flushEngine->start();
@@ -470,7 +481,7 @@ Proton::shutdown_config_fetching_and_state_exposing_components_once() noexcept
_customComponentBindToken.reset();
_stateServer.reset();
if (_metricsEngine) {
- _metricsEngine->removeMetricsHook(_metricsHook);
+ _metricsEngine->removeMetricsHook(*_metricsHook);
_metricsEngine->stop();
}
_has_shut_down_config_and_state_components = true;
@@ -714,7 +725,7 @@ updateExecutorMetrics(ExecutorMetrics &metrics,
}
void
-Proton::updateMetrics(const metrics::UpdateHook::MetricLockGuard &)
+Proton::updateMetrics(const metrics::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 f88cd7bf0cd..48df9dc5438 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton.h
+++ b/searchcore/src/vespa/searchcore/proton/server/proton.h
@@ -31,6 +31,7 @@
namespace vespalib { class StateServer; }
namespace search::transactionlog { class TransLogServerApp; }
+namespace metrics { class MetricLockGuard; }
namespace proton {
class DiskMemUsageSampler;
@@ -60,16 +61,6 @@ private:
using InitializeThreads = std::shared_ptr<vespalib::SyncableThreadExecutor>;
using BucketSpace = document::BucketSpace;
- struct MetricsUpdateHook : metrics::UpdateHook
- {
- Proton &self;
- MetricsUpdateHook(Proton &s)
- : metrics::UpdateHook("proton-hook"),
- self(s) {}
- void updateMetrics(const MetricLockGuard &guard) override { self.updateMetrics(guard); }
- };
- friend struct MetricsUpdateHook;
-
class ProtonFileHeaderContext : public search::common::FileHeaderContext
{
vespalib::string _hostName;
@@ -85,16 +76,16 @@ private:
void setClusterName(const vespalib::string &clusterName, const vespalib::string &baseDir);
};
- const config::ConfigUri _configUri;
- mutable std::shared_mutex _mutex;
- MetricsUpdateHook _metricsHook;
- std::unique_ptr<MetricsEngine> _metricsEngine;
- ProtonFileHeaderContext _fileHeaderContext;
- std::unique_ptr<TLS> _tls;
+ const config::ConfigUri _configUri;
+ mutable std::shared_mutex _mutex;
+ std::unique_ptr<metrics::UpdateHook> _metricsHook;
+ std::unique_ptr<MetricsEngine> _metricsEngine;
+ ProtonFileHeaderContext _fileHeaderContext;
+ std::unique_ptr<TLS> _tls;
std::unique_ptr<DiskMemUsageSampler> _diskMemUsageSampler;
PersistenceEngine::UP _persistenceEngine;
DocumentDBMap _documentDBMap;
- std::unique_ptr<MatchEngine> _matchEngine;
+ std::unique_ptr<MatchEngine> _matchEngine;
std::unique_ptr<SummaryEngine> _summaryEngine;
std::unique_ptr<DocsumBySlime> _docsumBySlime;
MemoryFlushConfigUpdater::UP _memoryFlushConfigUpdater;
@@ -138,12 +129,6 @@ private:
void applyConfig(const BootstrapConfig::SP & configSnapshot) override;
MonitorReply::UP ping(MonitorRequest::UP request, MonitorClient &client) override;
- /**
- * Called by the metrics update hook (typically in the context of
- * the metric manager). Do not call this function in multiple
- * threads at once.
- **/
- void updateMetrics(const metrics::UpdateHook::MetricLockGuard &guard);
void waitForInitDone();
void waitForOnlineState();
uint32_t getDistributionKey() const override { return _distributionKey; }
@@ -161,6 +146,13 @@ public:
~Proton() override;
/**
+ * Called by the metrics update hook (typically in the context of
+ * the metric manager). Do not call this function in multiple
+ * threads at once.
+ **/
+ void updateMetrics(const metrics::MetricLockGuard &guard);
+
+ /**
* This method must be called after the constructor and before the destructor.
* If not I will force a 'core' upon you.
* All relevant initialization is conducted here.
diff --git a/storage/src/tests/bucketdb/bucketmanagertest.cpp b/storage/src/tests/bucketdb/bucketmanagertest.cpp
index 8a508a2169d..89d82df62bd 100644
--- a/storage/src/tests/bucketdb/bucketmanagertest.cpp
+++ b/storage/src/tests/bucketdb/bucketmanagertest.cpp
@@ -14,6 +14,7 @@
#include <vespa/storageapi/message/persistence.h>
#include <vespa/storageapi/message/state.h>
#include <vespa/storageapi/message/bucketsplitting.h>
+#include <vespa/metrics/updatehook.h>
#include <tests/common/teststorageapp.h>
#include <tests/common/dummystoragelink.h>
#include <tests/common/testhelper.h>
diff --git a/storage/src/tests/distributor/distributortest.cpp b/storage/src/tests/distributor/distributortest.cpp
index 72115da8de6..928373d516b 100644
--- a/storage/src/tests/distributor/distributortest.cpp
+++ b/storage/src/tests/distributor/distributortest.cpp
@@ -16,6 +16,7 @@
#include <vespa/storage/distributor/distributor_bucket_space.h>
#include <vespa/storage/distributor/distributormetricsset.h>
#include <vespa/vespalib/text/stringtokenizer.h>
+#include <vespa/metrics/updatehook.h>
#include <thread>
#include <vespa/vespalib/gtest/gtest.h>
#include <gmock/gmock.h>
@@ -453,7 +454,7 @@ TEST_F(DistributorTest, metric_update_hook_updates_pending_maintenance_metrics)
// Force trigger update hook
std::mutex l;
- distributor_metric_update_hook().updateMetrics(std::unique_lock(l));
+ distributor_metric_update_hook().updateMetrics(metrics::MetricLockGuard(l));
// Metrics should now be updated to the last complete working state
{
const IdealStateMetricSet& metrics(getIdealStateManager().getMetrics());
@@ -482,7 +483,7 @@ TEST_F(DistributorTest, bucket_db_memory_usage_metrics_only_updated_at_fixed_tim
tickDistributorNTimes(10);
std::mutex l;
- distributor_metric_update_hook().updateMetrics(std::unique_lock(l));
+ distributor_metric_update_hook().updateMetrics(metrics::MetricLockGuard(l));
auto* m = getDistributor().getMetrics().mutable_dbs.memory_usage.getMetric("used_bytes");
ASSERT_TRUE(m != nullptr);
auto last_used = m->getLongValue("last");
@@ -496,7 +497,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(std::unique_lock(l));
+ distributor_metric_update_hook().updateMetrics(metrics::MetricLockGuard(l));
m = getDistributor().getMetrics().mutable_dbs.memory_usage.getMetric("used_bytes");
auto now_used = m->getLongValue("last");
@@ -504,7 +505,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(std::unique_lock(l));
+ distributor_metric_update_hook().updateMetrics(metrics::MetricLockGuard(l));
m = getDistributor().getMetrics().mutable_dbs.memory_usage.getMetric("used_bytes");
now_used = m->getLongValue("last");
diff --git a/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h b/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h
index fb9606f33e0..6292fe9a598 100644
--- a/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h
+++ b/storageframework/src/vespa/storageframework/generic/metric/metricupdatehook.h
@@ -9,10 +9,12 @@
#include <mutex>
+namespace metrics { class MetricLockGuard; }
+
namespace storage::framework {
struct MetricUpdateHook {
- using MetricLockGuard = std::unique_lock<std::mutex>;
+ using MetricLockGuard = metrics::MetricLockGuard;
virtual ~MetricUpdateHook() = default;
virtual void updateMetrics(const MetricLockGuard &) = 0;