aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2019-03-08 15:37:32 +0000
committerHenning Baldersheim <balder@yahoo-inc.com>2019-03-10 20:34:33 +0000
commitf20b2388ca2f6fee2ee21fb390c3f7bc5d3c41c3 (patch)
tree6781fc27910a4014a2e042cc3a33e60de8912e20
parentb04e96331e800c0d8e32125f02b55fd37e1510d1 (diff)
Trace match thread and iterator tree too.
Timing on 4 and 5, blueprint on 6, iterator on 7 Trace the limiter too. Move level to trace.
-rw-r--r--searchcore/src/tests/proton/matching/match_phase_limiter/match_phase_limiter_test.cpp16
-rw-r--r--searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_master.cpp22
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_master.h4
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.cpp21
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.h7
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp31
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.h13
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/matcher.cpp21
-rw-r--r--searchlib/src/vespa/searchlib/engine/packetconverter.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/engine/request.cpp3
-rw-r--r--searchlib/src/vespa/searchlib/engine/request.h6
-rw-r--r--searchlib/src/vespa/searchlib/engine/trace.cpp14
-rw-r--r--searchlib/src/vespa/searchlib/engine/trace.h13
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/searchiterator.cpp17
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/searchiterator.h15
16 files changed, 157 insertions, 52 deletions
diff --git a/searchcore/src/tests/proton/matching/match_phase_limiter/match_phase_limiter_test.cpp b/searchcore/src/tests/proton/matching/match_phase_limiter/match_phase_limiter_test.cpp
index d1e7adfedb8..90cdc800b73 100644
--- a/searchcore/src/tests/proton/matching/match_phase_limiter/match_phase_limiter_test.cpp
+++ b/searchcore/src/tests/proton/matching/match_phase_limiter/match_phase_limiter_test.cpp
@@ -197,7 +197,7 @@ TEST("require that no limiter has no behavior") {
MaybeMatchPhaseLimiter &limiter = no_limiter;
EXPECT_FALSE(limiter.is_enabled());
EXPECT_EQUAL(0u, limiter.sample_hits_per_thread(1));
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 1.0, 100000000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 1.0, 100000000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 9000);
EXPECT_EQUAL(std::numeric_limits<size_t>::max(), limiter.getDocIdSpaceEstimate());
MockSearch *ms = dynamic_cast<MockSearch*>(search.get());
@@ -215,8 +215,7 @@ TEST("require that the match phase limiter may chose not to limit the query") {
MaybeMatchPhaseLimiter &limiter = yes_limiter;
EXPECT_TRUE(limiter.is_enabled());
EXPECT_EQUAL(20u, limiter.sample_hits_per_thread(10));
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")),
- 0.005, 100000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.005, 100000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 9000);
EXPECT_EQUAL(10000u, limiter.getDocIdSpaceEstimate());
MockSearch *ms = dynamic_cast<MockSearch*>(search.get());
@@ -244,7 +243,7 @@ struct MaxFilterCoverageLimiterFixture {
TEST_F("require that the match phase limiter may chose not to limit the query when considering max-filter-coverage", MaxFilterCoverageLimiterFixture) {
MatchPhaseLimiter::UP limiterUP = f.getMaxFilterCoverageLimiter();
MaybeMatchPhaseLimiter & limiter = *limiterUP;
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.10, 1900000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.10, 1900000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 1899000);
EXPECT_EQUAL(1900000u, limiter.getDocIdSpaceEstimate());
MockSearch *ms = dynamic_cast<MockSearch *>(search.get());
@@ -256,7 +255,7 @@ TEST_F("require that the match phase limiter may chose not to limit the query wh
TEST_F("require that the match phase limiter may chose to limit the query even when considering max-filter-coverage", MaxFilterCoverageLimiterFixture) {
MatchPhaseLimiter::UP limiterUP = f.getMaxFilterCoverageLimiter();
MaybeMatchPhaseLimiter & limiter = *limiterUP;
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.10, 2100000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.10, 2100000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 2099000);
EXPECT_EQUAL(159684u, limiter.getDocIdSpaceEstimate());
LimitedSearch *strict_and = dynamic_cast<LimitedSearch*>(search.get());
@@ -281,8 +280,7 @@ TEST("require that the match phase limiter is able to pre-limit the query") {
MaybeMatchPhaseLimiter &limiter = yes_limiter;
EXPECT_TRUE(limiter.is_enabled());
EXPECT_EQUAL(12u, limiter.sample_hits_per_thread(10));
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")),
- 0.1, 100000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.1, 100000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 9000);
EXPECT_EQUAL(1680u, limiter.getDocIdSpaceEstimate());
LimitedSearch *strict_and = dynamic_cast<LimitedSearch*>(search.get());
@@ -313,7 +311,7 @@ TEST("require that the match phase limiter is able to post-limit the query") {
MaybeMatchPhaseLimiter &limiter = yes_limiter;
EXPECT_TRUE(limiter.is_enabled());
EXPECT_EQUAL(30u, limiter.sample_hits_per_thread(10));
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.1, 100000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.1, 100000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 9000);
EXPECT_EQUAL(1680u, limiter.getDocIdSpaceEstimate());
LimitedSearch *strict_and = dynamic_cast<LimitedSearch*>(search.get());
@@ -343,7 +341,7 @@ void verifyDiversity(AttributeLimiter::DiversityCutoffStrategy strategy)
DegradationParams("limiter_attribute", 500, true, 1.0, 0.2, 1.0),
DiversityParams("category", 10, 13.1, strategy));
MaybeMatchPhaseLimiter &limiter = yes_limiter;
- SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.1, 100000);
+ SearchIterator::UP search = limiter.maybe_limit(prepare(new MockSearch("search")), 0.1, 100000, nullptr);
limiter.updateDocIdSpaceEstimate(1000, 9000);
EXPECT_EQUAL(1680u, limiter.getDocIdSpaceEstimate());
LimitedSearch *strict_and = dynamic_cast<LimitedSearch*>(search.get());
diff --git a/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp b/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp
index 3a28eeb4dfd..e8fb05c5589 100644
--- a/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matchengine/matchengine.cpp
@@ -140,7 +140,7 @@ MatchEngine::performSearch(search::engine::SearchRequest::Source req,
}
ret->request = req.release();
ret->setDistributionKey(_distributionKey);
- if (ret->request->getTraceLevel() > 0) {
+ if (ret->request->trace().getLevel() > 0) {
ret->request->trace().getRoot().setLong("distribution-key", _distributionKey);
search::fef::Properties & trace = ret->propertiesMap.lookupCreate("trace");
vespalib::SmartBuffer output(4096);
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
index 4d49e9b5d1b..aedc14ef668 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
@@ -5,8 +5,12 @@
#include "match_loop_communicator.h"
#include "match_thread.h"
#include <vespa/searchlib/attribute/attribute_operation.h>
+#include <vespa/searchlib/engine/request.h>
#include <vespa/searchlib/common/featureset.h>
#include <vespa/vespalib/util/thread_bundle.h>
+#include <vespa/vespalib/data/slime/inserter.h>
+#include <vespa/vespalib/data/slime/inject.h>
+#include <vespa/vespalib/data/slime/cursor.h>
#include <vespa/log/log.h>
LOG_SETUP(".proton.matching.match_master");
@@ -54,7 +58,8 @@ createScheduler(uint32_t numThreads, uint32_t numSearchPartitions, uint32_t numD
} // namespace proton::matching::<unnamed>
ResultProcessor::Result::UP
-MatchMaster::match(const MatchParams &params,
+MatchMaster::match(const search::engine::Request & request,
+ const MatchParams &params,
vespalib::ThreadBundle &threadBundle,
const MatchToolsFactory &mtf,
ResultProcessor &resultProcessor,
@@ -75,7 +80,8 @@ MatchMaster::match(const MatchParams &params,
? static_cast<IMatchLoopCommunicator&>(timedCommunicator)
: static_cast<IMatchLoopCommunicator&>(communicator);
threadState.emplace_back(std::make_unique<MatchThread>(i, threadBundle.size(), params, mtf, com, *scheduler,
- resultProcessor, mergeDirector, distributionKey));
+ resultProcessor, mergeDirector, distributionKey,
+ request.getRelativeTime(), request.trace().getLevel()));
targets.push_back(threadState.back().get());
}
resultProcessor.prepareThreadContextCreation(threadBundle.size());
@@ -85,9 +91,17 @@ MatchMaster::match(const MatchParams &params,
double query_time_s = query_latency_time.elapsed().sec();
double rerank_time_s = timedCommunicator.rerank_time.elapsed().sec();
double match_time_s = 0.0;
+ std::unique_ptr<vespalib::slime::Inserter> inserter;
+ if (request.trace().shouldTrace(4)) {
+ inserter = std::make_unique<vespalib::slime::ArrayInserter>(request.trace().createCursor("match_threads").setArray("threads"));
+ }
for (size_t i = 0; i < threadState.size(); ++i) {
- match_time_s = std::max(match_time_s, threadState[i]->get_match_time());
- _stats.merge_partition(threadState[i]->get_thread_stats(), i);
+ const MatchThread & matchThread = *threadState[i];
+ match_time_s = std::max(match_time_s, matchThread.get_match_time());
+ _stats.merge_partition(matchThread.get_thread_stats(), i);
+ if (inserter) {
+ vespalib::slime::inject(matchThread.getTrace().getRoot(), *inserter);
+ }
}
_stats.queryLatency(query_time_s);
_stats.matchTime(match_time_s - rerank_time_s);
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.h b/searchcore/src/vespa/searchcore/proton/matching/match_master.h
index 5de7fc144ce..8e28d0350e4 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_master.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_master.h
@@ -7,6 +7,7 @@
namespace vespalib { struct ThreadBundle; }
namespace search { class FeatureSet; }
+namespace search::engine { class Request; }
namespace proton::matching {
@@ -23,7 +24,8 @@ private:
public:
const MatchingStats & getStats() const { return _stats; }
- ResultProcessor::Result::UP match(const MatchParams &params,
+ ResultProcessor::Result::UP match(const search::engine::Request &request,
+ const MatchParams &params,
vespalib::ThreadBundle &threadBundle,
const MatchToolsFactory &mtf,
ResultProcessor &resultProcessor,
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.cpp
index 5e965084a2d..691782523cd 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.cpp
@@ -2,6 +2,7 @@
#include "match_phase_limiter.h"
#include <vespa/searchlib/queryeval/andsearchstrict.h>
+#include <vespa/vespalib/data/slime/cursor.h>
#include <vespa/log/log.h>
LOG_SETUP(".proton.matching.match_phase_limiter");
@@ -89,7 +90,7 @@ do_limit(AttributeLimiter &limiter_factory, SearchIterator::UP search,
SearchIterator::UP limiter = limiter_factory.create_search(wanted_num_docs, max_group_size, PRE_FILTER);
limiter = search->andWith(std::move(limiter), wanted_num_docs);
if (limiter) {
- search.reset(new LimitedSearchT<PRE_FILTER>(std::move(limiter), std::move(search)));
+ search = std::make_unique<LimitedSearchT<PRE_FILTER>>(std::move(limiter), std::move(search));
}
search->initRange(current_id + 1, end_id);
return search;
@@ -98,12 +99,21 @@ do_limit(AttributeLimiter &limiter_factory, SearchIterator::UP search,
} // namespace proton::matching::<unnamed>
SearchIterator::UP
-MatchPhaseLimiter::maybe_limit(SearchIterator::UP search, double match_freq, size_t num_docs)
+MatchPhaseLimiter::maybe_limit(SearchIterator::UP search, double match_freq, size_t num_docs, Cursor * trace)
{
size_t wanted_num_docs = _calculator.wanted_num_docs(match_freq);
size_t max_filter_docs = static_cast<size_t>(num_docs * _maxFilterCoverage);
size_t upper_limited_corpus_size = std::min(num_docs, max_filter_docs);
+ if (trace) {
+ trace->setDouble("hit_rate", match_freq);
+ trace->setLong("num_docs", num_docs);
+ trace->setLong("max_filter_docs", max_filter_docs);
+ trace->setLong("wanted_docs", wanted_num_docs);
+ }
if (upper_limited_corpus_size <= wanted_num_docs) {
+ if (trace) {
+ trace->setString("reason", "No need to limit");
+ }
LOG(debug, "Will not limit ! maybe_limit(hit_rate=%g, num_docs=%ld, max_filter_docs=%ld) = wanted_num_docs=%ld",
match_freq, num_docs, max_filter_docs, wanted_num_docs);
return search;
@@ -113,6 +123,13 @@ MatchPhaseLimiter::maybe_limit(SearchIterator::UP search, double match_freq, siz
size_t total_query_hits = _calculator.estimated_hits(match_freq, num_docs);
size_t max_group_size = _calculator.max_group_size(wanted_num_docs);
bool use_pre_filter = (wanted_num_docs < (total_query_hits * _postFilterMultiplier));
+ if (trace) {
+ trace->setString("reason", use_pre_filter ? "Will limit with prefix filter" : "Will limit with postfix filter");
+ trace->setLong("max_group_size", max_group_size);
+ trace->setLong("current_docid", current_id);
+ trace->setLong("end_docid", end_id);
+ trace->setLong("total_query_hits", total_query_hits);
+ }
LOG(debug, "Will do %s filter : maybe_limit(hit_rate=%g, num_docs=%zu, max_filter_docs=%ld) = wanted_num_docs=%zu,"
" max_group_size=%zu, current_docid=%u, end_docid=%u, total_query_hits=%ld",
use_pre_filter ? "pre" : "post", match_freq, num_docs, max_filter_docs, wanted_num_docs,
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.h b/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.h
index b39b6695b7f..46973092a05 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_phase_limiter.h
@@ -43,12 +43,13 @@ private:
* limit the number of matches.
**/
struct MaybeMatchPhaseLimiter {
+ using Cursor = vespalib::slime::Cursor;
typedef search::queryeval::SearchIterator SearchIterator;
typedef std::unique_ptr<MaybeMatchPhaseLimiter> UP;
virtual bool is_enabled() const = 0;
virtual bool was_limited() const = 0;
virtual size_t sample_hits_per_thread(size_t num_threads) const = 0;
- virtual SearchIterator::UP maybe_limit(SearchIterator::UP search, double match_freq, size_t num_docs) = 0;
+ virtual SearchIterator::UP maybe_limit(SearchIterator::UP search, double match_freq, size_t num_docs, Cursor * trace) = 0;
virtual void updateDocIdSpaceEstimate(size_t searchedDocIdSpace, size_t remainingDocIdSpace) = 0;
virtual size_t getDocIdSpaceEstimate() const = 0;
virtual ~MaybeMatchPhaseLimiter() {}
@@ -61,7 +62,7 @@ struct NoMatchPhaseLimiter : MaybeMatchPhaseLimiter {
bool is_enabled() const override { return false; }
bool was_limited() const override { return false; }
size_t sample_hits_per_thread(size_t) const override { return 0; }
- SearchIterator::UP maybe_limit(SearchIterator::UP search, double, size_t) override {
+ SearchIterator::UP maybe_limit(SearchIterator::UP search, double, size_t, Cursor *) override {
return search;
}
void updateDocIdSpaceEstimate(size_t, size_t) override { }
@@ -144,7 +145,7 @@ public:
size_t sample_hits_per_thread(size_t num_threads) const override {
return _calculator.sample_hits_per_thread(num_threads);
}
- SearchIterator::UP maybe_limit(SearchIterator::UP search, double match_freq, size_t num_docs) override;
+ SearchIterator::UP maybe_limit(SearchIterator::UP search, double match_freq, size_t num_docs, Cursor * trace) override;
void updateDocIdSpaceEstimate(size_t searchedDocIdSpace, size_t remainingDocIdSpace) override;
size_t getDocIdSpaceEstimate() const override;
};
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
index 2e92f2d63c3..6c2420c8745 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
@@ -2,6 +2,7 @@
#include "match_thread.h"
#include "document_scorer.h"
+#include <vespa/searchlib/engine/trace.h>
#include <vespa/searchlib/attribute/attribute_operation.h>
#include <vespa/searchcommon/attribute/i_attribute_functor.h>
#include <vespa/searchcore/grouping/groupingmanager.h>
@@ -13,6 +14,8 @@
#include <vespa/searchlib/queryeval/andnotsearch.h>
#include <vespa/vespalib/util/closure.h>
#include <vespa/vespalib/util/thread_bundle.h>
+#include <vespa/vespalib/data/slime/cursor.h>
+#include <vespa/vespalib/data/slime/inserter.h>
#include <vespa/log/log.h>
LOG_SETUP(".proton.matching.match_thread");
@@ -117,14 +120,19 @@ MatchThread::maybe_limit(MatchTools &tools, uint32_t matches, uint32_t docId, ui
const size_t searchedSoFar = (scheduler.total_size(thread_id) - local_todo);
double match_freq = estimate_match_frequency(matches, searchedSoFar);
const size_t global_todo = scheduler.unassigned_size();
+ vespalib::slime::Cursor * traceCursor = trace->shouldTrace(5) ? & trace->createCursor("maybe_limit") : nullptr;
{
auto search = tools.borrow_search();
- search = tools.match_limiter().maybe_limit(std::move(search), match_freq, matchParams.numDocs);
+ search = tools.match_limiter().maybe_limit(std::move(search), match_freq, matchParams.numDocs, traceCursor);
tools.give_back_search(std::move(search));
if (tools.match_limiter().was_limited()) {
tools.tag_search_as_changed();
}
}
+ if (isFirstThread() && trace->shouldTrace(6)) {
+ vespalib::slime::ObjectInserter inserter(trace->createCursor("limited"), "query");
+ tools.search().asSlime(inserter);
+ }
size_t left = local_todo + (global_todo / num_threads);
tools.match_limiter().updateDocIdSpaceEstimate(searchedSoFar, left);
LOG(debug, "Limit=%d has been reached at docid=%d which is after %zu docs.",
@@ -267,17 +275,24 @@ MatchThread::findMatches(MatchTools &tools)
tools.give_back_search(search::queryeval::MultiBitVectorIteratorBase::optimize(tools.borrow_search()));
if (isFirstThread()) {
LOG(debug, "SearchIterator after MultiBitVectorIteratorBase::optimize(): %s", tools.search().asString().c_str());
+ if (trace->shouldTrace(7)) {
+ vespalib::slime::ObjectInserter inserter(trace->createCursor("iterator"), "optimized");
+ tools.search().asSlime(inserter);
+ }
}
HitCollector hits(matchParams.numDocs, matchParams.arraySize);
+ trace->addEvent(4, "Start match and first phase rank");
match_loop_helper(tools, hits);
if (tools.has_second_phase_rank()) {
{ // 2nd phase ranking
+ trace->addEvent(4, "Start second phase rerank");
tools.setup_second_phase();
DocidRange docid_range = scheduler.total_span(thread_id);
tools.search().initRange(docid_range.begin, docid_range.end);
auto sorted_hit_seq = matchToolsFactory.should_diversify()
? hits.getSortedHitSequence(matchParams.arraySize)
: hits.getSortedHitSequence(matchParams.heapSize);
+ trace->addEvent(5, "Synchronize before second phase rerank");
WaitTimer select_best_timer(wait_time_s);
auto kept_hits = communicator.selectBest(sorted_hit_seq);
select_best_timer.done();
@@ -292,6 +307,7 @@ MatchThread::findMatches(MatchTools &tools)
thread_stats.docsReRanked(reRanked);
}
{ // rank scaling
+ trace->addEvent(5, "Synchronize before rank scaling");
auto my_ranges = hits.getRanges();
WaitTimer range_cover_timer(wait_time_s);
auto ranges = communicator.rangeCover(my_ranges);
@@ -299,6 +315,7 @@ MatchThread::findMatches(MatchTools &tools)
hits.setRanges(ranges);
}
}
+ trace->addEvent(4, "Create result set");
return hits.getResultSet(fallback_rank_value());
}
@@ -373,7 +390,9 @@ MatchThread::MatchThread(size_t thread_id_in,
DocidRangeScheduler &sched,
ResultProcessor &rp,
vespalib::DualMergeDirector &md,
- uint32_t distributionKey) :
+ uint32_t distributionKey,
+ const RelativeTime & relativeTime,
+ uint32_t traceLevel) :
thread_id(thread_id_in),
num_threads(num_threads_in),
matchParams(mp),
@@ -389,7 +408,8 @@ MatchThread::MatchThread(size_t thread_id_in,
total_time_s(0.0),
match_time_s(0.0),
wait_time_s(0.0),
- match_with_ranking(mtf.has_first_phase_rank() && mp.save_rank_scores())
+ match_with_ranking(mtf.has_first_phase_rank() && mp.save_rank_scores()),
+ trace(std::make_unique<Trace>(relativeTime, traceLevel))
{
}
@@ -400,12 +420,14 @@ MatchThread::run()
fastos::StopWatch match_time;
total_time.start();
match_time.start();
+ 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);
{
+ trace->addEvent(5, "Wait for result processing token");
WaitTimer get_token_timer(wait_time_s);
QueryLimiter::Token::UP processToken(
matchTools->getQueryLimiter().getToken(matchTools->getHardDoom(),
@@ -414,12 +436,15 @@ MatchThread::run()
resultContext->sort->hasSortData(),
resultContext->grouping.get() != 0));
get_token_timer.done();
+ 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");
mergeDirector.dualMerge(thread_id, *resultContext->result, resultContext->groupingSource);
+ trace->addEvent(4, "MatchThread::run Done");
}
}
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.h b/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
index 06728dc006a..25f83be84a3 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
@@ -15,6 +15,11 @@
#include <vespa/searchlib/common/sortresults.h>
#include <vespa/searchlib/queryeval/hitcollector.h>
+namespace search::engine {
+ class Trace;
+ class RelativeTime;
+}
+
namespace proton::matching {
/**
@@ -30,6 +35,8 @@ public:
using RankProgram = search::fef::RankProgram;
using LazyValue = search::fef::LazyValue;
using Doom = vespalib::Doom;
+ using Trace = search::engine::Trace;
+ using RelativeTime = search::engine::RelativeTime;
private:
size_t thread_id;
@@ -48,6 +55,7 @@ private:
double match_time_s;
double wait_time_s;
bool match_with_ranking;
+ std::unique_ptr<Trace> trace;
class Context {
public:
@@ -103,11 +111,14 @@ public:
DocidRangeScheduler &sched,
ResultProcessor &rp,
vespalib::DualMergeDirector &md,
- uint32_t distributionKey);
+ uint32_t distributionKey,
+ const RelativeTime & relativeTime,
+ uint32_t traceLevel);
void run() override;
const MatchingStats::Partition &get_thread_stats() const { return thread_stats; }
double get_match_time() const { return match_time_s; }
PartialResult::UP extract_result() { return std::move(resultContext->result); }
+ const Trace & getTrace() const { return *trace; }
};
}
diff --git a/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp b/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
index 4ebf74c373f..5de014b4106 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/matcher.cpp
@@ -205,10 +205,10 @@ Matcher::computeNumThreadsPerSearch(Blueprint::HitEstimate hits, const Propertie
}
namespace {
- void traceQuery(const SearchRequest &request, const Query & query) {
- if (request.getTraceLevel() > 3) {
+ void traceQuery(uint32_t traceLevel, Trace & trace, const Query & query) {
+ if (traceLevel <= trace.getLevel()) {
if (query.peekRoot()) {
- vespalib::slime::ObjectInserter inserter(request.trace().createCursor("blueprint"), "optimized");
+ vespalib::slime::ObjectInserter inserter(trace.createCursor("blueprint"), "optimized");
query.peekRoot()->asSlime(inserter);
}
}
@@ -252,7 +252,7 @@ Matcher::match(const SearchRequest &request, vespalib::ThreadBundle &threadBundl
if (!mtf->valid()) {
reply->errorCode = ECODE_QUERY_PARSE_ERROR;
reply->errorMessage = "query execution failed (invalid query)";
- traceQuery(request, mtf->query());
+ traceQuery(1, request.trace(), mtf->query());
return reply;
}
@@ -269,10 +269,9 @@ Matcher::match(const SearchRequest &request, vespalib::ThreadBundle &threadBundl
size_t numThreadsPerSearch = computeNumThreadsPerSearch(mtf->estimate(), rankProperties);
LimitedThreadBundleWrapper limitedThreadBundle(threadBundle, numThreadsPerSearch);
MatchMaster master;
- uint32_t numSearchPartitions = NumSearchPartitions::lookup(rankProperties,
- _rankSetup->getNumSearchPartitions());
- ResultProcessor::Result::UP result = master.match(params, limitedThreadBundle, *mtf, rp,
- _distributionKey, numSearchPartitions);
+ uint32_t numParts = NumSearchPartitions::lookup(rankProperties, _rankSetup->getNumSearchPartitions());
+ ResultProcessor::Result::UP result = master.match(request, params, limitedThreadBundle, *mtf, rp,
+ _distributionKey, numParts);
my_stats = MatchMaster::getStats(std::move(master));
bool wasLimited = mtf->match_limiter().was_limited();
@@ -281,13 +280,13 @@ Matcher::match(const SearchRequest &request, vespalib::ThreadBundle &threadBundl
: mtf->match_limiter().getDocIdSpaceEstimate();
uint32_t estHits = mtf->estimate().estHits;
if (shouldCacheSearchSession && ((result->_numFs4Hits != 0) || shouldCacheGroupingSession)) {
- SearchSession::SP session = std::make_shared<SearchSession>(sessionId, request.getTimeOfDoom(),
- std::move(mtf), std::move(owned_objects));
+ auto session = std::make_shared<SearchSession>(sessionId, request.getTimeOfDoom(),
+ std::move(mtf), std::move(owned_objects));
session->releaseEnumGuards();
sessionMgr.insert(std::move(session));
}
reply = std::move(result->_reply);
- traceQuery(request, mtf->query());
+ traceQuery(6, request.trace(), mtf->query());
uint32_t numActiveLids = metaStore.getNumActiveLids();
// note: this is actually totalSpace+1, since 0 is reserved
diff --git a/searchlib/src/vespa/searchlib/engine/packetconverter.cpp b/searchlib/src/vespa/searchlib/engine/packetconverter.cpp
index d87d9888aae..c35ce9ded05 100644
--- a/searchlib/src/vespa/searchlib/engine/packetconverter.cpp
+++ b/searchlib/src/vespa/searchlib/engine/packetconverter.cpp
@@ -19,9 +19,7 @@ struct FS4PropertiesBuilder : public search::fef::IPropertiesVisitor {
uint32_t idx;
search::fs4transport::FS4Properties &props;
FS4PropertiesBuilder(search::fs4transport::FS4Properties &p) : idx(0), props(p) {}
- void visitProperty(const Property::Value &key,
- const Property &values) override
- {
+ void visitProperty(const Property::Value &key, const Property &values) override {
for (uint32_t i = 0; i < values.size(); ++i) {
props.setKey(idx, key.data(), key.size());
props.setValue(idx, values.getAt(i).data(), values.getAt(i).size());
diff --git a/searchlib/src/vespa/searchlib/engine/request.cpp b/searchlib/src/vespa/searchlib/engine/request.cpp
index 87f5d372609..956653d5269 100644
--- a/searchlib/src/vespa/searchlib/engine/request.cpp
+++ b/searchlib/src/vespa/searchlib/engine/request.cpp
@@ -8,14 +8,13 @@ namespace search::engine {
Request::Request(RelativeTime relativeTime)
: _relativeTime(std::move(relativeTime)),
_timeOfDoom(fastos::TimeStamp(fastos::TimeStamp::FUTURE)),
- _traceLevel(0),
queryFlags(0),
ranking(),
location(),
propertiesMap(),
stackItems(0),
stackDump(),
- _trace(_relativeTime)
+ _trace(_relativeTime, 0)
{
}
diff --git a/searchlib/src/vespa/searchlib/engine/request.h b/searchlib/src/vespa/searchlib/engine/request.h
index 9f8b0c9c1ae..733043b0e4e 100644
--- a/searchlib/src/vespa/searchlib/engine/request.h
+++ b/searchlib/src/vespa/searchlib/engine/request.h
@@ -21,7 +21,7 @@ public:
fastos::TimeStamp getTimeout() const { return _timeOfDoom - getStartTime(); }
fastos::TimeStamp getTimeUsed() const;
fastos::TimeStamp getTimeLeft() const;
- const RelativeTime & getRelativeTime() { return _relativeTime; }
+ const RelativeTime & getRelativeTime() const { return _relativeTime; }
bool expired() const { return getTimeLeft() <= 0l; }
const vespalib::stringref getStackRef() const {
@@ -30,14 +30,12 @@ public:
bool should_drop_sort_data() const;
- uint32_t getTraceLevel() const { return _traceLevel; }
- Request & setTraceLevel(uint32_t traceLevel) { _traceLevel = traceLevel; return *this; }
+ Request & setTraceLevel(uint32_t level) { _trace.setLevel(level); return *this; }
Trace & trace() const { return _trace; }
private:
RelativeTime _relativeTime;
fastos::TimeStamp _timeOfDoom;
- uint32_t _traceLevel;
public:
/// Everything here should move up to private section and have accessors
uint32_t queryFlags;
diff --git a/searchlib/src/vespa/searchlib/engine/trace.cpp b/searchlib/src/vespa/searchlib/engine/trace.cpp
index b11a44f3586..938d588f658 100644
--- a/searchlib/src/vespa/searchlib/engine/trace.cpp
+++ b/searchlib/src/vespa/searchlib/engine/trace.cpp
@@ -10,11 +10,12 @@ RelativeTime::RelativeTime(std::unique_ptr<Clock> clock)
_clock(std::move(clock))
{}
-Trace::Trace(const RelativeTime & relativeTime)
+Trace::Trace(const RelativeTime & relativeTime, uint32_t level)
: _trace(std::make_unique<vespalib::Slime>()),
_root(_trace->setObject()),
_traces(_root.setArray("traces")),
- _relativeTime(relativeTime)
+ _relativeTime(relativeTime),
+ _level(level)
{
_root.setLong("creation_time", _relativeTime.timeOfDawn());
}
@@ -29,6 +30,15 @@ Trace::createCursor(vespalib::stringref name) {
return trace;
}
+void
+Trace::addEvent(uint32_t level, vespalib::stringref event) {
+ if (!shouldTrace(level)) return;
+
+ Cursor & trace = _traces.addObject();
+ trace.setString("event", event);
+ trace.setLong("time", _relativeTime.timeSinceDawn());
+}
+
vespalib::string
Trace::toString() const {
return _trace->toString();
diff --git a/searchlib/src/vespa/searchlib/engine/trace.h b/searchlib/src/vespa/searchlib/engine/trace.h
index c9a6441c23d..c9052401f74 100644
--- a/searchlib/src/vespa/searchlib/engine/trace.h
+++ b/searchlib/src/vespa/searchlib/engine/trace.h
@@ -51,7 +51,7 @@ class Trace
{
public:
using Cursor = vespalib::slime::Cursor;
- Trace(const RelativeTime & relativeTime);
+ Trace(const RelativeTime & relativeTime, uint32_t traceLevel=0);
~Trace();
/**
@@ -60,14 +60,25 @@ public:
* @return a Cursor to use for further tracing.
*/
Cursor & createCursor(vespalib::stringref name);
+ /**
+ * Will add a simple 'event' string. It will also add a timestamp relative to the creation of the trace.
+ * @param level require for actually add the trace.
+ * @param event
+ */
+ void addEvent(uint32_t level, vespalib::stringref event);
+
vespalib::string toString() const;
Cursor & getRoot() const { return _root; }
vespalib::Slime & getSlime() const { return *_trace; }
+ bool shouldTrace(uint32_t level) const { return level <= _level; }
+ uint32_t getLevel() const { return _level; }
+ Trace & setLevel(uint32_t level) { _level = level; return *this; }
private:
std::unique_ptr<vespalib::Slime> _trace;
Cursor & _root;
Cursor & _traces;
const RelativeTime & _relativeTime;
+ uint32_t _level;
};
}
diff --git a/searchlib/src/vespa/searchlib/queryeval/searchiterator.cpp b/searchlib/src/vespa/searchlib/queryeval/searchiterator.cpp
index 3384e0fc8c8..c4a308a17f6 100644
--- a/searchlib/src/vespa/searchlib/queryeval/searchiterator.cpp
+++ b/searchlib/src/vespa/searchlib/queryeval/searchiterator.cpp
@@ -3,12 +3,13 @@
#include "searchiterator.h"
#include <vespa/searchlib/index/docidandfeatures.h>
#include <vespa/vespalib/objects/objectdumper.h>
+#include <vespa/vespalib/objects/object2slime.h>
#include <vespa/vespalib/objects/visit.h>
#include <vespa/vespalib/util/classname.h>
#include <vespa/searchlib/common/bitvector.h>
+#include <vespa/vespalib/data/slime/inserter.h>
-namespace search {
-namespace queryeval {
+namespace search::queryeval {
SearchIterator::SearchIterator() :
_docid(0),
@@ -106,6 +107,15 @@ SearchIterator::asString() const
return dumper.toString();
}
+vespalib::slime::Cursor &
+SearchIterator::asSlime(const vespalib::slime::Inserter & inserter) const
+{
+ vespalib::slime::Cursor & cursor = inserter.insertObject();
+ vespalib::Object2Slime dumper(cursor);
+ visit(dumper, "", this);
+ return cursor;
+}
+
vespalib::string
SearchIterator::getClassName() const
{
@@ -125,8 +135,7 @@ SearchIterator::getAttributeSearchContext() const
return nullptr;
}
-} // namespace queryeval
-} // namespace search
+}
//-----------------------------------------------------------------------------
diff --git a/searchlib/src/vespa/searchlib/queryeval/searchiterator.h b/searchlib/src/vespa/searchlib/queryeval/searchiterator.h
index dfa342b018a..ab662dd2592 100644
--- a/searchlib/src/vespa/searchlib/queryeval/searchiterator.h
+++ b/searchlib/src/vespa/searchlib/queryeval/searchiterator.h
@@ -10,6 +10,10 @@
#include <vector>
namespace vespalib { class ObjectVisitor; }
+namespace vespalib::slime {
+ class Cursor;
+ class Inserter;
+}
namespace search { class BitVector; }
namespace search::attribute { class ISearchContext; }
@@ -284,6 +288,15 @@ public:
vespalib::string asString() const;
/**
+ * Create a slime representation of this object. This
+ * method will use object visitation internally to capture the
+ * full structure of this object.
+ *
+ * @return structured slime representation of this object
+ **/
+ vespalib::slime::Cursor & asSlime(const vespalib::slime::Inserter & cursor) const;
+
+ /**
* Obtain the fully qualified name of the concrete class for this
* object. The default implementation will perform automatic name
* resolving. There is only a need to override this function if
@@ -310,7 +323,7 @@ public:
/**
* Empty, just defined to make it virtual.
**/
- virtual ~SearchIterator() { }
+ virtual ~SearchIterator() = default;
/**
* @return true if it is a bitvector