aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/activediskindexes.cpp13
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/activediskindexes.h18
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp5
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h28
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.h4
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/client_session.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/domain.cpp60
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/domain.h14
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/domainpart.cpp23
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/domainpart.h10
-rw-r--r--searchlib/src/vespa/searchlib/transactionlog/translogclient.h4
-rw-r--r--staging_vespalib/CMakeLists.txt1
-rw-r--r--staging_vespalib/src/tests/librarypool/.gitignore1
-rw-r--r--staging_vespalib/src/tests/librarypool/CMakeLists.txt8
-rw-r--r--staging_vespalib/src/tests/librarypool/librarypool_test.cpp38
-rw-r--r--staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.cpp13
-rw-r--r--staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.h5
-rw-r--r--staging_vespalib/src/vespa/vespalib/net/simple_health_producer.cpp11
-rw-r--r--staging_vespalib/src/vespa/vespalib/net/simple_health_producer.h6
-rw-r--r--staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.cpp12
-rw-r--r--staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.h10
-rw-r--r--staging_vespalib/src/vespa/vespalib/util/CMakeLists.txt1
-rw-r--r--staging_vespalib/src/vespa/vespalib/util/librarypool.cpp58
-rw-r--r--staging_vespalib/src/vespa/vespalib/util/librarypool.h39
-rw-r--r--staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.cpp8
-rw-r--r--staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.h5
-rw-r--r--vbench/src/vbench/core/dispatcher.h3
-rw-r--r--vbench/src/vbench/core/dispatcher.hpp10
29 files changed, 133 insertions, 283 deletions
diff --git a/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.cpp b/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.cpp
index 7a590b66a8a..bec6f02324a 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.cpp
@@ -3,26 +3,27 @@
#include "activediskindexes.h"
#include <cassert>
-using std::set;
using vespalib::string;
-using vespalib::LockGuard;
namespace searchcorespi::index {
+ActiveDiskIndexes::ActiveDiskIndexes() = default;
+ActiveDiskIndexes::~ActiveDiskIndexes() = default;
+
void ActiveDiskIndexes::setActive(const string &index) {
- LockGuard lock(_lock);
+ std::lock_guard lock(_lock);
_active.insert(index);
}
void ActiveDiskIndexes::notActive(const string & index) {
- LockGuard lock(_lock);
- set<string>::iterator it = _active.find(index);
+ std::lock_guard lock(_lock);
+ auto it = _active.find(index);
assert(it != _active.end());
_active.erase(it);
}
bool ActiveDiskIndexes::isActive(const string &index) const {
- LockGuard lock(_lock);
+ std::lock_guard lock(_lock);
return _active.find(index) != _active.end();
}
diff --git a/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.h b/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.h
index dff25906559..c1c0e4e4d88 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/activediskindexes.h
@@ -3,11 +3,10 @@
#pragma once
#include <vespa/vespalib/stllike/string.h>
-#include <vespa/vespalib/util/sync.h>
#include <set>
+#include <mutex>
-namespace searchcorespi {
-namespace index {
+namespace searchcorespi::index {
/**
* Class used to keep track of the set of active disk indexes in an index maintainer.
@@ -15,16 +14,17 @@ namespace index {
*/
class ActiveDiskIndexes {
std::multiset<vespalib::string> _active;
- vespalib::Lock _lock;
+ mutable std::mutex _lock;
public:
- typedef std::shared_ptr<ActiveDiskIndexes> SP;
-
+ using SP = std::shared_ptr<ActiveDiskIndexes>;
+ ActiveDiskIndexes();
+ ~ActiveDiskIndexes();
+ ActiveDiskIndexes(const ActiveDiskIndexes &) = delete;
+ ActiveDiskIndexes & operator = (const ActiveDiskIndexes &) = delete;
void setActive(const vespalib::string & index);
void notActive(const vespalib::string & index);
bool isActive(const vespalib::string & index) const;
};
-} // namespace index
-} // namespace searchcorespi
-
+}
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
index 32dc2531061..c75a74ff141 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
@@ -41,7 +41,6 @@ using vespalib::makeTask;
using vespalib::string;
using vespalib::Closure0;
using vespalib::Executor;
-using vespalib::LockGuard;
using vespalib::Runnable;
namespace searchcorespi::index {
@@ -349,10 +348,12 @@ IndexMaintainer::loadDiskIndexes(const FusionSpec &spec, ISearchableIndexCollect
namespace {
+ using LockGuard = std::lock_guard<std::mutex>;
+
ISearchableIndexCollection::SP
getLeaf(const LockGuard &newSearchLock, const ISearchableIndexCollection::SP & is, bool warn=false)
{
- if (dynamic_cast<const WarmupIndexCollection *>(is.get()) != NULL) {
+ if (dynamic_cast<const WarmupIndexCollection *>(is.get()) != nullptr) {
if (warn) {
LOG(info, "Already warming up an index '%s'. Start using it immediately."
" This is an indication that you have configured your warmup interval too long.",
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h
index 671f87ff35b..5ff805b53f7 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h
@@ -18,9 +18,6 @@
#include <vespa/searchcorespi/flush/flushstats.h>
#include <vespa/searchlib/attribute/fixedsourceselector.h>
#include <vespa/searchlib/common/serialnum.h>
-#include <vespa/vespalib/util/sync.h>
-#include <memory>
-#include <vector>
namespace document { class Document; }
@@ -74,6 +71,7 @@ class IndexMaintainer : public IIndexManager,
using FlushIds = std::vector<uint32_t>;
using FrozenMemoryIndexRefs = std::vector<FrozenMemoryIndexRef>;
using ISourceSelector = search::queryeval::ISourceSelector;
+ using LockGuard = std::lock_guard<std::mutex>;
const vespalib::string _base_dir;
const WarmupConfig _warmupConfig;
@@ -129,17 +127,17 @@ class IndexMaintainer : public IIndexManager,
* and pruning of removed fields, since this will trigger more retries for
* some of the operations.
*/
- vespalib::Lock _state_lock; // Outer lock (SL)
- vespalib::Lock _index_update_lock; // Inner lock (IUL)
- vespalib::Lock _new_search_lock; // Inner lock (NSL)
- vespalib::Lock _remove_lock; // Lock for removing indexes.
+ std::mutex _state_lock; // Outer lock (SL)
+ mutable std::mutex _index_update_lock; // Inner lock (IUL)
+ mutable std::mutex _new_search_lock; // Inner lock (NSL)
+ std::mutex _remove_lock; // Lock for removing indexes.
// Protected by SL + IUL
- FusionSpec _fusion_spec; // Protected by FL
- vespalib::Lock _fusion_lock; // Fusion spec lock (FL)
+ FusionSpec _fusion_spec; // Protected by FL
+ mutable std::mutex _fusion_lock; // Fusion spec lock (FL)
uint32_t _maxFlushed;
uint32_t _maxFrozen;
ChangeGens _changeGens; // Protected by SL + IUL
- vespalib::Lock _schemaUpdateLock; // Serialize rewrite of schema
+ std::mutex _schemaUpdateLock; // Serialize rewrite of schema
const search::TuneFileAttributes _tuneFileAttributes;
const IndexMaintainerContext _ctx;
IIndexMaintainerOperations &_operations;
@@ -179,8 +177,8 @@ class IndexMaintainer : public IIndexManager,
ISearchableIndexCollection::UP loadDiskIndexes(const FusionSpec &spec, ISearchableIndexCollection::UP sourceList);
void replaceSource(uint32_t sourceId, const IndexSearchable::SP &source);
void appendSource(uint32_t sourceId, const IndexSearchable::SP &source);
- void swapInNewIndex(vespalib::LockGuard & guard, ISearchableIndexCollection::SP indexes, IndexSearchable & source);
- ISearchableIndexCollection::UP createNewSourceCollection(const vespalib::LockGuard &newSearchLock);
+ void swapInNewIndex(LockGuard & guard, ISearchableIndexCollection::SP indexes, IndexSearchable & source);
+ ISearchableIndexCollection::UP createNewSourceCollection(const LockGuard &newSearchLock);
struct FlushArgs {
IMemoryIndex::SP old_index; // Last memory index
@@ -353,17 +351,17 @@ public:
}
IIndexCollection::SP getSourceCollection() const {
- vespalib::LockGuard lock(_new_search_lock);
+ LockGuard lock(_new_search_lock);
return _source_list;
}
searchcorespi::IndexSearchable::SP getSearchable() const override {
- vespalib::LockGuard lock(_new_search_lock);
+ LockGuard lock(_new_search_lock);
return _source_list;
}
search::SearchableStats getSearchableStats() const override {
- vespalib::LockGuard lock(_new_search_lock);
+ LockGuard lock(_new_search_lock);
return _source_list->getSearchableStats();
}
diff --git a/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.cpp b/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.cpp
index dc84eb45b8a..28c229596b0 100644
--- a/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.cpp
+++ b/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.cpp
@@ -12,7 +12,7 @@ SharedWeakAndPriorityQueue::SharedWeakAndPriorityQueue(uint32_t scoresToTrack) :
_bestScores.reserve(scoresToTrack);
}
-SharedWeakAndPriorityQueue::~SharedWeakAndPriorityQueue() { }
+SharedWeakAndPriorityQueue::~SharedWeakAndPriorityQueue() = default;
void
SharedWeakAndPriorityQueue::adjust(score_t *begin, score_t *end)
@@ -20,7 +20,7 @@ SharedWeakAndPriorityQueue::adjust(score_t *begin, score_t *end)
if (getScoresToTrack() == 0) {
return;
}
- vespalib::LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
for (score_t *itr = begin; itr != end; ++itr) {
score_t score = *itr;
if (!is_full()) {
diff --git a/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.h b/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.h
index d6105d7e6f2..28c317de3ee 100644
--- a/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.h
+++ b/searchlib/src/vespa/searchlib/queryeval/wand/weak_and_heap.h
@@ -3,7 +3,7 @@
#include "wand_parts.h"
#include <vespa/vespalib/util/priority_queue.h>
-#include <vespa/vespalib/util/sync.h>
+#include <mutex>
namespace search::queryeval {
@@ -51,7 +51,7 @@ class SharedWeakAndPriorityQueue : public WeakAndHeap
private:
typedef vespalib::PriorityQueue<score_t> Scores;
Scores _bestScores;
- vespalib::Lock _lock;
+ std::mutex _lock;
bool is_full() const { return (_bestScores.size() >= getScoresToTrack()); }
diff --git a/searchlib/src/vespa/searchlib/transactionlog/client_session.cpp b/searchlib/src/vespa/searchlib/transactionlog/client_session.cpp
index 8678d88b43c..40db92cbe78 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/client_session.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/client_session.cpp
@@ -123,7 +123,7 @@ void
Session::clear()
{
if (_sessionId > 0) {
- LockGuard guard(_tlc._lock);
+ std::lock_guard guard(_tlc._lock);
_tlc._sessions.erase(SessionKey(_domain, _sessionId));
}
_sessionId = 0;
@@ -145,7 +145,7 @@ Session::init(FRT_RPCRequest *req)
_sessionId = retval;
SessionKey key(_domain, _sessionId);
{
- LockGuard guard(_tlc._lock);
+ std::lock_guard guard(_tlc._lock);
_tlc._sessions[key] = this;
}
retval = run();
diff --git a/searchlib/src/vespa/searchlib/transactionlog/domain.cpp b/searchlib/src/vespa/searchlib/transactionlog/domain.cpp
index 126a7afed4d..454293dfc84 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/domain.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/domain.cpp
@@ -19,7 +19,6 @@ LOG_SETUP(".transactionlog.domain");
using vespalib::string;
using vespalib::make_string_short::fmt;
-using vespalib::LockGuard;
using vespalib::makeTask;
using vespalib::makeClosure;
using vespalib::makeLambdaTask;
@@ -104,7 +103,7 @@ Domain::addPart(SerialNum partId, bool isLastPart) {
dp->erase(dp->range().to() + 1);
} else {
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_parts[partId] = dp;
}
if (! isLastPart) {
@@ -123,7 +122,7 @@ Domain::~Domain() {
DomainInfo
Domain::getDomainInfo() const
{
- LockGuard guard(_lock);
+ UniqueLock guard(_lock);
DomainInfo info(SerialNumRange(begin(guard), end(guard)), size(guard), byteSize(guard), _maxSessionRunTime);
for (const auto &entry: _parts) {
const DomainPart &part = *entry.second;
@@ -135,14 +134,18 @@ Domain::getDomainInfo() const
SerialNum
Domain::begin() const
{
- return begin(LockGuard(_lock));
+ return begin(UniqueLock(_lock));
}
+void
+Domain::verifyLock(const UniqueLock & guard) const {
+ assert(guard.mutex() == &_lock);
+ assert(guard.owns_lock());
+}
SerialNum
-Domain::begin(const LockGuard & guard) const
+Domain::begin(const UniqueLock & guard) const
{
- (void) guard;
- assert(guard.locks(_lock));
+ verifyLock(guard);
SerialNum s(0);
if ( ! _parts.empty() ) {
s = _parts.cbegin()->second->range().from();
@@ -153,14 +156,13 @@ Domain::begin(const LockGuard & guard) const
SerialNum
Domain::end() const
{
- return end(LockGuard(_lock));
+ return end(UniqueLock(_lock));
}
SerialNum
-Domain::end(const LockGuard & guard) const
+Domain::end(const UniqueLock & guard) const
{
- (void) guard;
- assert(guard.locks(_lock));
+ verifyLock(guard);
SerialNum s(0);
if ( ! _parts.empty() ) {
s = _parts.crbegin()->second->range().to();
@@ -171,14 +173,13 @@ Domain::end(const LockGuard & guard) const
size_t
Domain::byteSize() const
{
- return byteSize(LockGuard(_lock));
+ return byteSize(UniqueLock(_lock));
}
size_t
-Domain::byteSize(const LockGuard & guard) const
+Domain::byteSize(const UniqueLock & guard) const
{
- (void) guard;
- assert(guard.locks(_lock));
+ verifyLock(guard);
size_t size = 0;
for (const auto &entry : _parts) {
const DomainPart &part = *entry.second;
@@ -191,7 +192,7 @@ SerialNum
Domain::getSynced() const
{
SerialNum s(0);
- LockGuard guard(_lock);
+ UniqueLock guard(_lock);
if (_parts.empty()) {
return s;
}
@@ -228,7 +229,7 @@ Domain::triggerSyncNow()
DomainPart::SP
Domain::findPart(SerialNum s)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
DomainPartList::iterator it(_parts.upper_bound(s));
if (!_parts.empty() && it != _parts.begin()) {
DomainPartList::iterator prev(it);
@@ -246,14 +247,13 @@ Domain::findPart(SerialNum s)
uint64_t
Domain::size() const
{
- return size(LockGuard(_lock));
+ return size(UniqueLock(_lock));
}
uint64_t
-Domain::size(const LockGuard & guard) const
+Domain::size(const UniqueLock & guard) const
{
- (void) guard;
- assert(guard.locks(_lock));
+ verifyLock(guard);
uint64_t sz(0);
for (const auto & part : _parts) {
sz += part.second->size();
@@ -265,7 +265,7 @@ SerialNum
Domain::findOldestActiveVisit() const
{
SerialNum oldestActive(std::numeric_limits<SerialNum>::max());
- LockGuard guard(_sessionLock);
+ std::lock_guard guard(_sessionLock);
for (const auto & pair : _sessions) {
Session * session(pair.second.get());
if (!session->inSync()) {
@@ -281,7 +281,7 @@ Domain::cleanSessions()
if ( _sessions.empty()) {
return;
}
- LockGuard guard(_sessionLock);
+ std::lock_guard guard(_sessionLock);
for (SessionList::iterator it(_sessions.begin()), mt(_sessions.end()); it != mt; ) {
Session * session(it->second.get());
if (session->inSync()) {
@@ -318,7 +318,7 @@ Domain::optionallyRotateFile(SerialNum serialNum) {
dp = std::make_shared<DomainPart>(_name, dir(), serialNum, _config.getEncoding(),
_config.getCompressionlevel(), _fileHeaderContext, false);
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_parts[serialNum] = dp;
assert(_parts.rbegin()->first == serialNum);
}
@@ -409,7 +409,7 @@ Domain::erase(SerialNum to)
for (DomainPartList::iterator it(_parts.begin()); (_parts.size() > 1) && (it->second.get()->range().to() < to); it = _parts.begin()) {
DomainPart::SP dp(it->second);
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_parts.erase(it);
}
retval = retval && dp->erase(to);
@@ -429,7 +429,7 @@ Domain::visit(const Domain::SP & domain, SerialNum from, SerialNum to, std::uniq
SerialNumRange range(from, to);
auto session = std::make_shared<Session>(_sessionId++, range, domain, std::move(dest));
int id = session->id();
- LockGuard guard(_sessionLock);
+ std::lock_guard guard(_sessionLock);
_sessions[id] = std::move(session);
return id;
}
@@ -438,7 +438,7 @@ int
Domain::startSession(int sessionId)
{
int retval(-1);
- LockGuard guard(_sessionLock);
+ std::lock_guard guard(_sessionLock);
SessionList::iterator found = _sessions.find(sessionId);
if (found != _sessions.end()) {
found->second->setStartTime(std::chrono::steady_clock::now());
@@ -458,7 +458,7 @@ Domain::closeSession(int sessionId)
int retval(-1);
DurationSeconds sessionRunTime(0);
{
- LockGuard guard(_sessionLock);
+ std::lock_guard guard(_sessionLock);
SessionList::iterator found = _sessions.find(sessionId);
if (found != _sessions.end()) {
sessionRunTime = (std::chrono::steady_clock::now() - found->second->getStartTime());
@@ -467,7 +467,7 @@ Domain::closeSession(int sessionId)
}
while (retval == 1) {
std::this_thread::sleep_for(10ms);
- LockGuard guard(_sessionLock);
+ std::lock_guard guard(_sessionLock);
SessionList::iterator found = _sessions.find(sessionId);
if (found != _sessions.end()) {
if ( ! found->second->isVisitRunning()) {
@@ -479,7 +479,7 @@ Domain::closeSession(int sessionId)
}
}
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
if (sessionRunTime > _maxSessionRunTime) {
_maxSessionRunTime = sessionRunTime;
}
diff --git a/searchlib/src/vespa/searchlib/transactionlog/domain.h b/searchlib/src/vespa/searchlib/transactionlog/domain.h
index e41ad930840..a883156f32c 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/domain.h
+++ b/searchlib/src/vespa/searchlib/transactionlog/domain.h
@@ -56,16 +56,18 @@ public:
uint64_t size() const;
Domain & setConfig(const DomainConfig & cfg);
private:
+ using UniqueLock = std::unique_lock<std::mutex>;
+ void verifyLock(const UniqueLock & guard) const;
void commitIfFull(const vespalib::MonitorGuard & guard);
void commitAndTransferResponses(const vespalib::MonitorGuard & guard);
std::unique_ptr<CommitChunk> grabCurrentChunk(const vespalib::MonitorGuard & guard);
void commitChunk(std::unique_ptr<CommitChunk> chunk, const vespalib::MonitorGuard & chunkOrderGuard);
void doCommit(std::unique_ptr<CommitChunk> chunk);
- SerialNum begin(const vespalib::LockGuard & guard) const;
- SerialNum end(const vespalib::LockGuard & guard) const;
- size_t byteSize(const vespalib::LockGuard & guard) const;
- uint64_t size(const vespalib::LockGuard & guard) const;
+ SerialNum begin(const UniqueLock & guard) const;
+ SerialNum end(const UniqueLock & guard) const;
+ size_t byteSize(const UniqueLock & guard) const;
+ uint64_t size(const UniqueLock & guard) const;
void cleanSessions();
vespalib::string dir() const { return getDir(_baseDir, _name); }
void addPart(SerialNum partId, bool isLastPart);
@@ -89,9 +91,9 @@ private:
bool _pendingSync;
vespalib::string _name;
DomainPartList _parts;
- vespalib::Lock _lock;
+ mutable std::mutex _lock;
vespalib::Monitor _currentChunkMonitor;
- vespalib::Lock _sessionLock;
+ mutable std::mutex _sessionLock;
SessionList _sessions;
DurationSeconds _maxSessionRunTime;
vespalib::string _baseDir;
diff --git a/searchlib/src/vespa/searchlib/transactionlog/domainpart.cpp b/searchlib/src/vespa/searchlib/transactionlog/domainpart.cpp
index 97ab61d3045..830384ee538 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/domainpart.cpp
+++ b/searchlib/src/vespa/searchlib/transactionlog/domainpart.cpp
@@ -15,7 +15,6 @@ using vespalib::FileHeader;
using vespalib::string;
using vespalib::getLastErrorString;
using vespalib::IllegalHeaderException;
-using vespalib::LockGuard;
using vespalib::nbostream;
using vespalib::nbostream_longlivedbuf;
using vespalib::alloc::Alloc;
@@ -239,7 +238,7 @@ DomainPart::buildPacketMapping(bool allowTruncate)
_range.to(packet.range().to());
_packets.insert(std::make_pair(firstSerial, std::move(packet)));
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_skipList.push_back(SkipInfo(firstSerial, firstPos));
}
} else {
@@ -330,7 +329,7 @@ DomainPart::close()
{
bool retval(false);
{
- LockGuard guard(_fileLock);
+ std::lock_guard guard(_fileLock);
/*
* Sync old domainpart before starting writing new, to avoid
* hole. XXX: Feed latency spike due to lack of delayed open
@@ -339,7 +338,7 @@ DomainPart::close()
handleSync(*_transLog);
_transLog->dropFromCache();
retval = _transLog->Close();
- LockGuard wguard(_writeLock);
+ std::lock_guard wguard(_writeLock);
_syncedSerial = _writtenSerial;
}
if ( ! retval ) {
@@ -347,7 +346,7 @@ DomainPart::close()
_transLog->GetFileName(), _transLog->GetSize()));
}
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_packets.clear();
}
return retval;
@@ -364,7 +363,7 @@ DomainPart::openAndFind(FastOS_FileInterface &file, const SerialNum &from)
bool retval(file.OpenReadOnly(_transLog->GetFileName()));
if (retval) {
int64_t pos(_headerLen);
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
for(SkipList::const_iterator it(_skipList.begin()), mt(_skipList.end());
(it < mt) && (it->id() <= from);
it++)
@@ -422,7 +421,7 @@ DomainPart::commit(SerialNum firstSerial, const Packet &packet)
}
bool merged(false);
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
if ( ! _packets.empty() ) {
Packet & lastPacket = _packets.rbegin()->second;
if (lastPacket.sizeBytes() < 0xf000) {
@@ -441,12 +440,12 @@ DomainPart::sync()
{
SerialNum syncSerial(0);
{
- LockGuard guard(_writeLock);
+ std::lock_guard guard(_writeLock);
syncSerial = _writtenSerial;
}
- LockGuard guard(_fileLock);
+ std::lock_guard guard(_fileLock);
handleSync(*_transLog);
- LockGuard wguard(_writeLock);
+ std::lock_guard wguard(_writeLock);
if (_syncedSerial < syncSerial) {
_syncedSerial = syncSerial;
}
@@ -456,7 +455,7 @@ bool
DomainPart::visit(SerialNumRange &r, Packet &packet)
{
bool retval(false);
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
LOG(spam, "Visit r(%" PRIu64 ", %" PRIu64 "] Checking %" PRIu64 " packets",
r.from(), r.to(), uint64_t(_packets.size()));
if ( ! isClosed() ) {
@@ -550,7 +549,7 @@ DomainPart::write(FastOS_FileInterface &file, const IChunk & chunk)
os << realEncoding.getRaw(); //Patching real encoding
os << uint32_t(end - (begin + sizeof(uint32_t) + sizeof(uint8_t))); // Patching actual size.
os.wp(end);
- LockGuard guard(_writeLock);
+ std::lock_guard guard(_writeLock);
if ( ! file.CheckedWrite(os.data(), os.size()) ) {
throw runtime_error(handleWriteError("Failed writing the entry.", file, byteSize(), chunk.range(), os.size()));
}
diff --git a/searchlib/src/vespa/searchlib/transactionlog/domainpart.h b/searchlib/src/vespa/searchlib/transactionlog/domainpart.h
index 31d6938b654..a956932be19 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/domainpart.h
+++ b/searchlib/src/vespa/searchlib/transactionlog/domainpart.h
@@ -3,11 +3,11 @@
#include "common.h"
#include "ichunk.h"
-#include <vespa/vespalib/util/sync.h>
#include <vespa/vespalib/util/memory.h>
#include <map>
#include <vector>
#include <atomic>
+#include <mutex>
class FastOS_FileInterface;
@@ -34,7 +34,7 @@ public:
SerialNumRange range() const { return _range; }
SerialNum getSynced() const {
- vespalib::LockGuard guard(_writeLock);
+ std::lock_guard guard(_writeLock);
return _syncedSerial;
}
@@ -74,8 +74,8 @@ private:
typedef std::map<SerialNum, Packet> PacketList;
const Encoding _encoding;
const uint8_t _compressionLevel;
- vespalib::Lock _lock;
- vespalib::Lock _fileLock;
+ std::mutex _lock;
+ std::mutex _fileLock;
SerialNumRange _range;
size_t _sz;
std::atomic<uint64_t> _byteSize;
@@ -84,7 +84,7 @@ private:
std::unique_ptr<FastOS_FileInterface> _transLog;
SkipList _skipList;
uint32_t _headerLen;
- vespalib::Lock _writeLock;
+ mutable std::mutex _writeLock;
// Protected by _writeLock
SerialNum _writtenSerial;
SerialNum _syncedSerial;
diff --git a/searchlib/src/vespa/searchlib/transactionlog/translogclient.h b/searchlib/src/vespa/searchlib/transactionlog/translogclient.h
index 289a0fcb8c0..c232dfdad69 100644
--- a/searchlib/src/vespa/searchlib/transactionlog/translogclient.h
+++ b/searchlib/src/vespa/searchlib/transactionlog/translogclient.h
@@ -3,10 +3,10 @@
#include "client_common.h"
#include "client_session.h"
-#include <vespa/vespalib/util/sync.h>
#include <vespa/fnet/frt/invokable.h>
#include <map>
#include <vector>
+#include <mutex>
class FNET_Transport;
class FRT_Supervisor;
@@ -58,7 +58,7 @@ private:
vespalib::string _rpcTarget;
SessionMap _sessions;
//Brute force lock for subscriptions. For multithread safety.
- vespalib::Lock _lock;
+ std::mutex _lock;
std::unique_ptr<FastOS_ThreadPool> _threadPool;
std::unique_ptr<FNET_Transport> _transport;
std::unique_ptr<FRT_Supervisor> _supervisor;
diff --git a/staging_vespalib/CMakeLists.txt b/staging_vespalib/CMakeLists.txt
index f2f8a41b68d..efb57618823 100644
--- a/staging_vespalib/CMakeLists.txt
+++ b/staging_vespalib/CMakeLists.txt
@@ -23,7 +23,6 @@ vespa_define_module(
src/tests/floatingpointtype
src/tests/growablebytebuffer
src/tests/json
- src/tests/librarypool
src/tests/memorydatastore
src/tests/metrics
src/tests/objectdump
diff --git a/staging_vespalib/src/tests/librarypool/.gitignore b/staging_vespalib/src/tests/librarypool/.gitignore
deleted file mode 100644
index 1a1aea2fda0..00000000000
--- a/staging_vespalib/src/tests/librarypool/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-staging_vespalib_librarypool_test_app
diff --git a/staging_vespalib/src/tests/librarypool/CMakeLists.txt b/staging_vespalib/src/tests/librarypool/CMakeLists.txt
deleted file mode 100644
index 83e1e92e680..00000000000
--- a/staging_vespalib/src/tests/librarypool/CMakeLists.txt
+++ /dev/null
@@ -1,8 +0,0 @@
-# Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-vespa_add_executable(staging_vespalib_librarypool_test_app TEST
- SOURCES
- librarypool_test.cpp
- DEPENDS
- staging_vespalib
-)
-vespa_add_test(NAME staging_vespalib_librarypool_test_app COMMAND staging_vespalib_librarypool_test_app)
diff --git a/staging_vespalib/src/tests/librarypool/librarypool_test.cpp b/staging_vespalib/src/tests/librarypool/librarypool_test.cpp
deleted file mode 100644
index adefdf3aa6b..00000000000
--- a/staging_vespalib/src/tests/librarypool/librarypool_test.cpp
+++ /dev/null
@@ -1,38 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/vespalib/testkit/testapp.h>
-#include <vespa/vespalib/util/librarypool.h>
-#include <vespa/vespalib/util/exceptions.h>
-
-using namespace vespalib;
-
-class Test : public TestApp
-{
-public:
- int Main() override;
-};
-
-int
-Test::Main()
-{
- TEST_INIT("librarypool_test");
- LibraryPool p;
- ASSERT_TRUE(p.get("z") == NULL);
- p.loadLibrary("z");
- ASSERT_TRUE(p.get("z") != NULL);
- ASSERT_TRUE(p.get("z")->GetSymbol("some_symbol_that_is_not_there") == NULL);
- ASSERT_TRUE(p.get("z")->GetSymbol("compress") != NULL);
- try {
- p.loadLibrary("not_found");
- ASSERT_TRUE(false);
- } catch (const IllegalArgumentException & e) {
- ASSERT_TRUE(p.get("not_found") == NULL);
- }
- {
- const LibraryPool & c(p);
- ASSERT_TRUE(c.get("z") != NULL);
- ASSERT_TRUE(c.get("not_found") == NULL);
- }
- TEST_DONE();
-}
-
-TEST_APPHOOK(Test)
diff --git a/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.cpp b/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.cpp
index d016b052dae..d4fe0ef43e1 100644
--- a/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.cpp
+++ b/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.cpp
@@ -10,27 +10,28 @@ SimpleComponentConfigProducer::SimpleComponentConfigProducer()
{
}
+SimpleComponentConfigProducer::~SimpleComponentConfigProducer() = default;
+
void
SimpleComponentConfigProducer::addConfig(const Config &config)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_state.insert(std::make_pair(config.name, config)).first->second = config;
}
void
SimpleComponentConfigProducer::removeConfig(const vespalib::string &name)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_state.erase(name);
}
void
SimpleComponentConfigProducer::getComponentConfig(Consumer &consumer)
{
- typedef std::map<vespalib::string, Config>::const_iterator ITR;
- LockGuard guard(_lock);
- for (ITR itr = _state.begin(); itr != _state.end(); ++itr) {
- consumer.add(itr->second);
+ std::lock_guard guard(_lock);
+ for (const auto & entry : _state) {
+ consumer.add(entry.second);
}
}
diff --git a/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.h b/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.h
index 5783139c9d5..70dd00a8792 100644
--- a/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.h
+++ b/staging_vespalib/src/vespa/vespalib/net/simple_component_config_producer.h
@@ -3,19 +3,20 @@
#pragma once
#include "component_config_producer.h"
-#include <vespa/vespalib/util/sync.h>
#include <map>
+#include <mutex>
namespace vespalib {
class SimpleComponentConfigProducer : public ComponentConfigProducer
{
private:
- Lock _lock;
+ std::mutex _lock;
std::map<vespalib::string, Config> _state;
public:
SimpleComponentConfigProducer();
+ ~SimpleComponentConfigProducer() override;
void addConfig(const Config &config);
void removeConfig(const vespalib::string &name);
void getComponentConfig(Consumer &consumer) override;
diff --git a/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.cpp b/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.cpp
index 651dab97e68..a25888399c1 100644
--- a/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.cpp
+++ b/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.cpp
@@ -4,7 +4,6 @@
#include <vespa/defaults.h>
#include <fcntl.h>
#include <sys/stat.h>
-#include <sys/types.h>
#include <unistd.h>
namespace {
@@ -51,28 +50,26 @@ SimpleHealthProducer::SimpleHealthProducer()
setOk();
}
-SimpleHealthProducer::~SimpleHealthProducer()
-{
-}
+SimpleHealthProducer::~SimpleHealthProducer() = default;
void
SimpleHealthProducer::setOk()
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_health = Health(true, "All OK");
}
void
SimpleHealthProducer::setFailed(const vespalib::string &msg)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_health = Health(false, msg);
}
HealthProducer::Health
SimpleHealthProducer::getHealth() const
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
if (_health.ok && diskFailed()) {
return Health(false, "disk ping failed");
}
diff --git a/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.h b/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.h
index 90fe2489b0a..fc89f5c7644 100644
--- a/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.h
+++ b/staging_vespalib/src/vespa/vespalib/net/simple_health_producer.h
@@ -3,19 +3,19 @@
#pragma once
#include "health_producer.h"
-#include <vespa/vespalib/util/sync.h>
+#include <mutex>
namespace vespalib {
class SimpleHealthProducer : public HealthProducer
{
private:
- Lock _lock;
+ mutable std::mutex _lock;
HealthProducer::Health _health;
public:
SimpleHealthProducer();
- ~SimpleHealthProducer();
+ ~SimpleHealthProducer() override;
void setOk();
void setFailed(const vespalib::string &msg);
Health getHealth() const override;
diff --git a/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.cpp b/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.cpp
index 97be2a61235..b39bc96a5b6 100644
--- a/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.cpp
+++ b/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.cpp
@@ -11,35 +11,33 @@ SimpleMetricsProducer::SimpleMetricsProducer()
{
}
-SimpleMetricsProducer::~SimpleMetricsProducer()
-{
-}
+SimpleMetricsProducer::~SimpleMetricsProducer() = default;
void
SimpleMetricsProducer::setMetrics(const vespalib::string &metrics)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_metrics = metrics;
}
vespalib::string
SimpleMetricsProducer::getMetrics(const vespalib::string &)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
return _metrics;
}
void
SimpleMetricsProducer::setTotalMetrics(const vespalib::string &metrics)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_totalMetrics = metrics;
}
vespalib::string
SimpleMetricsProducer::getTotalMetrics(const vespalib::string &)
{
- LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
return _totalMetrics;
}
diff --git a/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.h b/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.h
index 1dd1452c32f..fdcf1dce6b6 100644
--- a/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.h
+++ b/staging_vespalib/src/vespa/vespalib/net/simple_metrics_producer.h
@@ -3,24 +3,24 @@
#pragma once
#include "metrics_producer.h"
-#include <vespa/vespalib/util/sync.h>
+#include <mutex>
namespace vespalib {
class SimpleMetricsProducer : public MetricsProducer
{
private:
- Lock _lock;
+ std::mutex _lock;
vespalib::string _metrics;
vespalib::string _totalMetrics;
public:
SimpleMetricsProducer();
- ~SimpleMetricsProducer();
+ ~SimpleMetricsProducer() override;
void setMetrics(const vespalib::string &metrics);
- virtual vespalib::string getMetrics(const vespalib::string &consumer) override;
+ vespalib::string getMetrics(const vespalib::string &consumer) override;
void setTotalMetrics(const vespalib::string &metrics);
- virtual vespalib::string getTotalMetrics(const vespalib::string &consumer) override;
+ vespalib::string getTotalMetrics(const vespalib::string &consumer) override;
};
} // namespace vespalib
diff --git a/staging_vespalib/src/vespa/vespalib/util/CMakeLists.txt b/staging_vespalib/src/vespa/vespalib/util/CMakeLists.txt
index 586e06396e7..70f17f76e4c 100644
--- a/staging_vespalib/src/vespa/vespalib/util/CMakeLists.txt
+++ b/staging_vespalib/src/vespa/vespalib/util/CMakeLists.txt
@@ -12,7 +12,6 @@ vespa_add_library(staging_vespalib_vespalib_util OBJECT
jsonexception.cpp
jsonstream.cpp
jsonwriter.cpp
- librarypool.cpp
process_memory_stats.cpp
programoptions.cpp
programoptions_testutils.cpp
diff --git a/staging_vespalib/src/vespa/vespalib/util/librarypool.cpp b/staging_vespalib/src/vespa/vespalib/util/librarypool.cpp
deleted file mode 100644
index 2a3ca21c369..00000000000
--- a/staging_vespalib/src/vespa/vespalib/util/librarypool.cpp
+++ /dev/null
@@ -1,58 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include <vespa/vespalib/util/librarypool.h>
-#include <vespa/vespalib/util/exceptions.h>
-#include <vespa/vespalib/util/stringfmt.h>
-
-namespace vespalib {
-
-LibraryPool::LibraryPool() :
- _libraries(),
- _lock()
-{
-}
-
-LibraryPool::~LibraryPool()
-{
- LockGuard guard(_lock);
- _libraries.clear();
-}
-
-void
-LibraryPool::loadLibrary(stringref libName)
-{
- LockGuard guard(_lock);
- if (_libraries.find(libName) == _libraries.end()) {
- DynamicLibrarySP lib(new FastOS_DynamicLibrary);
- string file(libName);
- if (!lib->Open(file.c_str())) {
- string error = lib->GetLastErrorString();
- throw IllegalArgumentException(make_string("Failed loading dynamic library '%s' due to '%s'.",
- file.c_str(), error.c_str()));
- } else {
- _libraries[libName] = lib;
- }
- }
-}
-
-FastOS_DynamicLibrary *
-LibraryPool::get(stringref name)
-{
- LockGuard guard(_lock);
- LibraryMap::const_iterator found(_libraries.find(name));
- return (found != _libraries.end())
- ? found->second.get()
- : NULL;
-}
-
-const FastOS_DynamicLibrary *
-LibraryPool::get(stringref name) const
-{
- LockGuard guard(_lock);
- LibraryMap::const_iterator found(_libraries.find(name));
- return (found != _libraries.end())
- ? found->second.get()
- : NULL;
-}
-
-}
diff --git a/staging_vespalib/src/vespa/vespalib/util/librarypool.h b/staging_vespalib/src/vespa/vespalib/util/librarypool.h
deleted file mode 100644
index f9149589338..00000000000
--- a/staging_vespalib/src/vespa/vespalib/util/librarypool.h
+++ /dev/null
@@ -1,39 +0,0 @@
-// 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 <vespa/vespalib/stllike/string.h>
-#include <vespa/fastos/dynamiclibrary.h>
-#include <map>
-
-namespace vespalib {
-
-class LibraryPool
-{
-public:
- LibraryPool();
- ~LibraryPool();
- /**
- * This will load the library with the name given.
- * - It will verify linkage at load time.
- * - Symbols will be private.
- * @param name The name of the library to load. That is without the 'lib' prefix and the '.so' extension.
- * @throws IllegalArgumentException if there are any errors.
- */
- void loadLibrary(stringref name);
- /**
- * Will return the library requested. NULL if not found.
- * @param name The name of the library as given in the @ref loadLibrary call.
- * @return The library that has already been loaded. NULL if not found.
- */
- FastOS_DynamicLibrary *get(stringref name);
- const FastOS_DynamicLibrary *get(stringref name) const;
-private:
- typedef std::shared_ptr<FastOS_DynamicLibrary> DynamicLibrarySP;
- typedef std::map<vespalib::string, DynamicLibrarySP> LibraryMap;
- LibraryMap _libraries;
- Lock _lock;
-};
-
-}
-
diff --git a/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.cpp b/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.cpp
index d9b4feda293..3f5b3b79656 100644
--- a/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.cpp
+++ b/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.cpp
@@ -45,7 +45,7 @@ ScheduledExecutor::ScheduledExecutor()
ScheduledExecutor::~ScheduledExecutor()
{
- vespalib::LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_transport->ShutDown(true);
_threadPool.Close();
_taskList.clear();
@@ -55,7 +55,7 @@ ScheduledExecutor::~ScheduledExecutor()
void
ScheduledExecutor::scheduleAtFixedRate(vespalib::Executor::Task::UP task, duration delay, duration interval)
{
- vespalib::LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
TimerTaskPtr tTask(new TimerTask(_transport->GetScheduler(), std::move(task), interval));
_taskList.push_back(std::move(tTask));
_taskList.back()->Schedule(to_s(delay));
@@ -64,10 +64,10 @@ ScheduledExecutor::scheduleAtFixedRate(vespalib::Executor::Task::UP task, durati
void
ScheduledExecutor::reset()
{
- vespalib::LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
_transport->ShutDown(true);
_taskList.clear();
- _transport.reset(new FNET_Transport());
+ _transport = std::make_unique<FNET_Transport>();
_transport->Start(&_threadPool);
}
diff --git a/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.h b/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.h
index 0f052c762a7..0b98a236f74 100644
--- a/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.h
+++ b/staging_vespalib/src/vespa/vespalib/util/scheduledexecutor.h
@@ -2,7 +2,6 @@
#pragma once
#include <vespa/vespalib/util/executor.h>
-#include <vespa/vespalib/util/sync.h>
#include <vespa/vespalib/util/time.h>
#include <vespa/fastos/thread.h>
#include <vector>
@@ -25,8 +24,8 @@ private:
typedef std::vector<TimerTaskPtr> TaskList;
FastOS_ThreadPool _threadPool;
std::unique_ptr<FNET_Transport> _transport;
- vespalib::Lock _lock;
- TaskList _taskList;
+ std::mutex _lock;
+ TaskList _taskList;
public:
/**
diff --git a/vbench/src/vbench/core/dispatcher.h b/vbench/src/vbench/core/dispatcher.h
index 03e036fe261..21db6c931a4 100644
--- a/vbench/src/vbench/core/dispatcher.h
+++ b/vbench/src/vbench/core/dispatcher.h
@@ -5,7 +5,6 @@
#include "handler.h"
#include "provider.h"
#include "closeable.h"
-#include <vespa/vespalib/util/sync.h>
#include <vespa/vespalib/util/gate.h>
#include <vector>
@@ -31,7 +30,7 @@ private:
};
Handler<T> &_fallback;
- vespalib::Lock _lock;
+ mutable std::mutex _lock;
std::vector<ThreadState*> _threads;
bool _closed;
diff --git a/vbench/src/vbench/core/dispatcher.hpp b/vbench/src/vbench/core/dispatcher.hpp
index 9fd5c5dd527..9a985797967 100644
--- a/vbench/src/vbench/core/dispatcher.hpp
+++ b/vbench/src/vbench/core/dispatcher.hpp
@@ -14,7 +14,7 @@ Dispatcher<T>::Dispatcher(Handler<T> &fallback)
}
template <typename T>
-Dispatcher<T>::~Dispatcher() {}
+Dispatcher<T>::~Dispatcher() = default;
template <typename T>
bool
@@ -25,7 +25,7 @@ Dispatcher<T>::waitForThreads(size_t threads, size_t pollCnt) const
vespalib::Thread::sleep(20);
}
{
- vespalib::LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
if (_threads.size() >= threads) {
return true;
}
@@ -40,7 +40,7 @@ Dispatcher<T>::close()
{
std::vector<ThreadState*> threads;
{
- vespalib::LockGuard guard(_lock);
+ std::lock_guard guard(_lock);
std::swap(_threads, threads);
_closed = true;
}
@@ -53,7 +53,7 @@ template <typename T>
void
Dispatcher<T>::handle(std::unique_ptr<T> obj)
{
- vespalib::LockGuard guard(_lock);
+ std::unique_lock guard(_lock);
if (!_threads.empty()) {
ThreadState *state = _threads.back();
_threads.pop_back();
@@ -75,7 +75,7 @@ Dispatcher<T>::provide()
{
ThreadState state;
{
- vespalib::LockGuard guard(_lock);
+ std::unique_lock guard(_lock);
if (!_closed) {
_threads.push_back(&state);
guard.unlock();