summaryrefslogtreecommitdiffstats
path: root/searchcore
diff options
context:
space:
mode:
authorHåvard Pettersen <havardpe@oath.com>2017-09-19 12:20:03 +0000
committerHåvard Pettersen <havardpe@oath.com>2017-09-19 12:31:44 +0000
commit3f3c65cc510d26a7146e8814e30cd31a2ee886b6 (patch)
tree4bbb724ab71f7d4eafc2558e64599ed3dc9f10b5 /searchcore
parentf29abebf697e89959368c57a57614b0166543bf3 (diff)
let match tools own rank_program and search
allows re-using query with new rank programs where possible refactor match loop to handle externally owned query refactor match loop iterator inlining for optimized blacklisting
Diffstat (limited to 'searchcore')
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_master.cpp20
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp188
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_thread.h41
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp91
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_tools.h42
5 files changed, 196 insertions, 186 deletions
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
index c6c75cc7c65..370c4b930e1 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
@@ -106,11 +106,15 @@ MatchMaster::getFeatureSet(const MatchToolsFactory &matchToolsFactory,
const std::vector<uint32_t> &docs, bool summaryFeatures)
{
MatchTools::UP matchTools = matchToolsFactory.createMatchTools();
- RankProgram::UP rankProgram = summaryFeatures ? matchTools->summary_program() :
- matchTools->dump_program();
+ if (summaryFeatures) {
+ matchTools->setup_summary();
+ } else {
+ matchTools->setup_dump();
+ }
+ RankProgram &rankProgram = matchTools->rank_program();
std::vector<vespalib::string> featureNames;
- FeatureResolver resolver(rankProgram->get_seeds());
+ FeatureResolver resolver(rankProgram.get_seeds());
featureNames.reserve(resolver.num_features());
for (size_t i = 0; i < resolver.num_features(); ++i) {
featureNames.emplace_back(resolver.name_of(i));
@@ -121,12 +125,12 @@ MatchMaster::getFeatureSet(const MatchToolsFactory &matchToolsFactory,
}
FeatureSet &fs = *retval.get();
- SearchIterator::UP search = matchTools->createSearch();
- search->initRange(docs.front(), docs.back()+1);
+ SearchIterator &search = matchTools->search();
+ search.initRange(docs.front(), docs.back()+1);
for (uint32_t i = 0; i < docs.size(); ++i) {
- if (search->seek(docs[i])) {
- uint32_t docId = search->getDocId();
- search->unpack(docId);
+ if (search.seek(docs[i])) {
+ uint32_t docId = search.getDocId();
+ search.unpack(docId);
search::feature_t * f = fs.getFeaturesByIndex(
fs.addDocId(docId));
for (uint32_t j = 0; j < featureNames.size(); ++j) {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
index ae01973439b..73cd5ffca06 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.cpp
@@ -40,39 +40,22 @@ struct WaitTimer {
}
};
-class FastSeekWrapper
-{
-private:
- typedef search::queryeval::SearchIterator SearchIterator;
-public:
- FastSeekWrapper(SearchIterator::UP iterator)
- {
- reset(iterator.release());
- }
- void initRange(uint32_t begin_id, uint32_t end_id) {
- _search->initRange(begin_id, end_id);
- }
- uint32_t seekFirst(uint32_t docId) {
- return _search->seekFirst(docId);
- }
- uint32_t seekNext(uint32_t docId) {
- return _search->seekFast(docId);
+// seek_next maps to SearchIterator::seekNext
+struct SimpleStrategy {
+ static uint32_t seek_next(SearchIterator &search, uint32_t docid) {
+ return search.seekNext(docid);
}
- vespalib::string asString() const {
- return _search->asString();
- }
- void unpack(uint32_t docId) {
- _search->unpack(docId);
- }
- void reset(SearchIterator * search) {
- _search.reset(&dynamic_cast<OptimizedAndNotForBlackListing &>(*search));
+};
+
+// seek_next maps to OptimizedAndNotForBlackListing::seekFast
+struct FastBlackListingStrategy {
+ static bool can_use(bool do_limit, SearchIterator &search) {
+ return (!do_limit &&
+ (dynamic_cast<OptimizedAndNotForBlackListing *>(&search) != nullptr));
}
- OptimizedAndNotForBlackListing * release() {
- return _search.release();
+ static uint32_t seek_next(SearchIterator &search, uint32_t docid) {
+ return static_cast<OptimizedAndNotForBlackListing &>(search).seekFast(docid);
}
- FastSeekWrapper * operator ->() { return this; }
-private:
- std::unique_ptr<OptimizedAndNotForBlackListing> _search;
};
LazyValue get_score_feature(const RankProgram &rankProgram) {
@@ -85,17 +68,17 @@ LazyValue get_score_feature(const RankProgram &rankProgram) {
//-----------------------------------------------------------------------------
-MatchThread::Context::Context(double rankDropLimit, MatchTools & matchTools, RankProgram & ranking, HitCollector & hits,
- uint32_t num_threads) :
- matches(0),
- _matches_limit(matchTools.match_limiter().sample_hits_per_thread(num_threads)),
- _score_feature(get_score_feature(ranking)),
- _ranking(ranking),
- _rankDropLimit(rankDropLimit),
- _hits(hits),
- _softDoom(matchTools.getSoftDoom()),
- _limiter(matchTools.match_limiter())
-{ }
+MatchThread::Context::Context(double rankDropLimit, MatchTools &tools, HitCollector &hits,
+ uint32_t num_threads)
+ : matches(0),
+ _matches_limit(tools.match_limiter().sample_hits_per_thread(num_threads)),
+ _score_feature(get_score_feature(tools.rank_program())),
+ _ranking(tools.rank_program()),
+ _rankDropLimit(rankDropLimit),
+ _hits(hits),
+ _softDoom(tools.getSoftDoom())
+{
+}
void
MatchThread::Context::rankHit(uint32_t docId) {
@@ -122,27 +105,27 @@ MatchThread::estimate_match_frequency(uint32_t matches, uint32_t searchedSoFar)
return match_freq;
}
-template <typename IteratorT>
-void
-MatchThread::maybe_limit(MaybeMatchPhaseLimiter & limiter, IteratorT & search, uint32_t matches, uint32_t docId, uint32_t endId)
+SearchIterator *
+MatchThread::maybe_limit(MatchTools &tools, uint32_t matches, uint32_t docId, uint32_t endId)
{
const uint32_t local_todo = (endId - docId - 1);
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();
- search = limiter.maybe_limit(std::move(search), match_freq, matchParams.numDocs);
+ {
+ auto search = tools.borrow_search();
+ search = tools.match_limiter().maybe_limit(std::move(search), match_freq, matchParams.numDocs);
+ tools.give_back_search(std::move(search));
+ if (tools.match_limiter().was_limited()) {
+ tools.tag_search_as_changed();
+ }
+ }
size_t left = local_todo + (global_todo / num_threads);
- limiter.updateDocIdSpaceEstimate(searchedSoFar, left);
+ tools.match_limiter().updateDocIdSpaceEstimate(searchedSoFar, left);
LOG(debug, "Limit=%d has been reached at docid=%d which is after %zu docs.",
matches, docId, (scheduler.total_size(thread_id) - local_todo));
- LOG(debug, "SearchIterator after limiter: %s", search->asString().c_str());
-}
-
-template <>
-void
-MatchThread::maybe_limit(MaybeMatchPhaseLimiter &, FastSeekWrapper &, uint32_t, uint32_t, uint32_t)
-{
- abort(); // We cannot replace the iterator if we inline the loop.
+ LOG(debug, "SearchIterator after limiter: %s", tools.search().asString().c_str());
+ return &tools.search();
}
bool
@@ -156,10 +139,11 @@ MatchThread::try_share(DocidRange &docid_range, uint32_t next_docid) {
return false;
}
-template <typename IteratorT, bool do_rank, bool do_limit, bool do_share_work>
+template <typename Strategy, bool do_rank, bool do_limit, bool do_share_work>
bool
-MatchThread::inner_match_loop(Context & context, IteratorT & search, DocidRange docid_range)
+MatchThread::inner_match_loop(Context &context, MatchTools &tools, DocidRange docid_range)
{
+ SearchIterator *search = &tools.search();
search->initRange(docid_range.begin, docid_range.end);
uint32_t docId = search->seekFirst(docid_range.begin);
while ((docId < docid_range.end) && !context.atSoftDoom()) {
@@ -171,30 +155,29 @@ MatchThread::inner_match_loop(Context & context, IteratorT & search, DocidRange
}
context.matches++;
if (do_limit && context.isAtLimit()) {
- maybe_limit(context.limiter(), search, context.matches, docId, docid_range.end);
+ search = maybe_limit(tools, context.matches, docId, docid_range.end);
docId = search->seekFirst(docId + 1);
} else if (do_share_work && any_idle() && try_share(docid_range, docId + 1)) {
search->initRange(docid_range.begin, docid_range.end);
docId = search->seekFirst(docid_range.begin);
} else {
- docId = search->seekNext(docId + 1);
+ docId = Strategy::seek_next(*search, docId + 1);
}
}
return (docId < docid_range.end);
}
-template <typename IteratorT, bool do_rank, bool do_limit, bool do_share_work>
+template <typename Strategy, bool do_rank, bool do_limit, bool do_share_work>
void
-MatchThread::match_loop(MatchTools &matchTools, IteratorT search,
- RankProgram &ranking, HitCollector &hits)
+MatchThread::match_loop(MatchTools &tools, HitCollector &hits)
{
bool softDoomed = false;
- Context context(matchParams.rankDropLimit, matchTools, ranking, hits, num_threads);
+ Context context(matchParams.rankDropLimit, tools, hits, num_threads);
for (DocidRange docid_range = scheduler.first_range(thread_id);
!docid_range.empty() && ! softDoomed;
docid_range = scheduler.next_range(thread_id))
{
- softDoomed = inner_match_loop<IteratorT, do_rank, do_limit, do_share_work>(context, search, docid_range);
+ softDoomed = inner_match_loop<Strategy, do_rank, do_limit, do_share_work>(context, tools, docid_range);
}
uint32_t matches = context.matches;
if (do_limit && context.isBelowLimit()) {
@@ -202,7 +185,7 @@ MatchThread::match_loop(MatchTools &matchTools, IteratorT search,
LOG(debug, "Limit not reached (had %d) at docid=%d which is after %zu docs.",
matches, scheduler.total_span(thread_id).end, searchedSoFar);
estimate_match_frequency(matches, searchedSoFar);
- context.limiter().updateDocIdSpaceEstimate(searchedSoFar, 0);
+ tools.match_limiter().updateDocIdSpaceEstimate(searchedSoFar, 0);
}
thread_stats.docsMatched(matches);
thread_stats.softDoomed(softDoomed);
@@ -211,68 +194,77 @@ MatchThread::match_loop(MatchTools &matchTools, IteratorT search,
}
}
-template <typename IteratorT, bool do_rank, bool do_limit>
+//-----------------------------------------------------------------------------
+
+template <bool do_rank, bool do_limit, bool do_share>
+void
+MatchThread::match_loop_helper_rank_limit_share(MatchTools &tools, HitCollector &hits)
+{
+ if (FastBlackListingStrategy::can_use(do_limit, tools.search())) {
+ match_loop<FastBlackListingStrategy, do_rank, do_limit, do_share>(tools, hits);
+ } else {
+ match_loop<SimpleStrategy, do_rank, do_limit, do_share>(tools, hits);
+ }
+}
+
+template <bool do_rank, bool do_limit>
void
-MatchThread::match_loop_helper_2(MatchTools &matchTools, IteratorT search,
- RankProgram &ranking, HitCollector &hits)
+MatchThread::match_loop_helper_rank_limit(MatchTools &tools, HitCollector &hits)
{
if (idle_observer.is_always_zero()) {
- match_loop<IteratorT, do_rank, do_limit, false>(matchTools, std::move(search), ranking, hits);
+ match_loop_helper_rank_limit_share<do_rank, do_limit, false>(tools, hits);
+ } else {
+ match_loop_helper_rank_limit_share<do_rank, do_limit, true>(tools, hits);
+ }
+}
+
+template <bool do_rank>
+void
+MatchThread::match_loop_helper_rank(MatchTools &tools, HitCollector &hits)
+{
+ if (tools.match_limiter().is_enabled()) {
+ match_loop_helper_rank_limit<do_rank, true>(tools, hits);
} else {
- match_loop<IteratorT, do_rank, do_limit, true>(matchTools, std::move(search), ranking, hits);
+ match_loop_helper_rank_limit<do_rank, false>(tools, hits);
}
}
-template <typename IteratorT, bool do_rank>
void
-MatchThread::match_loop_helper(MatchTools &matchTools, IteratorT search,
- RankProgram &ranking, HitCollector &hits)
+MatchThread::match_loop_helper(MatchTools &tools, HitCollector &hits)
{
- if (matchTools.match_limiter().is_enabled()) {
- match_loop_helper_2<IteratorT, do_rank, true>(matchTools, std::move(search), ranking, hits);
+ if (match_with_ranking) {
+ match_loop_helper_rank<true>(tools, hits);
} else {
- match_loop_helper_2<IteratorT, do_rank, false>(matchTools, std::move(search), ranking, hits);
+ match_loop_helper_rank<false>(tools, hits);
}
}
//-----------------------------------------------------------------------------
search::ResultSet::UP
-MatchThread::findMatches(MatchTools &matchTools)
+MatchThread::findMatches(MatchTools &tools)
{
- RankProgram::UP ranking = matchTools.first_phase_program();
- SearchIterator::UP search = matchTools.createSearch();
+ tools.setup_first_phase();
if (isFirstThread()) {
- LOG(spam, "SearchIterator: %s", search->asString().c_str());
+ LOG(spam, "SearchIterator: %s", tools.search().asString().c_str());
}
- search = search::queryeval::MultiBitVectorIteratorBase::optimize(std::move(search));
+ tools.give_back_search(search::queryeval::MultiBitVectorIteratorBase::optimize(tools.borrow_search()));
if (isFirstThread()) {
- LOG(debug, "SearchIterator after MultiBitVectorIteratorBase::optimize(): %s", search->asString().c_str());
+ LOG(debug, "SearchIterator after MultiBitVectorIteratorBase::optimize(): %s", tools.search().asString().c_str());
}
HitCollector hits(matchParams.numDocs, matchParams.arraySize, matchParams.heapSize);
- if (match_with_ranking) {
- match_loop_helper<SearchIterator::UP, true>(matchTools, std::move(search), *ranking, hits);
- } else {
- if ((dynamic_cast<const OptimizedAndNotForBlackListing *>(search.get()) != 0) &&
- ! matchTools.match_limiter().is_enabled()) // We cannot replace the iterator if we inline the loop.
- {
- match_loop_helper_2<FastSeekWrapper, false, false>(matchTools, FastSeekWrapper(std::move(search)), *ranking, hits);
- } else {
- match_loop_helper<SearchIterator::UP, false>(matchTools, std::move(search), *ranking, hits);
- }
- }
- if (matchTools.has_second_phase_rank()) {
+ match_loop_helper(tools, hits);
+ if (tools.has_second_phase_rank()) {
{ // 2nd phase ranking
- ranking = matchTools.second_phase_program();
- search = matchTools.createSearch();
+ tools.setup_second_phase();
DocidRange docid_range = scheduler.total_span(thread_id);
- search->initRange(docid_range.begin, docid_range.end);
+ tools.search().initRange(docid_range.begin, docid_range.end);
auto sorted_scores = hits.getSortedHeapScores();
WaitTimer select_best_timer(wait_time_s);
size_t useHits = communicator.selectBest(sorted_scores);
select_best_timer.done();
- DocumentScorer scorer(*ranking, *search);
- uint32_t reRanked = hits.reRank(scorer, matchTools.getHardDoom().doom() ? 0 : useHits);
+ DocumentScorer scorer(tools.rank_program(), tools.search());
+ uint32_t reRanked = hits.reRank(scorer, tools.getHardDoom().doom() ? 0 : useHits);
thread_stats.docsReRanked(reRanked);
}
{ // rank scaling
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_thread.h b/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
index a279e83f032..b08323fa099 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_thread.h
@@ -51,46 +51,41 @@ private:
class Context {
public:
- Context(double rankDropLimit, MatchTools &matchTools, RankProgram & ranking, HitCollector & hits,
+ Context(double rankDropLimit, MatchTools &tools, HitCollector &hits,
uint32_t num_threads) __attribute__((noinline));
void rankHit(uint32_t docId);
void addHit(uint32_t docId) { _hits.addHit(docId, search::zero_rank_value); }
bool isBelowLimit() const { return matches < _matches_limit; }
bool isAtLimit() const { return matches == _matches_limit; }
bool atSoftDoom() const { return _softDoom.doom(); }
- MaybeMatchPhaseLimiter & limiter() { return _limiter; }
- uint32_t matches;
+ uint32_t matches;
private:
- uint32_t _matches_limit;
- LazyValue _score_feature;
- RankProgram & _ranking;
- double _rankDropLimit;
- HitCollector & _hits;
- const Doom & _softDoom;
- MaybeMatchPhaseLimiter & _limiter;
+ uint32_t _matches_limit;
+ LazyValue _score_feature;
+ RankProgram &_ranking;
+ double _rankDropLimit;
+ HitCollector &_hits;
+ const Doom &_softDoom;
};
double estimate_match_frequency(uint32_t matches, uint32_t searchedSoFar) __attribute__((noinline));
-
- template <typename IteratorT>
- void maybe_limit(MaybeMatchPhaseLimiter & limiter, IteratorT & search, uint32_t matches, uint32_t docId, uint32_t endId) __attribute__((noinline));
+ SearchIterator *maybe_limit(MatchTools &tools, uint32_t matches, uint32_t docId, uint32_t endId) __attribute__((noinline));
bool any_idle() const { return (idle_observer.get() > 0); }
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>
- 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));
+ template <typename Strategy, bool do_rank, bool do_limit, bool do_share_work>
+ bool inner_match_loop(Context &context, MatchTools &tools, DocidRange docid_range) __attribute__((noinline));
- template <typename IteratorT, bool do_rank, bool do_limit>
- void match_loop_helper_2(MatchTools &matchTools, IteratorT search, RankProgram &ranking, HitCollector &hits);
+ template <typename Strategy, bool do_rank, bool do_limit, bool do_share_work>
+ void match_loop(MatchTools &tools, HitCollector &hits) __attribute__((noinline));
- template <typename IteratorT, bool do_rank>
- void match_loop_helper(MatchTools &matchTools, IteratorT search, RankProgram &ranking, HitCollector &hits);
+ template <bool do_rank, bool do_limit, bool do_share> void match_loop_helper_rank_limit_share(MatchTools &tools, HitCollector &hits);
+ template <bool do_rank, bool do_limit> void match_loop_helper_rank_limit(MatchTools &tools, HitCollector &hits);
+ template <bool do_rank> void match_loop_helper_rank(MatchTools &tools, HitCollector &hits);
+ void match_loop_helper(MatchTools &tools, HitCollector &hits);
- search::ResultSet::UP findMatches(MatchTools &matchTools);
+ search::ResultSet::UP findMatches(MatchTools &tools);
void processResult(const Doom & hardDoom, search::ResultSet::UP result, ResultProcessor::Context &context);
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
index 9c1b0b3ea0d..9044cb79220 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
@@ -19,34 +19,48 @@ namespace proton::matching {
namespace {
-size_t
-tagMatchData(const HandleRecorder::HandleSet & handles, MatchData & md)
+bool contains_all(const HandleRecorder::HandleSet &old_set,
+ const HandleRecorder::HandleSet &new_set)
{
- size_t ignored(0);
- for (TermFieldHandle handle(0); handle < md.getNumTermFields(); handle++) {
+ for (TermFieldHandle handle: new_set) {
+ if (old_set.find(handle) == old_set.end()) {
+ return false;
+ }
+ }
+ return true;
+}
+
+void tag_match_data(const HandleRecorder::HandleSet &handles, MatchData &match_data) {
+ for (TermFieldHandle handle = 0; handle < match_data.getNumTermFields(); ++handle) {
if (handles.find(handle) == handles.end()) {
- md.resolveTermField(handle)->tagAsNotNeeded();
- ignored++;
+ match_data.resolveTermField(handle)->tagAsNotNeeded();
}
}
- return ignored;
}
-search::fef::RankProgram::UP setup_program(search::fef::RankProgram::UP program,
- MatchData &match_data,
- const QueryEnvironment &queryEnv,
- const Properties &featureOverrides)
+} // namespace proton::matching::<unnamed>
+
+void
+MatchTools::setup(search::fef::RankProgram::UP rank_program, double termwise_limit)
{
- match_data.soft_reset();
+ if (_search) {
+ _match_data->soft_reset();
+ }
+ _rank_program = std::move(rank_program);
HandleRecorder recorder;
{
HandleRecorder::Binder bind(recorder);
- program->setup(match_data, queryEnv, featureOverrides);
+ _rank_program->setup(*_match_data, _queryEnv, _featureOverrides);
+ }
+ bool can_reuse_search = (_search && !_search_has_changed &&
+ contains_all(_used_handles, recorder.getHandles()));
+ if (!can_reuse_search) {
+ tag_match_data(recorder.getHandles(), *_match_data);
+ _match_data->set_termwise_limit(termwise_limit);
+ _search = _query.createSearch(*_match_data);
+ _used_handles = recorder.getHandles();
+ _search_has_changed = false;
}
- tagMatchData(recorder.getHandles(), match_data);
- return program;
-}
-
}
MatchTools::MatchTools(QueryLimiter & queryLimiter,
@@ -66,41 +80,42 @@ MatchTools::MatchTools(QueryLimiter & queryLimiter,
_queryEnv(queryEnv),
_rankSetup(rankSetup),
_featureOverrides(featureOverrides),
- _match_data(mdl.createMatchData())
+ _match_data(mdl.createMatchData()),
+ _rank_program(),
+ _search(),
+ _used_handles(),
+ _search_has_changed(false)
{
}
-MatchTools::~MatchTools() {}
+MatchTools::~MatchTools()
+{
+}
-search::fef::RankProgram::UP
-MatchTools::first_phase_program()
+void
+MatchTools::setup_first_phase()
{
- auto program = setup_program(_rankSetup.create_first_phase_program(),
- *_match_data, _queryEnv, _featureOverrides);
- _match_data->set_termwise_limit(TermwiseLimit::lookup(_queryEnv.getProperties(),
- _rankSetup.get_termwise_limit()));
- return program;
+ setup(_rankSetup.create_first_phase_program(),
+ TermwiseLimit::lookup(_queryEnv.getProperties(),
+ _rankSetup.get_termwise_limit()));
}
-search::fef::RankProgram::UP
-MatchTools::second_phase_program()
+void
+MatchTools::setup_second_phase()
{
- return setup_program(_rankSetup.create_second_phase_program(),
- *_match_data, _queryEnv, _featureOverrides);
+ setup(_rankSetup.create_second_phase_program());
}
-search::fef::RankProgram::UP
-MatchTools::summary_program()
+void
+MatchTools::setup_summary()
{
- return setup_program(_rankSetup.create_summary_program(),
- *_match_data, _queryEnv, _featureOverrides);
+ setup(_rankSetup.create_summary_program());
}
-search::fef::RankProgram::UP
-MatchTools::dump_program()
+void
+MatchTools::setup_dump()
{
- return setup_program(_rankSetup.create_dump_program(),
- *_match_data, _queryEnv, _featureOverrides);
+ setup(_rankSetup.create_dump_program());
}
//-----------------------------------------------------------------------------
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_tools.h b/searchcore/src/vespa/searchcore/proton/matching/match_tools.h
index 460462da35b..f47eda16cc1 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_tools.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_tools.h
@@ -23,15 +23,20 @@ class MatchTools
{
private:
using IRequestContext = search::queryeval::IRequestContext;
- QueryLimiter & _queryLimiter;
- const vespalib::Doom & _softDoom;
- const vespalib::Doom & _hardDoom;
- const Query & _query;
- MaybeMatchPhaseLimiter & _match_limiter;
- const QueryEnvironment & _queryEnv;
- const search::fef::RankSetup & _rankSetup;
- const search::fef::Properties & _featureOverrides;
- search::fef::MatchData::UP _match_data;
+ QueryLimiter &_queryLimiter;
+ const vespalib::Doom &_softDoom;
+ const vespalib::Doom &_hardDoom;
+ const Query &_query;
+ MaybeMatchPhaseLimiter &_match_limiter;
+ const QueryEnvironment &_queryEnv;
+ const search::fef::RankSetup &_rankSetup;
+ const search::fef::Properties &_featureOverrides;
+ search::fef::MatchData::UP _match_data;
+ search::fef::RankProgram::UP _rank_program;
+ search::queryeval::SearchIterator::UP _search;
+ HandleRecorder::HandleSet _used_handles;
+ bool _search_has_changed;
+ void setup(search::fef::RankProgram::UP, double termwise_limit = 1.0);
public:
typedef std::unique_ptr<MatchTools> UP;
MatchTools(const MatchTools &) = delete;
@@ -50,18 +55,17 @@ public:
const vespalib::Doom &getHardDoom() const { return _hardDoom; }
QueryLimiter & getQueryLimiter() { return _queryLimiter; }
MaybeMatchPhaseLimiter &match_limiter() { return _match_limiter; }
- search::queryeval::SearchIterator::UP createSearch() {
- return _query.createSearch(*_match_data);
- }
bool has_second_phase_rank() const { return !_rankSetup.getSecondPhaseRank().empty(); }
const search::fef::MatchData &match_data() const { return *_match_data; }
- // NOTE: since rank programs share a single match data, creating a
- // new program will invalidate the underlying match data and thus
- // also any previously created rank programs.
- search::fef::RankProgram::UP first_phase_program();
- search::fef::RankProgram::UP second_phase_program();
- search::fef::RankProgram::UP summary_program();
- search::fef::RankProgram::UP dump_program();
+ search::fef::RankProgram &rank_program() { return *_rank_program; }
+ search::queryeval::SearchIterator &search() { return *_search; }
+ search::queryeval::SearchIterator::UP borrow_search() { return std::move(_search); }
+ void give_back_search(search::queryeval::SearchIterator::UP search_in) { _search = std::move(search_in); }
+ void tag_search_as_changed() { _search_has_changed = true; }
+ void setup_first_phase();
+ void setup_second_phase();
+ void setup_summary();
+ void setup_dump();
};
class MatchToolsFactory : public vespalib::noncopyable