summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2019-11-19 15:54:28 +0000
committerHenning Baldersheim <balder@yahoo-inc.com>2019-11-20 10:16:59 +0000
commit1776164d76a34d81e887924c372be4c5b3cddc2a (patch)
tree2bc98d5fb22653eb259b4656f5215ecd8cdd2a81
parentd328b6ce8389fb12c76d813476fff925c452c1cb (diff)
Address comment by specifying timeunit in the type.
-rw-r--r--fastos/src/vespa/fastos/timestamp.cpp12
-rw-r--r--fastos/src/vespa/fastos/timestamp.h9
-rw-r--r--messagebus_test/src/tests/speed/cpp-client.cpp4
-rw-r--r--searchcore/src/tests/proton/common/cachedselect_test.cpp9
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attribute_initializer.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastoreinitializer.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/flushengine/flushengine.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_master.cpp8
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp3
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/matcher.cpp1
-rw-r--r--searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.cpp16
-rw-r--r--searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.cpp11
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.h16
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.h4
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/fusionrunner.cpp2
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp4
-rw-r--r--searchlib/src/apps/vespa-attribute-inspect/vespa-attribute-inspect.cpp6
-rw-r--r--searchlib/src/tests/util/bufferwriter/CMakeLists.txt8
-rw-r--r--searchlib/src/tests/util/bufferwriter/bm.cpp93
-rw-r--r--searchlib/src/tests/util/bufferwriter/bufferwriter_test.cpp5
-rw-r--r--searchlib/src/tests/util/bufferwriter/work.cpp92
-rw-r--r--searchlib/src/tests/util/bufferwriter/work.h22
-rw-r--r--searchlib/src/vespa/searchlib/features/debug_attribute_wait.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/features/debug_wait.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/features/random_normal_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/randomfeature.cpp2
-rw-r--r--searchsummary/src/tests/extractkeywords/extractkeywordstest.cpp2
-rw-r--r--storage/src/tests/storageserver/documentapiconvertertest.cpp2
-rw-r--r--storage/src/vespa/storage/storageserver/documentapiconverter.cpp2
-rw-r--r--vespalib/src/tests/btree/iteratespeed.cpp2
-rw-r--r--vespalib/src/vespa/vespalib/util/time_tracker.h7
34 files changed, 66 insertions, 304 deletions
diff --git a/fastos/src/vespa/fastos/timestamp.cpp b/fastos/src/vespa/fastos/timestamp.cpp
index 5daad63f698..c7f269b59b9 100644
--- a/fastos/src/vespa/fastos/timestamp.cpp
+++ b/fastos/src/vespa/fastos/timestamp.cpp
@@ -87,20 +87,18 @@ SteadyTimeStamp::toUTC() const {
}
StopWatch::StopWatch()
- : _startTime(steady_now()),
- _stopTime(_startTime)
+ : _startTime(steady_now())
{ }
void
StopWatch::restart() {
_startTime = steady_now();
- _stopTime = _startTime;
}
-StopWatch &
-StopWatch::stop() {
- _stopTime = steady_now();
- return *this;
+TimeStamp
+StopWatch::elapsed() const {
+ TimeStamp diff(steady_now() - _startTime);
+ return (diff > 0) ? diff : TimeStamp(0);
}
void
diff --git a/fastos/src/vespa/fastos/timestamp.h b/fastos/src/vespa/fastos/timestamp.h
index 36f296b891f..79d6ef5eed6 100644
--- a/fastos/src/vespa/fastos/timestamp.h
+++ b/fastos/src/vespa/fastos/timestamp.h
@@ -151,18 +151,11 @@ class StopWatch
{
public:
StopWatch();
-
- StopWatch & stop();
void restart();
-
- TimeStamp elapsed() const {
- TimeStamp diff(_stopTime - _startTime);
- return (diff > 0) ? diff : TimeStamp(0);
- }
+ TimeStamp elapsed() const;
static void waitAtLeast(std::chrono::microseconds us, bool busyWait);
private:
SteadyTimeStamp _startTime;
- SteadyTimeStamp _stopTime;
};
time_t time();
diff --git a/messagebus_test/src/tests/speed/cpp-client.cpp b/messagebus_test/src/tests/speed/cpp-client.cpp
index 59ca47514c9..c1d8a8c1c80 100644
--- a/messagebus_test/src/tests/speed/cpp-client.cpp
+++ b/messagebus_test/src/tests/speed/cpp-client.cpp
@@ -117,9 +117,9 @@ App::Main()
client.sample(okBefore, failBefore);
FastOS_Thread::Sleep(10000); // Benchmark time
- stopWatch.stop();
+ fastos::TimeStamp elapsed = stopWatch.elapsed();
client.sample(okAfter, failAfter);
- double time = stopWatch.elapsed().ms();
+ double time = elapsed.ms();
double msgCnt = (double)(okAfter - okBefore);
double throughput = (msgCnt / time) * 1000.0;
fprintf(stdout, "CPP-CLIENT: %g msg/s\n", throughput);
diff --git a/searchcore/src/tests/proton/common/cachedselect_test.cpp b/searchcore/src/tests/proton/common/cachedselect_test.cpp
index aa2106923e3..74d65a5bf7f 100644
--- a/searchcore/src/tests/proton/common/cachedselect_test.cpp
+++ b/searchcore/src/tests/proton/common/cachedselect_test.cpp
@@ -632,14 +632,11 @@ TEST_F("Test performance when using attributes", TestFixture)
if (sel->contains(ctx) != Result::Invalid)
break;
}
- sw.stop();
+ fastos::TimeStamp elapsed = sw.elapsed();
EXPECT_EQUAL(loopcnt, i);
LOG(info,
- "Elapsed time for %u iterations of 4 docs each: %" PRId64 " ns, "
- "%8.4f ns/doc",
- i,
- sw.elapsed().ns(),
- static_cast<double>(sw.elapsed().ns()) / ( 4 * i));
+ "Elapsed time for %u iterations of 4 docs each: %" PRId64 " ns, %8.4f ns/doc",
+ i, elapsed.ns(), static_cast<double>(elapsed.ns()) / ( 4 * i));
}
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attribute_initializer.cpp b/searchcore/src/vespa/searchcore/proton/attribute/attribute_initializer.cpp
index c3063521a99..bb41004d834 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/attribute_initializer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attribute_initializer.cpp
@@ -183,7 +183,7 @@ AttributeInitializer::loadAttribute(const AttributeVectorSP &attr,
return false;
} else {
attr->commit(serialNum, serialNum);
- EventLogger::loadAttributeComplete(_documentSubDbName, attr->getName(), stopWatch.stop().elapsed().ms());
+ EventLogger::loadAttributeComplete(_documentSubDbName, attr->getName(), stopWatch.elapsed().ms());
}
return true;
}
diff --git a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp
index 5cdfbf03c07..1915d9107cc 100644
--- a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp
@@ -43,7 +43,7 @@ SummaryManagerInitializer::run()
*_result = std::make_shared<SummaryManager>
(_summaryExecutor, _storeCfg, _grow, _baseDir, _docTypeName,
_tuneFile, _fileHeaderContext, _tlSyncer, _bucketizer);
- EventLogger::loadDocumentStoreComplete(_subDbName, stopWatch.stop().elapsed().ms());
+ EventLogger::loadDocumentStoreComplete(_subDbName, stopWatch.elapsed().ms());
}
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastoreinitializer.cpp b/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastoreinitializer.cpp
index eed333a11d8..c4b6d88c1c3 100644
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastoreinitializer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/documentmetastore/documentmetastoreinitializer.cpp
@@ -51,7 +51,7 @@ DocumentMetaStoreInitializer::run()
} else {
_dms->commit(snap.syncToken, snap.syncToken);
}
- EventLogger::loadDocumentMetaStoreComplete(_subDbName, stopWatch.stop().elapsed().ms());
+ EventLogger::loadDocumentMetaStoreComplete(_subDbName, stopWatch.elapsed().ms());
}
} else {
vespalib::mkdir(_baseDir, false);
diff --git a/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.h b/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.h
index 38f8eabf828..97ba51a1b97 100644
--- a/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.h
+++ b/searchcore/src/vespa/searchcore/proton/flushengine/flushengine.h
@@ -24,12 +24,12 @@ public:
~FlushMeta();
const vespalib::string & getName() const { return _name; }
fastos::UTCTimeStamp getStart() const { return fastos::ClockSystem::now() - elapsed(); }
- fastos::TimeStamp elapsed() const { return _stopWatch.stop().elapsed(); }
+ fastos::TimeStamp elapsed() const { return _stopWatch.elapsed(); }
uint32_t getId() const { return _id; }
bool operator < (const FlushMeta & rhs) const { return _id < rhs._id; }
private:
vespalib::string _name;
- mutable fastos::StopWatch _stopWatch;
+ fastos::StopWatch _stopWatch;
uint32_t _id;
};
typedef std::set<FlushMeta> FlushMetaSet;
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
index 20cd739020e..b3b97da6752 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
@@ -30,18 +30,19 @@ namespace {
struct TimedMatchLoopCommunicator : IMatchLoopCommunicator {
IMatchLoopCommunicator &communicator;
fastos::StopWatch rerank_time;
+ fastos::TimeStamp elapsed;
TimedMatchLoopCommunicator(IMatchLoopCommunicator &com) : communicator(com) {}
double estimate_match_frequency(const Matches &matches) override {
return communicator.estimate_match_frequency(matches);
}
Hits selectBest(SortedHitSequence sortedHits) override {
auto result = communicator.selectBest(sortedHits);
- rerank_time = fastos::StopWatch();
+ rerank_time.restart();
return result;
}
RangePair rangeCover(const RangePair &ranges) override {
RangePair result = communicator.rangeCover(ranges);
- rerank_time.stop();
+ elapsed = rerank_time.elapsed();
return result;
}
};
@@ -89,9 +90,8 @@ MatchMaster::match(search::engine::Trace & trace,
resultProcessor.prepareThreadContextCreation(threadBundle.size());
threadBundle.run(targets);
ResultProcessor::Result::UP reply = resultProcessor.makeReply(threadState[0]->extract_result());
- query_latency_time.stop();
double query_time_s = query_latency_time.elapsed().sec();
- double rerank_time_s = timedCommunicator.rerank_time.elapsed().sec();
+ double rerank_time_s = timedCommunicator.elapsed.sec();
double match_time_s = 0.0;
std::unique_ptr<vespalib::slime::Inserter> inserter;
if (trace.shouldTrace(4)) {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
index 71ff9696050..e2695340725 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
@@ -41,7 +41,6 @@ struct WaitTimer {
: wait_time_s(wait_time_s_in), wait_time()
{ }
void done() {
- wait_time.stop();
wait_time_s += wait_time.elapsed().sec();
}
};
@@ -434,7 +433,6 @@ MatchThread::run()
trace->addEvent(4, "Start MatchThread::run");
MatchTools::UP matchTools = matchToolsFactory.createMatchTools();
search::ResultSet::UP result = findMatches(*matchTools);
- match_time.stop();
match_time_s = match_time.elapsed().sec();
resultContext = resultProcessor.createThreadContext(matchTools->getHardDoom(), thread_id, _distributionKey);
{
@@ -450,7 +448,6 @@ MatchThread::run()
trace->addEvent(5, "Start result processing");
processResult(matchTools->getHardDoom(), std::move(result), *resultContext);
}
- total_time.stop();
total_time_s = total_time.elapsed().sec();
thread_stats.active_time(total_time_s - wait_time_s).wait_time(wait_time_s);
trace->addEvent(4, "Start thread merge");
diff --git a/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp b/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
index 48599b1ecff..dace9674d64 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
@@ -279,7 +279,6 @@ Matcher::match(const SearchRequest &request, vespalib::ThreadBundle &threadBundl
numThreadsPerSearch, _rankSetup->getNumThreadsPerSearch(), estHits, reply->totalHitCount,
request.ranking.c_str());
}
- total_matching_time.stop();
my_stats.queryCollateralTime(total_matching_time.elapsed().sec() - my_stats.queryLatencyAvg());
{
fastos::TimeStamp duration = request.getTimeUsed();
diff --git a/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.cpp b/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.cpp
index 12e612869d1..d8e0fcc7843 100644
--- a/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.cpp
+++ b/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.cpp
@@ -5,6 +5,9 @@
#include "document_reprocessing_handler.h"
#include <vespa/searchcore/proton/common/eventlogger.h>
+using namespace std::chrono_literals;
+using namespace std::chrono;
+
namespace proton {
ReprocessDocumentsTask::
@@ -33,7 +36,7 @@ ReprocessDocumentsTask::run()
{
if (_handler.hasProcessors()) {
EventLogger::reprocessDocumentsStart(_subDbName, _visitorCost);
- _stopWatch = fastos::StopWatch();
+ _start = clock::now();
search::IDocumentStore &docstore = _sm->getBackingStore();
if (_handler.hasRewriters()) {
docstore.accept(_handler.getRewriteVisitor(), *this, *_docTypeRepo);
@@ -41,8 +44,8 @@ ReprocessDocumentsTask::run()
docstore.accept(_handler, *this, *_docTypeRepo);
}
_handler.done();
- _stopWatch.stop();
- EventLogger::reprocessDocumentsComplete(_subDbName, _visitorCost, _stopWatch.elapsed().ms());
+ EventLogger::reprocessDocumentsComplete(_subDbName, _visitorCost,
+ duration_cast<milliseconds>(clock::now() - _start).count());
}
}
@@ -52,11 +55,10 @@ ReprocessDocumentsTask::updateProgress(double progress)
_visitorProgress = progress;
double deltaProgress = progress - _loggedProgress;
if (deltaProgress >= 0.01) {
- fastos::StopWatch intermediate = _stopWatch;
- fastos::TimeStamp logDelayTime = intermediate.stop().elapsed() - _stopWatch.elapsed();
- if (logDelayTime.ms() >= 60000 || deltaProgress >= 0.10) {
+ auto secondsSinceLastLog = duration_cast<seconds>(clock::now() - _lastLogTime);
+ if (secondsSinceLastLog >= 60s || deltaProgress >= 0.10) {
EventLogger::reprocessDocumentsProgress(_subDbName, progress, _visitorCost);
- _stopWatch.stop();
+ _lastLogTime = clock::now();
_loggedProgress = progress;
}
}
diff --git a/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.h b/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.h
index f64eddd29ee..fc82f475412 100644
--- a/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.h
+++ b/searchcore/src/vespa/searchcore/proton/reprocessing/reprocess_documents_task.h
@@ -20,13 +20,15 @@ namespace proton
class ReprocessDocumentsTask : public IReprocessingTask,
public search::IDocumentStoreVisitorProgress
{
+ using clock = std::chrono::steady_clock;
proton::ISummaryManager::SP _sm;
std::shared_ptr<const document::DocumentTypeRepo> _docTypeRepo;
vespalib::string _subDbName;
double _visitorProgress;
double _visitorCost;
DocumentReprocessingHandler _handler;
- fastos::StopWatch _stopWatch;
+ clock::time_point _start;
+ clock::time_point _lastLogTime;
double _loggedProgress;
public:
diff --git a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.cpp b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.cpp
index 2f6eaeadf5c..271393ea3c8 100644
--- a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.cpp
@@ -18,14 +18,13 @@ namespace proton {
void
TransactionLogManager::doLogReplayComplete(const vespalib::string &domainName,
- int64_t elapsedTime) const
+ std::chrono::milliseconds elapsedTime) const
{
- EventLogger::transactionLogReplayComplete(domainName, elapsedTime);
+ EventLogger::transactionLogReplayComplete(domainName, elapsedTime.count());
}
-TransactionLogManager::TransactionLogManager(const vespalib::string &tlsSpec,
- const vespalib::string &domainName)
+TransactionLogManager::TransactionLogManager(const vespalib::string &tlsSpec, const vespalib::string &domainName)
: TransactionLogManagerBase(tlsSpec, domainName),
_visitor()
{
@@ -34,9 +33,7 @@ TransactionLogManager::TransactionLogManager(const vespalib::string &tlsSpec,
TransactionLogManager::~TransactionLogManager() = default;
void
-TransactionLogManager::init(SerialNum oldestConfigSerial,
- SerialNum &prunedSerialNum,
- SerialNum &serialNum)
+TransactionLogManager::init(SerialNum oldestConfigSerial, SerialNum &prunedSerialNum, SerialNum &serialNum)
{
StatusResult res = TransactionLogManagerBase::init();
prunedSerialNum = res.serialBegin > 0 ? (res.serialBegin - 1) : 0;
diff --git a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.h b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.h
index 8f48b9adc48..82c4f9f7449 100644
--- a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.h
+++ b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanager.h
@@ -18,8 +18,7 @@ class TransactionLogManager : public TransactionLogManagerBase
{
TransLogClient::Visitor::UP _visitor;
- virtual void doLogReplayComplete(const vespalib::string &domainName,
- int64_t elapsedTime) const override;
+ void doLogReplayComplete(const vespalib::string &domainName, std::chrono::milliseconds elapsedTime) const override;
public:
/**
@@ -28,8 +27,7 @@ public:
* @param tlsSpec the spec of the transaction log server.
* @param domainName the name of the domain this manager should handle.
**/
- TransactionLogManager(const vespalib::string &tlsSpec,
- const vespalib::string &domainName);
+ TransactionLogManager(const vespalib::string &tlsSpec, const vespalib::string &domainName);
~TransactionLogManager();
/**
@@ -41,10 +39,7 @@ public:
* @param the current serial num will be set to 1 higher than
* the serial num of the last entry in the transaction log.
**/
- void
- init(SerialNum oldestConfigSerial,
- SerialNum &prunedSerialNum,
- SerialNum &serialNum);
+ void init(SerialNum oldestConfigSerial, SerialNum &prunedSerialNum, SerialNum &serialNum);
/**
* Prepare replay of the transaction log.
@@ -59,10 +54,7 @@ public:
/**
* Start replay of the transaction log.
**/
- TlsReplayProgress::UP
- startReplay(SerialNum first,
- SerialNum syncToken,
- TransLogClient::Session::Callback &callback);
+ TlsReplayProgress::UP startReplay(SerialNum first, SerialNum syncToken, TransLogClient::Session::Callback &callback);
/**
* Indicate that replay is done.
diff --git a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.cpp b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.cpp
index 5598b80b1ca..985e042c97b 100644
--- a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.cpp
@@ -69,7 +69,7 @@ TransactionLogManagerBase::internalStartReplay()
std::lock_guard<std::mutex> guard(_replayLock);
_replayStarted = true;
_replayDone = false;
- _replayStopWatch = fastos::StopWatch();
+ _replayStopWatch.restart();
}
void TransactionLogManagerBase::changeReplayDone()
@@ -117,7 +117,7 @@ bool TransactionLogManagerBase::isDoingReplay() const {
}
void TransactionLogManagerBase::logReplayComplete() const {
- doLogReplayComplete(_domainName, _replayStopWatch.stop().elapsed().ms());
+ doLogReplayComplete(_domainName, std::chrono::milliseconds(_replayStopWatch.elapsed().ms()));
}
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.h b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.h
index d5a7ab41af0..8c4bc2bbfa3 100644
--- a/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.h
+++ b/searchcore/src/vespa/searchcore/proton/server/transactionlogmanagerbase.h
@@ -23,7 +23,7 @@ private:
mutable std::condition_variable _replayCond;
volatile bool _replayDone;
bool _replayStarted;
- mutable fastos::StopWatch _replayStopWatch;
+ fastos::StopWatch _replayStopWatch;
protected:
typedef search::SerialNum SerialNum;
@@ -38,7 +38,7 @@ protected:
StatusResult init();
void internalStartReplay();
- virtual void doLogReplayComplete(const vespalib::string &domainName, int64_t elapsedTime) const = 0;
+ virtual void doLogReplayComplete(const vespalib::string &domainName, std::chrono::milliseconds elapsedTime) const = 0;
public:
TransactionLogManagerBase(const TransactionLogManagerBase &) = delete;
diff --git a/searchcorespi/src/vespa/searchcorespi/index/fusionrunner.cpp b/searchcorespi/src/vespa/searchcorespi/index/fusionrunner.cpp
index fd9aea19faa..ca709362152 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/fusionrunner.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/fusionrunner.cpp
@@ -124,7 +124,7 @@ FusionRunner::fuse(const FusionSpec &fusion_spec,
}
if (LOG_WOULD_LOG(event)) {
- EventLogger::diskFusionComplete(fusion_dir, stopWatch.stop().elapsed().ms());
+ EventLogger::diskFusionComplete(fusion_dir, stopWatch.elapsed().ms());
}
return fusion_id;
}
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
index f9bd2c0776c..da3058e9bbd 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
@@ -282,7 +282,7 @@ IndexMaintainer::loadDiskIndex(const string &indexDir)
(_operations.loadDiskIndex(indexDir),
makeClosure(this, &IndexMaintainer::deactivateDiskIndexes, indexDir)));
if (LOG_WOULD_LOG(event)) {
- EventLogger::diskIndexLoadComplete(indexDir, stopWatch.stop().elapsed().ms());
+ EventLogger::diskIndexLoadComplete(indexDir, stopWatch.elapsed().ms());
}
return retval;
}
@@ -303,7 +303,7 @@ IndexMaintainer::reloadDiskIndex(const IDiskIndex &oldIndex)
(_operations.reloadDiskIndex(wrappedDiskIndex),
makeClosure(this, &IndexMaintainer::deactivateDiskIndexes, indexDir)));
if (LOG_WOULD_LOG(event)) {
- EventLogger::diskIndexLoadComplete(indexDir, stopWatch.stop().elapsed().ms());
+ EventLogger::diskIndexLoadComplete(indexDir, stopWatch.elapsed().ms());
}
return retval;
}
diff --git a/searchlib/src/apps/vespa-attribute-inspect/vespa-attribute-inspect.cpp b/searchlib/src/apps/vespa-attribute-inspect/vespa-attribute-inspect.cpp
index b976f76c3fb..10d63090464 100644
--- a/searchlib/src/apps/vespa-attribute-inspect/vespa-attribute-inspect.cpp
+++ b/searchlib/src/apps/vespa-attribute-inspect/vespa-attribute-inspect.cpp
@@ -167,14 +167,14 @@ LoadAttribute::Main()
AttributePtr ptr = AttributeFactory::createAttribute(fileName, c);
fastos::StopWatch timer;
load(ptr);
- std::cout << "load time: " << timer.stop().elapsed().ms() << " seconds " << std::endl;
+ std::cout << "load time: " << timer.elapsed().ms() << " seconds " << std::endl;
std::cout << "numDocs: " << ptr->getNumDocs() << std::endl;
if (doApplyUpdate) {
timer.restart();
applyUpdate(ptr);
- std::cout << "update time: " << timer.stop().elapsed().ms() << " seconds " << std::endl;
+ std::cout << "update time: " << timer.elapsed().ms() << " seconds " << std::endl;
}
if (doPrintContent) {
@@ -193,7 +193,7 @@ LoadAttribute::Main()
std::cout << "saving attribute: " << saveFile << std::endl;
timer.restart();
ptr->save(saveFile);
- std::cout << "save time: " << timer.stop().elapsed().ms() << " seconds " << std::endl;
+ std::cout << "save time: " << timer.elapsed().ms() << " seconds " << std::endl;
}
return 0;
diff --git a/searchlib/src/tests/util/bufferwriter/CMakeLists.txt b/searchlib/src/tests/util/bufferwriter/CMakeLists.txt
index 511c7b566cf..406dddb9a05 100644
--- a/searchlib/src/tests/util/bufferwriter/CMakeLists.txt
+++ b/searchlib/src/tests/util/bufferwriter/CMakeLists.txt
@@ -6,11 +6,3 @@ vespa_add_executable(searchlib_bufferwriter_test_app TEST
searchlib
)
vespa_add_test(NAME searchlib_bufferwriter_test_app COMMAND searchlib_bufferwriter_test_app)
-vespa_add_executable(searchlib_bufferwriter_bm_app
- SOURCES
- work.cpp
- bm.cpp
- DEPENDS
- searchlib
-)
-vespa_add_test(NAME searchlib_bufferwriter_bm_app COMMAND searchlib_bufferwriter_bm_app BENCHMARK)
diff --git a/searchlib/src/tests/util/bufferwriter/bm.cpp b/searchlib/src/tests/util/bufferwriter/bm.cpp
deleted file mode 100644
index ac3cc3a09b1..00000000000
--- a/searchlib/src/tests/util/bufferwriter/bm.cpp
+++ /dev/null
@@ -1,93 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include "work.h"
-#include <vespa/searchlib/util/drainingbufferwriter.h>
-#include <vespa/vespalib/testkit/testapp.h>
-#include <vespa/fastos/timestamp.h>
-#include <iostream>
-
-#include <vespa/log/log.h>
-LOG_SETUP("bufferwriter_bm");
-
-using search::DrainingBufferWriter;
-
-constexpr size_t million = 1000000;
-
-enum class WorkFuncDispatch
-{
- DIRECT,
- LAMBDA,
- FUNCTOR,
- FUNCTOR2
-};
-
-
-template <typename T>
-void
-callWork(size_t size, WorkFuncDispatch dispatch)
-{
- std::vector<T> foo;
- DrainingBufferWriter writer;
- foo.resize(size);
- std::cout << "will write " << size << " elements of size " << sizeof(T) <<
- std::endl;
- fastos::StopWatch stopWatch;
- switch (dispatch) {
- case WorkFuncDispatch::DIRECT:
- work(foo, writer);
- break;
- case WorkFuncDispatch::LAMBDA:
- workLambda(foo, writer);
- break;
- case WorkFuncDispatch::FUNCTOR:
- workFunctor(foo, writer);
- break;
- case WorkFuncDispatch::FUNCTOR2:
- workFunctor2(foo, writer);
- break;
- default:
- LOG_ABORT("should not be reached");
- }
- double delta = stopWatch.stop().elapsed();
- double writeSpeed = writer.getBytesWritten() / delta;
- EXPECT_GREATER(writeSpeed, 1000);
- std::cout << "written is " << writer.getBytesWritten() << std::endl;
- std::cout << "time used is " << (delta * 1000.0) << " ms" << std::endl;
- std::cout << "write speed is " << writeSpeed << std::endl;
-}
-
-
-void
-callWorks(WorkFuncDispatch dispatch)
-{
- callWork<char>(million * 1000, dispatch);
- callWork<short>(million * 500, dispatch);
- callWork<int>(million * 250, dispatch);
- callWork<long>(million * 125, dispatch);
-}
-
-TEST("simple bufferwriter speed test")
-{
- callWorks(WorkFuncDispatch::DIRECT);
-}
-
-TEST("lambda func bufferwriter speed test")
-{
- callWorks(WorkFuncDispatch::LAMBDA);
-}
-
-TEST("functor bufferwriter speed test")
-{
- callWorks(WorkFuncDispatch::FUNCTOR);
-}
-
-TEST("functor2 bufferwriter speed test")
-{
- callWorks(WorkFuncDispatch::FUNCTOR2);
-}
-
-
-TEST_MAIN()
-{
- TEST_RUN_ALL();
-}
diff --git a/searchlib/src/tests/util/bufferwriter/bufferwriter_test.cpp b/searchlib/src/tests/util/bufferwriter/bufferwriter_test.cpp
index bafb4105996..33afa67e660 100644
--- a/searchlib/src/tests/util/bufferwriter/bufferwriter_test.cpp
+++ b/searchlib/src/tests/util/bufferwriter/bufferwriter_test.cpp
@@ -6,8 +6,7 @@
#include <vespa/searchlib/util/drainingbufferwriter.h>
#include <vespa/searchlib/util/rand48.h>
-namespace search
-{
+namespace search {
namespace {
@@ -39,7 +38,7 @@ StoreBufferWriter::StoreBufferWriter()
setup(&_buf[0], _buf.size());
}
-StoreBufferWriter::~StoreBufferWriter() {}
+StoreBufferWriter::~StoreBufferWriter() = default;
void
diff --git a/searchlib/src/tests/util/bufferwriter/work.cpp b/searchlib/src/tests/util/bufferwriter/work.cpp
deleted file mode 100644
index bd5bf4a9d81..00000000000
--- a/searchlib/src/tests/util/bufferwriter/work.cpp
+++ /dev/null
@@ -1,92 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include "work.h"
-#include <vespa/vespalib/util/bufferwriter.h>
-
-namespace search
-{
-
-template <class T>
-class WriteFunctor
-{
- BufferWriter &_writer;
-public:
- WriteFunctor(BufferWriter &writer)
- : _writer(writer)
- {
- }
-
- void operator()(const T &val) { _writer.write(&val, sizeof(val)); }
-};
-
-template <class T>
-class WriteFunctor2
-{
- BufferWriter &_writer;
-public:
- WriteFunctor2(BufferWriter &writer)
- : _writer(writer)
- {
- }
-
- void operator()(const T &val) __attribute((noinline))
- { _writer.write(&val, sizeof(val)); }
-};
-
-template <class T, class Func>
-void workLoop(const std::vector<T> &v, Func &&func)
-{
- for (const auto &val : v) {
- func(val);
- }
-}
-
-template <class T>
-void work(const std::vector<T> &v, BufferWriter &writer)
-{
- for (const auto &val : v) {
- writer.write(&val, sizeof(val));
- }
- writer.flush();
-}
-
-template <class T>
-void workLambda(const std::vector<T> &v, BufferWriter &writer)
-{
- workLoop<T>(v,
- [&writer](const T &val) { writer.write(&val, sizeof(val)); });
- writer.flush();
-}
-
-template <class T>
-void workFunctor(const std::vector<T> &v, BufferWriter &writer)
-{
- workLoop<T>(v, WriteFunctor<T>(writer));
- writer.flush();
-}
-
-template <class T>
-void workFunctor2(const std::vector<T> &v, BufferWriter &writer)
-{
- workLoop<T>(v, WriteFunctor2<T>(writer));
- writer.flush();
-}
-
-template void work(const std::vector<char> &v, BufferWriter &writer);
-template void work(const std::vector<short> &v, BufferWriter &writer);
-template void work(const std::vector<int> &v, BufferWriter &writer);
-template void work(const std::vector<long> &v, BufferWriter &writer);
-template void workLambda(const std::vector<char> &v, BufferWriter &writer);
-template void workLambda(const std::vector<short> &v, BufferWriter &writer);
-template void workLambda(const std::vector<int> &v, BufferWriter &writer);
-template void workLambda(const std::vector<long> &v, BufferWriter &writer);
-template void workFunctor(const std::vector<char> &v, BufferWriter &writer);
-template void workFunctor(const std::vector<short> &v, BufferWriter &writer);
-template void workFunctor(const std::vector<int> &v, BufferWriter &writer);
-template void workFunctor(const std::vector<long> &v, BufferWriter &writer);
-template void workFunctor2(const std::vector<char> &v, BufferWriter &writer);
-template void workFunctor2(const std::vector<short> &v, BufferWriter &writer);
-template void workFunctor2(const std::vector<int> &v, BufferWriter &writer);
-template void workFunctor2(const std::vector<long> &v, BufferWriter &writer);
-
-} // namespace search
diff --git a/searchlib/src/tests/util/bufferwriter/work.h b/searchlib/src/tests/util/bufferwriter/work.h
deleted file mode 100644
index 17f381d99eb..00000000000
--- a/searchlib/src/tests/util/bufferwriter/work.h
+++ /dev/null
@@ -1,22 +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 <vector>
-
-namespace search {
-
-class BufferWriter;
-
-template <class T>
-using WorkFunc = void (*)(const std::vector<T> &v, BufferWriter &writer);
-template <class T>
-void work(const std::vector<T> &v, BufferWriter &writer);
-template <class T>
-void workLambda(const std::vector<T> &v, BufferWriter &writer);
-template <class T>
-void workFunctor(const std::vector<T> &v, BufferWriter &writer);
-template <class T>
-void workFunctor2(const std::vector<T> &v, BufferWriter &writer);
-
-} // namespace search
diff --git a/searchlib/src/vespa/searchlib/features/debug_attribute_wait.cpp b/searchlib/src/vespa/searchlib/features/debug_attribute_wait.cpp
index 57c4b3ca9a1..9d43ecfbeb4 100644
--- a/searchlib/src/vespa/searchlib/features/debug_attribute_wait.cpp
+++ b/searchlib/src/vespa/searchlib/features/debug_attribute_wait.cpp
@@ -42,9 +42,9 @@ DebugAttributeWaitExecutor::execute(uint32_t docId)
_buf.fill(*_attribute, docId);
waitTime = _buf[0];
}
- steady_clock::time_point start = steady_clock::now();
+ fastos::StopWatch timer;
fastos::StopWatch::waitAtLeast(microseconds(static_cast<long>(waitTime * 1000000)), _params.busyWait);
- outputs().set_number(0, (1.0e-6 * (steady_clock::now() - start)).count());
+ outputs().set_number(0, timer.elapsed().sec());
}
//-----------------------------------------------------------------------------
diff --git a/searchlib/src/vespa/searchlib/features/debug_wait.cpp b/searchlib/src/vespa/searchlib/features/debug_wait.cpp
index c30a026137c..f0a50ce8742 100644
--- a/searchlib/src/vespa/searchlib/features/debug_wait.cpp
+++ b/searchlib/src/vespa/searchlib/features/debug_wait.cpp
@@ -29,9 +29,9 @@ using namespace std::chrono;
void
DebugWaitExecutor::execute(uint32_t)
{
- steady_clock::time_point start = steady_clock::now();
+ fastos::StopWatch timer;
fastos::StopWatch::waitAtLeast(microseconds(static_cast<long>(_params.waitTime * 1000000)), _params.busyWait);
- outputs().set_number(0, (1.0e-6 * (steady_clock::now() - start)).count());
+ outputs().set_number(0, timer.elapsed().sec());
}
//-----------------------------------------------------------------------------
diff --git a/searchlib/src/vespa/searchlib/features/random_normal_feature.cpp b/searchlib/src/vespa/searchlib/features/random_normal_feature.cpp
index 91ed78fafa3..dd0c67df45c 100644
--- a/searchlib/src/vespa/searchlib/features/random_normal_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/random_normal_feature.cpp
@@ -69,7 +69,7 @@ RandomNormalBlueprint::createExecutor(const fef::IQueryEnvironment &, vespalib::
{
uint64_t seed = _seed;
if (seed == 0) {
- seed = static_cast<uint64_t>(duration_cast<microseconds>(steady_clock::now().time_since_epoch()).count()) ^
+ seed = static_cast<uint64_t>(duration_cast<microseconds>(system_clock::now().time_since_epoch()).count()) ^
reinterpret_cast<uint64_t>(&seed); // results in different seeds in different threads
}
return stash.create<RandomNormalExecutor>(seed, _mean, _stddev);
diff --git a/searchlib/src/vespa/searchlib/features/randomfeature.cpp b/searchlib/src/vespa/searchlib/features/randomfeature.cpp
index db5ccc67a60..18b0cf616d4 100644
--- a/searchlib/src/vespa/searchlib/features/randomfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/randomfeature.cpp
@@ -66,7 +66,7 @@ RandomBlueprint::createExecutor(const fef::IQueryEnvironment &env, vespalib::Sta
{
uint64_t seed = _seed;
if (seed == 0) {
- seed = static_cast<uint64_t>(duration_cast<microseconds>(steady_clock::now().time_since_epoch()).count()) ^
+ seed = static_cast<uint64_t>(duration_cast<microseconds>(system_clock::now().time_since_epoch()).count()) ^
reinterpret_cast<uint64_t>(&seed); // results in different seeds in different threads
}
uint64_t matchSeed = util::strToNum<uint64_t>
diff --git a/searchsummary/src/tests/extractkeywords/extractkeywordstest.cpp b/searchsummary/src/tests/extractkeywords/extractkeywordstest.cpp
index b414671748c..5d1025c6de2 100644
--- a/searchsummary/src/tests/extractkeywords/extractkeywordstest.cpp
+++ b/searchsummary/src/tests/extractkeywords/extractkeywordstest.cpp
@@ -109,7 +109,7 @@ ExtractKeywordsTest::Main()
}
// Print time taken
- double timeTaken = timer.stop().elapsed().ms();
+ double timeTaken = timer.elapsed().ms();
printf("Time taken : %f ms\n", timeTaken);
printf("Number of tests run: %d\n", testCnt);
diff --git a/storage/src/tests/storageserver/documentapiconvertertest.cpp b/storage/src/tests/storageserver/documentapiconvertertest.cpp
index 6ef39ecf858..e8721051262 100644
--- a/storage/src/tests/storageserver/documentapiconvertertest.cpp
+++ b/storage/src/tests/storageserver/documentapiconvertertest.cpp
@@ -203,7 +203,7 @@ TEST_F(DocumentApiConverterTest, create_visitor) {
TEST_F(DocumentApiConverterTest, create_visitor_high_timeout) {
documentapi::CreateVisitorMessage cv("mylib", "myinstance", "control-dest", "data-dest");
- cv.setTimeRemaining(std::chrono::milliseconds(std::numeric_limits<uint32_t>::max() + 1)); // Will be INT_MAX
+ cv.setTimeRemaining(std::chrono::milliseconds(std::numeric_limits<uint32_t>::max() + 1l)); // Will be INT_MAX
auto cmd = toStorageAPI<api::CreateVisitorCommand>(cv);
EXPECT_EQ("mylib", cmd->getLibraryName());
diff --git a/storage/src/vespa/storage/storageserver/documentapiconverter.cpp b/storage/src/vespa/storage/storageserver/documentapiconverter.cpp
index 7318d690004..c6a16de3282 100644
--- a/storage/src/vespa/storage/storageserver/documentapiconverter.cpp
+++ b/storage/src/vespa/storage/storageserver/documentapiconverter.cpp
@@ -141,7 +141,7 @@ DocumentApiConverter::toStorageAPI(documentapi::DocumentMessage& fromMsg)
}
if (toMsg.get() != 0) {
- milliseconds timeout = std::max(milliseconds(INT_MAX), fromMsg.getTimeRemaining());
+ milliseconds timeout = std::min(milliseconds(INT_MAX), fromMsg.getTimeRemaining());
toMsg->setTimeout(timeout.count());
toMsg->setPriority(_priConverter->toStoragePriority(fromMsg.getPriority()));
toMsg->setLoadType(fromMsg.getLoadType());
diff --git a/vespalib/src/tests/btree/iteratespeed.cpp b/vespalib/src/tests/btree/iteratespeed.cpp
index 20aad948f33..82aa9bb5f54 100644
--- a/vespalib/src/tests/btree/iteratespeed.cpp
+++ b/vespalib/src/tests/btree/iteratespeed.cpp
@@ -106,7 +106,7 @@ IterateSpeed::workLoop(int loops, bool enableForward, bool enableBackwards,
[&](int key) { sum += key; } );
}
}
- double used = stopWatch.stop().elapsed().sec();
+ double used = stopWatch.elapsed().sec();
printf("Elapsed time for iterating %ld steps is %8.5f, "
"direction=%s, fanout=%u,%u, sum=%" PRIu64 "\n",
numEntries * numInnerLoops,
diff --git a/vespalib/src/vespa/vespalib/util/time_tracker.h b/vespalib/src/vespa/vespalib/util/time_tracker.h
index 99f88d69110..95dc8feb325 100644
--- a/vespalib/src/vespa/vespalib/util/time_tracker.h
+++ b/vespalib/src/vespa/vespalib/util/time_tracker.h
@@ -16,11 +16,12 @@ private:
struct Task {
vespalib::string name;
fastos::StopWatch task_time;
+ fastos::TimeStamp elapsed;
std::vector<Task> sub_tasks;
- Task(const char *name_in) : name(name_in), task_time() { }
+ Task(const char *name_in) : name(name_in), task_time(), elapsed() { }
~Task();
- void close_task() { task_time.stop(); }
- double ms() const { return (task_time.elapsed().sec() * 1000.0); }
+ void close_task() { elapsed = task_time.elapsed(); }
+ double ms() const { return elapsed.sec()*1000.0; }
};
std::vector<Task> _tasks;