summaryrefslogtreecommitdiffstats
path: root/searchcore
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2017-01-26 11:36:14 +0100
committerGitHub <noreply@github.com>2017-01-26 11:36:14 +0100
commit1ea6768e53a738ea89683c3985617a58334df327 (patch)
treec3fd82fbebe6fcbc6c482532d59b619f24a60c5a /searchcore
parentd724f46259ac6984e51e7cb68b94adab543d4fde (diff)
Revert "Revert "Balder/return timeout information up 4""
Diffstat (limited to 'searchcore')
-rw-r--r--searchcore/src/vespa/searchcore/fdispatch/common/search.h6
-rw-r--r--searchcore/src/vespa/searchcore/fdispatch/search/fnet_search.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/fdispatch/search/querycacheutil.h27
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.cpp1
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.h2
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_master.cpp11
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_master.h13
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp12
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.h2
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp1
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/matcher.cpp38
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/matcher.h1
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/matching_stats.cpp38
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/matching_stats.h36
-rw-r--r--searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.cpp8
-rw-r--r--searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.h1
16 files changed, 133 insertions, 70 deletions
diff --git a/searchcore/src/vespa/searchcore/fdispatch/common/search.h b/searchcore/src/vespa/searchcore/fdispatch/common/search.h
index b69f6ab8537..b823dfdf31a 100644
--- a/searchcore/src/vespa/searchcore/fdispatch/common/search.h
+++ b/searchcore/src/vespa/searchcore/fdispatch/common/search.h
@@ -99,12 +99,16 @@ public:
uint32_t _maxHits;
uint64_t _coverageDocs;
uint64_t _activeDocs;
+ uint64_t _soonActiveDocs;
+ uint32_t _degradeReason;
FastS_SearchInfo()
: _searchOffset(0),
_maxHits(0),
_coverageDocs(0),
- _activeDocs(0)
+ _activeDocs(0),
+ _soonActiveDocs(0),
+ _degradeReason(0)
{
}
};
diff --git a/searchcore/src/vespa/searchcore/fdispatch/search/fnet_search.cpp b/searchcore/src/vespa/searchcore/fdispatch/search/fnet_search.cpp
index 4131533423c..33ad5e2c0ca 100644
--- a/searchcore/src/vespa/searchcore/fdispatch/search/fnet_search.cpp
+++ b/searchcore/src/vespa/searchcore/fdispatch/search/fnet_search.cpp
@@ -856,12 +856,16 @@ FastS_FNET_Search::CheckCoverage()
{
uint64_t covDocs = 0;
uint64_t activeDocs = 0;
+ uint64_t soonActiveDocs = 0;
+ uint32_t degradedReason = 0;
size_t cntNone(0);
for (const FastS_FNET_SearchNode & node : _nodes) {
if (node._qresult != NULL) {
covDocs += node._qresult->_coverageDocs;
activeDocs += node._qresult->_activeDocs;
+ soonActiveDocs += node._qresult->_soonActiveDocs;
+ degradedReason |= node._qresult->_coverageDegradeReason;
} else {
cntNone++;
}
@@ -869,7 +873,7 @@ FastS_FNET_Search::CheckCoverage()
if ((cntNone > 0) && (cntNone != _nodes.size())) {
activeDocs += cntNone * activeDocs/(_nodes.size() - cntNone);
}
- _util.SetCoverage(covDocs, activeDocs);
+ _util.SetCoverage(covDocs, activeDocs, soonActiveDocs, degradedReason);
}
diff --git a/searchcore/src/vespa/searchcore/fdispatch/search/querycacheutil.h b/searchcore/src/vespa/searchcore/fdispatch/search/querycacheutil.h
index c9fe99202dd..1907afc62f5 100644
--- a/searchcore/src/vespa/searchcore/fdispatch/search/querycacheutil.h
+++ b/searchcore/src/vespa/searchcore/fdispatch/search/querycacheutil.h
@@ -81,36 +81,35 @@ public:
ret = 0.0;
return ret;
}
- void SetCoverage(uint64_t coverageDocs,
- uint64_t activeDocs)
+ void SetCoverage(uint64_t coverageDocs, uint64_t activeDocs, uint64_t soonActiveDocs, uint32_t degradeReason)
{
_searchInfo._coverageDocs = coverageDocs;
_searchInfo._activeDocs = activeDocs;
+ _searchInfo._soonActiveDocs = soonActiveDocs;
+ _searchInfo._degradeReason = degradeReason;
}
void SetAlignedHitCount(uint32_t alignedHitCount) {
- if (alignedHitCount > _alignedMaxHits)
+ if (alignedHitCount > _alignedMaxHits) {
alignedHitCount = _alignedMaxHits;
+ }
_alignedHitCount = alignedHitCount;
}
void CalcHitCount(void) {
- if (_alignedHitCount + _alignedSearchOffset >
- _searchInfo._searchOffset)
- _queryResult._hitCount = _alignedHitCount + _alignedSearchOffset -
- _searchInfo._searchOffset;
- else
+ if (_alignedHitCount + _alignedSearchOffset > _searchInfo._searchOffset) {
+ _queryResult._hitCount = _alignedHitCount + _alignedSearchOffset - _searchInfo._searchOffset;
+ } else {
_queryResult._hitCount = 0;
- if (_queryResult._hitCount > _searchInfo._maxHits)
+ }
+ if (_queryResult._hitCount > _searchInfo._maxHits) {
_queryResult._hitCount = _searchInfo._maxHits;
+ }
}
void AllocAlignedHitBuf(void) {
FastS_assert(_alignedHitBuf == NULL);
if (_alignedHitCount != 0) {
- _alignedHitBuf =
- (FastS_hitresult*)malloc(sizeof(FastS_hitresult) *
- _alignedHitCount);
+ _alignedHitBuf = (FastS_hitresult*)malloc(sizeof(FastS_hitresult) * _alignedHitCount);
_hitbuf_needfree = true;
- _queryResult._hitbuf =
- _alignedHitBuf + _searchInfo._searchOffset - _alignedSearchOffset;
+ _queryResult._hitbuf = _alignedHitBuf + _searchInfo._searchOffset - _alignedSearchOffset;
}
}
void AllocSortData(uint32_t sortDataLen)
diff --git a/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.cpp b/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.cpp
index a218407d826..fa617f57f53 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.cpp
@@ -1,6 +1,5 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/fastos/fastos.h>
#include "docid_range_scheduler.h"
namespace proton {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.h b/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.h
index 95bc3bf4fda..bdd0053025e 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/docid_range_scheduler.h
@@ -7,6 +7,8 @@
#include <condition_variable>
#include <atomic>
#include <algorithm>
+#include <vector>
+#include <vespa/fastos/dynamiclibrary.h>
namespace proton {
namespace matching {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
index 08bf2029e18..e5f6f72f39a 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
@@ -1,17 +1,14 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/fastos/fastos.h>
-#include <vespa/log/log.h>
-LOG_SETUP(".proton.matching.match_master");
#include "match_master.h"
-
#include "docid_range_scheduler.h"
#include "match_loop_communicator.h"
#include "match_thread.h"
-#include "matching_stats.h"
-#include <memory>
#include <vespa/searchlib/common/featureset.h>
-#include <vespa/searchlib/common/resultset.h>
+#include <vespa/vespalib/util/thread_bundle.h>
+
+#include <vespa/log/log.h>
+LOG_SETUP(".proton.matching.match_master");
namespace proton {
namespace matching {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.h b/searchcore/src/vespa/searchcore/proton/matching/match_master.h
index b992863e67e..7ca62b7faff 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_master.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_master.h
@@ -2,17 +2,17 @@
#pragma once
-#include <vespa/vespalib/util/clock.h>
-#include <vespa/vespalib/util/thread_bundle.h>
-#include <vespa/searchlib/common/featureset.h>
#include "result_processor.h"
-#include "match_params.h"
#include "matching_stats.h"
+namespace vespalib { class ThreadBundle; }
+namespace search { class FeatureSet; }
+
namespace proton {
namespace matching {
class MatchToolsFactory;
+class MatchParams;
/**
* Handles overall matching and keeps track of match threads.
@@ -23,7 +23,7 @@ private:
MatchingStats _stats;
public:
- const MatchingStats &getStats() const { return _stats; }
+ const MatchingStats & getStats() const { return _stats; }
ResultProcessor::Result::UP match(const MatchParams &params,
vespalib::ThreadBundle &threadBundle,
const MatchToolsFactory &matchToolsFactory,
@@ -31,9 +31,10 @@ public:
uint32_t distributionKey,
uint32_t numSearchPartitions);
- static search::FeatureSet::SP
+ static std::shared_ptr<search::FeatureSet>
getFeatureSet(const MatchToolsFactory &matchToolsFactory,
const std::vector<uint32_t> &docs, bool summaryFeatures);
+ static MatchingStats getStats(MatchMaster && rhs) { return std::move(rhs._stats); }
};
} // namespace proton::matching
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
index fb78f25cc41..6203881a7dd 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
@@ -106,8 +106,7 @@ MatchThread::Context::Context(double rankDropLimit, MatchTools & matchTools, Ran
_hits(hits),
_softDoom(matchTools.getSoftDoom()),
_limiter(matchTools.match_limiter())
-{
-}
+{ }
void
MatchThread::Context::rankHit(uint32_t docId) {
@@ -170,7 +169,7 @@ MatchThread::try_share(DocidRange &docid_range, uint32_t next_docid) {
}
template <typename IteratorT, bool do_rank, bool do_limit, bool do_share_work>
-void
+bool
MatchThread::inner_match_loop(Context & context, IteratorT & search, DocidRange docid_range)
{
search->initRange(docid_range.begin, docid_range.end);
@@ -193,6 +192,7 @@ MatchThread::inner_match_loop(Context & context, IteratorT & search, DocidRange
docId = search->seekNext(docId + 1);
}
}
+ return (docId < docid_range.end);
}
template <typename IteratorT, bool do_rank, bool do_limit, bool do_share_work>
@@ -200,12 +200,13 @@ void
MatchThread::match_loop(MatchTools &matchTools, IteratorT search,
RankProgram &ranking, HitCollector &hits)
{
+ bool softDoomed = false;
Context context(matchParams.rankDropLimit, matchTools, ranking, hits, num_threads);
for (DocidRange docid_range = scheduler.first_range(thread_id);
- !docid_range.empty();
+ !docid_range.empty() && ! softDoomed;
docid_range = scheduler.next_range(thread_id))
{
- inner_match_loop<IteratorT, do_rank, do_limit, do_share_work>(context, search, docid_range);
+ softDoomed = inner_match_loop<IteratorT, do_rank, do_limit, do_share_work>(context, search, docid_range);
}
uint32_t matches = context.matches;
if (do_limit && context.isBelowLimit()) {
@@ -216,6 +217,7 @@ MatchThread::match_loop(MatchTools &matchTools, IteratorT search,
context.limiter().updateDocIdSpaceEstimate(searchedSoFar, 0);
}
thread_stats.docsMatched(matches);
+ thread_stats.softDoomed(softDoomed);
if (do_rank) {
thread_stats.docsRanked(matches);
}
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.h b/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
index 014d935ba00..a9be0b0c7eb 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
@@ -78,7 +78,7 @@ private:
bool try_share(DocidRange &docid_range, uint32_t next_docid) __attribute__((noinline));
template <typename IteratorT, bool do_rank, bool do_limit, bool do_share_work>
- void inner_match_loop(Context & params, IteratorT & search, DocidRange docid_range) __attribute__((noinline));
+ bool inner_match_loop(Context & params, IteratorT & search, DocidRange docid_range) __attribute__((noinline));
template <typename IteratorT, bool do_rank, bool do_limit, bool do_share_work>
void match_loop(MatchTools &matchTools, IteratorT search, RankProgram &ranking, HitCollector &hits) __attribute__((noinline));
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
index bad8aa915c1..937e913d13b 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
@@ -123,6 +123,7 @@ MatchToolsFactory(QueryLimiter & queryLimiter,
_rankSetup(rankSetup),
_featureOverrides(featureOverrides)
{
+ LOG(info, "softtimeout=%1.3f harddomm=%1.3f", softDoom.left().sec(), hardDoom.left().sec());
_valid = _query.buildTree(queryStack, location, viewResolver, indexEnv);
if (_valid) {
_query.extractTerms(_queryEnv.terms());
diff --git a/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp b/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
index 94169b662ef..6f4596b60d3 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
@@ -1,6 +1,5 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/fastos/fastos.h>
#include "isearchcontext.h"
#include "match_master.h"
#include "match_context.h"
@@ -157,11 +156,15 @@ MatchingStats
Matcher::getStats()
{
vespalib::LockGuard guard(_statsLock);
- MatchingStats stats = _stats;
- _stats = MatchingStats();
+ MatchingStats stats = std::move(_stats);
+ _stats = std::move(MatchingStats());
+ _stats.softDoomFactor(stats.softDoomFactor());
return stats;
}
+using search::fef::indexproperties::softtimeout::Enabled;
+using search::fef::indexproperties::softtimeout::Factor;
+
std::unique_ptr<MatchToolsFactory>
Matcher::create_match_tools_factory(const search::engine::Request &request,
ISearchContext &searchContext,
@@ -169,7 +172,14 @@ Matcher::create_match_tools_factory(const search::engine::Request &request,
const IDocumentMetaStore &metaStore,
const Properties &feature_overrides) const
{
- uint64_t safeLeft = request.getTimeLeft() * computeFirstPhase2RestRatio()*0.95;
+ const Properties & rankProperties = request.propertiesMap.rankProperties();
+ bool softTimeoutEnabled = Enabled::lookup(rankProperties, _rankSetup->getSoftTimeoutEnabled());
+ double factor = 0.95;
+ if (softTimeoutEnabled) {
+ factor = Factor::lookup(rankProperties, _stats.softDoomFactor());
+ LOG(info, "Enabling soft-timeout computed factor=%1.3f, used factor=%1.3f", _stats.softDoomFactor(), factor);
+ }
+ uint64_t safeLeft = request.getTimeLeft() * factor;
fastos::TimeStamp safeDoom(fastos::ClockSystem::now() + safeLeft);
return std::make_unique<MatchToolsFactory>(_queryLimiter, vespalib::Doom(_clock, safeDoom),
vespalib::Doom(_clock, request.getTimeOfDoom()), searchContext,
@@ -200,11 +210,6 @@ size_t Matcher::computeNumThreadsPerSearch(Blueprint::HitEstimate hits) const {
return threads;
}
-double
-Matcher::computeFirstPhase2RestRatio() const {
- return 1.0;
-}
-
SearchReply::UP
Matcher::match(const SearchRequest &request,
vespalib::ThreadBundle &threadBundle,
@@ -257,7 +262,7 @@ Matcher::match(const SearchRequest &request,
LimitedThreadBundleWrapper limitedThreadBundle(threadBundle, numThreadsPerSearch);
MatchMaster master;
ResultProcessor::Result::UP result = master.match(params, limitedThreadBundle, *mtf, rp, _distributionKey, _rankSetup->getNumSearchPartitions());
- my_stats = master.getStats();
+ my_stats = MatchMaster::getStats(std::move(master));
size_t estimate = std::min(static_cast<size_t>(metaStore.getCommittedDocIdLimit()), mtf->match_limiter().getDocIdSpaceEstimate());
if (shouldCacheSearchSession && ((result->_numFs4Hits != 0) || shouldCacheGroupingSession)) {
SearchSession::SP session = std::make_shared<SearchSession>(sessionId, request.getTimeOfDoom(), std::move(mtf), std::move(owned_objects));
@@ -265,7 +270,14 @@ Matcher::match(const SearchRequest &request,
sessionMgr.insert(std::move(session));
}
reply = std::move(result->_reply);
+ if (mtf->match_limiter().was_limited()) {
+ reply->coverage.degradeMatchPhase();
+ }
+ if (my_stats.softDoomed()) {
+ reply->coverage.degradeTimeout();
+ }
reply->coverage.setActive(metaStore.getNumActiveLids());
+ reply->coverage.setSoonActive(metaStore.getNumActiveLids()); //TODO this should be calculated with ClusterState calculator.
reply->coverage.setCovered(std::min(static_cast<size_t>(metaStore.getNumActiveLids()),
(estimate * metaStore.getNumActiveLids())/metaStore.getCommittedDocIdLimit()));
LOG(debug, "numThreadsPerSearch = %d. Configured = %d, estimated hits=%d, totalHits=%ld",
@@ -274,8 +286,14 @@ Matcher::match(const SearchRequest &request,
total_matching_time.stop();
my_stats.queryCollateralTime(total_matching_time.elapsed().sec() - my_stats.queryLatencyAvg());
{
+ fastos::TimeStamp softLimit = uint64_t((1.0 - _rankSetup->getSoftTimeoutTailCost()) * request.getTimeout());
+ fastos::TimeStamp duration = request.getTimeUsed();
vespalib::LockGuard guard(_statsLock);
_stats.add(my_stats);
+ if (my_stats.softDoomed()) {
+ LOG(info, "Triggered softtimeout limit=%1.3f and duration=%1.3f", softLimit.sec(), duration.sec());
+ _stats.updatesoftDoomFactor(request.getTimeout(), softLimit, duration);
+ }
}
return reply;
}
diff --git a/searchcore/src/vespa/searchcore/proton/matching/matcher.h b/searchcore/src/vespa/searchcore/proton/matching/matcher.h
index 569ca857737..d8128f90f9b 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/matcher.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/matcher.h
@@ -72,7 +72,6 @@ private:
std::unique_ptr<search::grouping::GroupingSession> gs);
size_t computeNumThreadsPerSearch(search::queryeval::Blueprint::HitEstimate hits) const;
- double computeFirstPhase2RestRatio() const;
public:
/**
* Convenience typedefs.
diff --git a/searchcore/src/vespa/searchcore/proton/matching/matching_stats.cpp b/searchcore/src/vespa/searchcore/proton/matching/matching_stats.cpp
index 79227cf9d88..0345a1affc1 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/matching_stats.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/matching_stats.cpp
@@ -1,12 +1,6 @@
// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/fastos/fastos.h>
#include "matching_stats.h"
-#include <vespa/vespalib/util/stringfmt.h>
-#include <vespa/vespalib/util/exceptions.h>
-
-using vespalib::make_string;
-using vespalib::IllegalStateException;
namespace proton {
namespace matching {
@@ -22,6 +16,24 @@ MatchingStats::Partition &get_writable_partition(std::vector<MatchingStats::Part
} // namespace proton::matching::<unnamed>
+MatchingStats::MatchingStats()
+ : _queries(0),
+ _limited_queries(0),
+ _docsMatched(0),
+ _docsRanked(0),
+ _docsReRanked(0),
+ _softDoomed(0),
+ _softDoomFactor(0.5),
+ _queryCollateralTime(),
+ _queryLatency(),
+ _matchTime(),
+ _groupingTime(),
+ _rerankTime(),
+ _partitions()
+{ }
+
+MatchingStats::~MatchingStats() { }
+
MatchingStats &
MatchingStats::merge_partition(const Partition &partition, size_t id)
{
@@ -30,6 +42,9 @@ MatchingStats::merge_partition(const Partition &partition, size_t id)
_docsMatched += partition.docsMatched();
_docsRanked += partition.docsRanked();
_docsReRanked += partition.docsReRanked();
+ if (partition.softDoomed()) {
+ _softDoomed = 1;
+ }
return *this;
}
@@ -44,6 +59,7 @@ MatchingStats::add(const MatchingStats &rhs)
_docsMatched += rhs._docsMatched;
_docsRanked += rhs._docsRanked;
_docsReRanked += rhs._docsReRanked;
+ _softDoomed += rhs.softDoomed();
_queryCollateralTime.add(rhs._queryCollateralTime);
_queryLatency.add(rhs._queryLatency);
@@ -56,5 +72,15 @@ MatchingStats::add(const MatchingStats &rhs)
return *this;
}
+MatchingStats &
+MatchingStats::updatesoftDoomFactor(double hardLimit, double softLimit, double duration) {
+ if (duration < softLimit) {
+ _softDoomFactor += 0.01*(softLimit - duration)/hardLimit;
+ } else {
+ _softDoomFactor += 0.02*(softLimit - duration)/hardLimit;
+ }
+ return *this;
+}
+
}
} // namespace searchcore
diff --git a/searchcore/src/vespa/searchcore/proton/matching/matching_stats.h b/searchcore/src/vespa/searchcore/proton/matching/matching_stats.h
index 7cc8f0d3ef2..e3f67355e35 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/matching_stats.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/matching_stats.h
@@ -2,6 +2,9 @@
#pragma once
+#include <vector>
+#include <cstddef>
+
namespace proton {
namespace matching {
@@ -38,6 +41,7 @@ public:
size_t _docsMatched;
size_t _docsRanked;
size_t _docsReRanked;
+ size_t _softDoomed;
Avg _active_time;
Avg _wait_time;
public:
@@ -45,8 +49,9 @@ public:
: _docsMatched(0),
_docsRanked(0),
_docsReRanked(0),
+ _softDoomed(0),
_active_time(),
- _wait_time() {}
+ _wait_time() { }
Partition &docsMatched(size_t value) { _docsMatched = value; return *this; }
size_t docsMatched() const { return _docsMatched; }
@@ -54,6 +59,8 @@ public:
size_t docsRanked() const { return _docsRanked; }
Partition &docsReRanked(size_t value) { _docsReRanked = value; return *this; }
size_t docsReRanked() const { return _docsReRanked; }
+ Partition &softDoomed(bool v) { _softDoomed += v ? 1 : 0; return *this; }
+ size_t softDoomed() const { return _softDoomed; }
Partition &active_time(double time_s) { _active_time.set(time_s); return *this; }
double active_time_avg() const { return _active_time.avg(); }
@@ -66,6 +73,7 @@ public:
_docsMatched += rhs._docsMatched;
_docsRanked += rhs._docsRanked;
_docsReRanked += rhs._docsReRanked;
+ _softDoomed += rhs._softDoomed;
_active_time.add(rhs._active_time);
_wait_time.add(rhs._wait_time);
@@ -79,6 +87,8 @@ private:
size_t _docsMatched;
size_t _docsRanked;
size_t _docsReRanked;
+ size_t _softDoomed;
+ double _softDoomFactor;
Avg _queryCollateralTime;
Avg _queryLatency;
Avg _matchTime;
@@ -87,18 +97,12 @@ private:
std::vector<Partition> _partitions;
public:
- MatchingStats()
- : _queries(0),
- _limited_queries(0),
- _docsMatched(0),
- _docsRanked(0),
- _docsReRanked(0),
- _queryCollateralTime(),
- _queryLatency(),
- _matchTime(),
- _groupingTime(),
- _rerankTime(),
- _partitions() {}
+ MatchingStats(const MatchingStats &) = delete;
+ MatchingStats & operator = (const MatchingStats &) = delete;
+ MatchingStats(MatchingStats &&) = default;
+ MatchingStats & operator = (MatchingStats &&) = default;
+ MatchingStats();
+ ~MatchingStats();
MatchingStats &queries(size_t value) { _queries = value; return *this; }
size_t queries() const { return _queries; }
@@ -115,6 +119,12 @@ public:
MatchingStats &docsReRanked(size_t value) { _docsReRanked = value; return *this; }
size_t docsReRanked() const { return _docsReRanked; }
+ MatchingStats &softDoomed(size_t value) { _softDoomed = value; return *this; }
+ size_t softDoomed() const { return _softDoomed; }
+ MatchingStats &softDoomFactor(double value) { _softDoomFactor = value; return *this; }
+ double softDoomFactor() const { return _softDoomFactor; }
+ MatchingStats &updatesoftDoomFactor(double hardLimit, double softLimit, double duration);
+
MatchingStats &queryCollateralTime(double time_s) { _queryCollateralTime.set(time_s); return *this; }
double queryCollateralTimeAvg() const { return _queryCollateralTime.avg(); }
size_t queryCollateralTimeCount() const { return _queryCollateralTime.count(); }
diff --git a/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.cpp b/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.cpp
index a82535c9e97..1cc20a95315 100644
--- a/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.cpp
+++ b/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.cpp
@@ -35,11 +35,10 @@ LegacyDocumentDBMetrics::MatchingMetrics::update(const MatchingStats &stats)
docsMatched.inc(stats.docsMatched());
docsRanked.inc(stats.docsRanked());
docsReRanked.inc(stats.docsReRanked());
+ softDoomFactor.set(stats.softDoomFactor());
queries.inc(stats.queries());
- queryCollateralTime.addValueBatch(stats.queryCollateralTimeAvg(),
- stats.queryCollateralTimeCount());
- queryLatency.addValueBatch(stats.queryLatencyAvg(),
- stats.queryLatencyCount());
+ queryCollateralTime.addValueBatch(stats.queryCollateralTimeAvg(), stats.queryCollateralTimeCount());
+ queryLatency.addValueBatch(stats.queryLatencyAvg(), stats.queryLatencyCount());
}
LegacyDocumentDBMetrics::MatchingMetrics::MatchingMetrics(MetricSet *parent)
@@ -48,6 +47,7 @@ LegacyDocumentDBMetrics::MatchingMetrics::MatchingMetrics(MetricSet *parent)
docsRanked("docsranked", "", "Number of documents ranked (first phase)", this),
docsReRanked("docsreranked", "", "Number of documents re-ranked (second phase)", this),
queries("queries", "", "Number of queries executed", this),
+ softDoomFactor("softdoomfactor", "", "Factor used to compute soft-timeout", this),
queryCollateralTime("querycollateraltime", "", "Average time spent setting up and tearing down queries", this),
queryLatency("querylatency", "", "Average latency when matching a query", this)
{ }
diff --git a/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.h b/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.h
index fe55d7e5719..16cc176e377 100644
--- a/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.h
+++ b/searchcore/src/vespa/searchcore/proton/metrics/legacy_documentdb_metrics.h
@@ -45,6 +45,7 @@ struct LegacyDocumentDBMetrics : metrics::MetricSet
metrics::LongCountMetric docsRanked;
metrics::LongCountMetric docsReRanked;
metrics::LongCountMetric queries;
+ metrics::DoubleValueMetric softDoomFactor;
metrics::DoubleAverageMetric queryCollateralTime;
metrics::DoubleAverageMetric queryLatency;