summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHåvard Pettersen <havardpe@oath.com>2017-09-07 12:26:22 +0000
committerHåvard Pettersen <havardpe@oath.com>2017-09-20 09:16:17 +0000
commit07145ad5ee9cc0f6d1bef0c76fd93a1053d9dde2 (patch)
tree8d3a391aef94fd7d8010820fb7f4f7947f97b660
parentf1018cb802aeb7b7731a0fc29ee7c708b665148d (diff)
enable re-using termwise bitvectors in second phase
detach match data from rank program and refactor match loop re-use bitvector in termwise search if range is the same
-rw-r--r--searchcore/src/tests/proton/matching/matching_test.cpp4
-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.cpp96
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_tools.h42
-rw-r--r--searchlib/src/tests/features/item_raw_score/item_raw_score_test.cpp10
-rw-r--r--searchlib/src/tests/features/native_dot_product/native_dot_product_test.cpp16
-rw-r--r--searchlib/src/tests/features/raw_score/raw_score_test.cpp10
-rw-r--r--searchlib/src/tests/features/subqueries/subqueries_test.cpp10
-rw-r--r--searchlib/src/tests/fef/featureoverride/featureoverride.cpp3
-rw-r--r--searchlib/src/tests/fef/object_passing/object_passing_test.cpp4
-rw-r--r--searchlib/src/tests/fef/rank_program/rank_program_test.cpp7
-rw-r--r--searchlib/src/tests/fef/termfieldmodel/termfieldmodel_test.cpp35
-rw-r--r--searchlib/src/tests/ranksetup/ranksetup_test.cpp23
-rw-r--r--searchlib/src/vespa/searchlib/features/attributematchfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/attributematchfeature.h4
-rw-r--r--searchlib/src/vespa/searchlib/features/element_completeness_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/element_completeness_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/element_similarity_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldinfofeature.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldinfofeature.h4
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldlengthfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldlengthfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldmatchfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldmatchfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/flow_completeness_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/flow_completeness_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/item_raw_score_feature.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/features/item_raw_score_feature.h4
-rw-r--r--searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/matchcountfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/matchcountfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/matchesfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/matchesfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/native_dot_product_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/native_dot_product_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/nativeattributematchfeature.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/features/nativeattributematchfeature.h4
-rw-r--r--searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/nativeproximityfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/nativeproximityfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/proximityfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/proximityfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/querycompletenessfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/querycompletenessfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/raw_score_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/raw_score_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/reverseproximityfeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/reverseproximityfeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/subqueries_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/subqueries_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/term_field_md_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/term_field_md_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/termdistancefeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/termdistancefeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/termeditdistancefeature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/termeditdistancefeature.h2
-rw-r--r--searchlib/src/vespa/searchlib/features/text_similarity_feature.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/features/text_similarity_feature.h2
-rw-r--r--searchlib/src/vespa/searchlib/fef/featureexecutor.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/fef/featureexecutor.h4
-rw-r--r--searchlib/src/vespa/searchlib/fef/featureoverrider.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/fef/featureoverrider.h2
-rw-r--r--searchlib/src/vespa/searchlib/fef/matchdata.cpp9
-rw-r--r--searchlib/src/vespa/searchlib/fef/matchdata.h9
-rw-r--r--searchlib/src/vespa/searchlib/fef/rank_program.cpp12
-rw-r--r--searchlib/src/vespa/searchlib/fef/rank_program.h25
-rw-r--r--searchlib/src/vespa/searchlib/fef/termfieldmatchdata.h7
-rw-r--r--searchlib/src/vespa/searchlib/fef/test/featuretest.cpp7
-rw-r--r--searchlib/src/vespa/searchlib/fef/test/featuretest.h1
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/termwise_search.cpp22
-rw-r--r--streamingvisitors/src/vespa/searchvisitor/rankprocessor.cpp15
-rw-r--r--streamingvisitors/src/vespa/searchvisitor/rankprocessor.h3
78 files changed, 421 insertions, 322 deletions
diff --git a/searchcore/src/tests/proton/matching/matching_test.cpp b/searchcore/src/tests/proton/matching/matching_test.cpp
index fefc6f361a9..261827a1e06 100644
--- a/searchcore/src/tests/proton/matching/matching_test.cpp
+++ b/searchcore/src/tests/proton/matching/matching_test.cpp
@@ -292,8 +292,8 @@ struct MyWorld {
MatchToolsFactory::UP match_tools_factory = matcher->create_match_tools_factory(
*request, searchContext, attributeContext, metaStore, overrides);
MatchTools::UP match_tools = match_tools_factory->createMatchTools();
- RankProgram::UP rank_program = match_tools->first_phase_program();
- return rank_program->match_data().get_termwise_limit();
+ match_tools->setup_first_phase();
+ return match_tools->match_data().get_termwise_limit();
}
SearchReply::UP performSearch(SearchRequest::SP req, size_t threads) {
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_master.cpp
index 0b5287fb094..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(rankProgram->match_data());
- 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 3a9f59680f9..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(ranking->match_data());
+ 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(ranking->match_data());
+ 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 1e429616e7e..9044cb79220 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
@@ -19,33 +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,
- const MatchDataLayout &mdl,
- const QueryEnvironment &queryEnv,
- const Properties &featureOverrides)
+} // namespace proton::matching::<unnamed>
+
+void
+MatchTools::setup(search::fef::RankProgram::UP rank_program, double termwise_limit)
{
+ if (_search) {
+ _match_data->soft_reset();
+ }
+ _rank_program = std::move(rank_program);
HandleRecorder recorder;
{
HandleRecorder::Binder bind(recorder);
- program->setup(mdl, 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(), program->match_data());
- return program;
-}
-
}
MatchTools::MatchTools(QueryLimiter & queryLimiter,
@@ -65,39 +80,42 @@ MatchTools::MatchTools(QueryLimiter & queryLimiter,
_queryEnv(queryEnv),
_rankSetup(rankSetup),
_featureOverrides(featureOverrides),
- _mdl(mdl),
- _handleRecorder()
+ _match_data(mdl.createMatchData()),
+ _rank_program(),
+ _search(),
+ _used_handles(),
+ _search_has_changed(false)
{
- HandleRecorder::Binder bind(_handleRecorder);
}
-MatchTools::~MatchTools() {}
+MatchTools::~MatchTools()
+{
+}
-search::fef::RankProgram::UP
-MatchTools::first_phase_program() const {
- auto program = setup_program(_rankSetup.create_first_phase_program(),
- _mdl, _queryEnv, _featureOverrides);
- program->match_data().set_termwise_limit(TermwiseLimit::lookup(_queryEnv.getProperties(),
- _rankSetup.get_termwise_limit()));
- return program;
+void
+MatchTools::setup_first_phase()
+{
+ setup(_rankSetup.create_first_phase_program(),
+ TermwiseLimit::lookup(_queryEnv.getProperties(),
+ _rankSetup.get_termwise_limit()));
}
-search::fef::RankProgram::UP
-MatchTools::second_phase_program() const {
- return setup_program(_rankSetup.create_second_phase_program(),
- _mdl, _queryEnv, _featureOverrides);
+void
+MatchTools::setup_second_phase()
+{
+ setup(_rankSetup.create_second_phase_program());
}
-search::fef::RankProgram::UP
-MatchTools::summary_program() const {
- return setup_program(_rankSetup.create_summary_program(),
- _mdl, _queryEnv, _featureOverrides);
+void
+MatchTools::setup_summary()
+{
+ setup(_rankSetup.create_summary_program());
}
-search::fef::RankProgram::UP
-MatchTools::dump_program() const {
- return setup_program(_rankSetup.create_dump_program(),
- _mdl, _queryEnv, _featureOverrides);
+void
+MatchTools::setup_dump()
+{
+ 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 b4f1f997d32..f47eda16cc1 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_tools.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_tools.h
@@ -23,16 +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::MatchDataLayout _mdl;
- HandleRecorder _handleRecorder;
+ 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;
@@ -51,15 +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(search::fef::MatchData &matchData) const {
- return _query.createSearch(matchData);
- }
bool has_second_phase_rank() const { return !_rankSetup.getSecondPhaseRank().empty(); }
- search::fef::RankProgram::UP first_phase_program() const;
- search::fef::RankProgram::UP second_phase_program() const;
- search::fef::RankProgram::UP summary_program() const;
- search::fef::RankProgram::UP dump_program() const;
+ const search::fef::MatchData &match_data() const { return *_match_data; }
+ 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
diff --git a/searchlib/src/tests/features/item_raw_score/item_raw_score_test.cpp b/searchlib/src/tests/features/item_raw_score/item_raw_score_test.cpp
index f407aea811b..a954710f153 100644
--- a/searchlib/src/tests/features/item_raw_score/item_raw_score_test.cpp
+++ b/searchlib/src/tests/features/item_raw_score/item_raw_score_test.cpp
@@ -67,13 +67,14 @@ struct SingleLabel : public Labels {
struct RankFixture : BlueprintFactoryFixture, IndexFixture {
QueryEnvironment queryEnv;
RankSetup rankSetup;
- RankProgram::UP rankProgram;
MatchDataLayout mdl;
+ MatchData::UP match_data;
+ RankProgram::UP rankProgram;
std::vector<TermFieldHandle> fooHandles;
std::vector<TermFieldHandle> barHandles;
RankFixture(size_t fooCnt, size_t barCnt, const Labels &labels)
: queryEnv(&indexEnv), rankSetup(factory, indexEnv),
- rankProgram(), mdl(), fooHandles(), barHandles()
+ mdl(), match_data(), rankProgram(), fooHandles(), barHandles()
{
for (size_t i = 0; i < fooCnt; ++i) {
uint32_t fieldId = indexEnv.getFieldByName("foo")->id();
@@ -95,14 +96,15 @@ struct RankFixture : BlueprintFactoryFixture, IndexFixture {
rankSetup.setFirstPhaseRank(featureName);
rankSetup.setIgnoreDefaultRankFeatures(true);
ASSERT_TRUE(rankSetup.compile());
+ match_data = mdl.createMatchData();
rankProgram = rankSetup.create_first_phase_program();
- rankProgram->setup(mdl, queryEnv);
+ rankProgram->setup(*match_data, queryEnv);
}
feature_t getScore(uint32_t docId) {
return Utils::getScoreFeature(*rankProgram, docId);
}
void setScore(TermFieldHandle handle, uint32_t docId, feature_t score) {
- rankProgram->match_data().resolveTermField(handle)->setRawScore(docId, score);
+ match_data->resolveTermField(handle)->setRawScore(docId, score);
}
void setFooScore(uint32_t i, uint32_t docId, feature_t score) {
ASSERT_LESS(i, fooHandles.size());
diff --git a/searchlib/src/tests/features/native_dot_product/native_dot_product_test.cpp b/searchlib/src/tests/features/native_dot_product/native_dot_product_test.cpp
index 5ba7504f7d2..1da912ccb3a 100644
--- a/searchlib/src/tests/features/native_dot_product/native_dot_product_test.cpp
+++ b/searchlib/src/tests/features/native_dot_product/native_dot_product_test.cpp
@@ -72,14 +72,15 @@ std::vector<uint32_t> vec(uint32_t w1, uint32_t w2, uint32_t w3) {
struct RankFixture : BlueprintFactoryFixture, IndexFixture {
QueryEnvironment queryEnv;
RankSetup rankSetup;
- RankProgram::UP rankProgram;
MatchDataLayout mdl;
+ MatchData::UP match_data;
+ RankProgram::UP rankProgram;
std::vector<TermFieldHandle> fooHandles;
std::vector<TermFieldHandle> barHandles;
RankFixture(const std::vector<uint32_t> &fooWeights,
const std::vector<uint32_t> &barWeights)
: queryEnv(&indexEnv), rankSetup(factory, indexEnv),
- rankProgram(), mdl(), fooHandles(), barHandles()
+ mdl(), match_data(), rankProgram(), fooHandles(), barHandles()
{
for (size_t i = 0; i < fooWeights.size(); ++i) {
uint32_t fieldId = indexEnv.getFieldByName("foo")->id();
@@ -100,8 +101,9 @@ struct RankFixture : BlueprintFactoryFixture, IndexFixture {
rankSetup.setFirstPhaseRank(featureName);
rankSetup.setIgnoreDefaultRankFeatures(true);
ASSERT_TRUE(rankSetup.compile());
+ match_data = mdl.createMatchData();
rankProgram = rankSetup.create_first_phase_program();
- rankProgram->setup(mdl, queryEnv);
+ rankProgram->setup(*match_data, queryEnv);
}
feature_t getScore(uint32_t docId) {
return Utils::getScoreFeature(*rankProgram, docId);
@@ -110,15 +112,15 @@ struct RankFixture : BlueprintFactoryFixture, IndexFixture {
ASSERT_LESS(i, fooHandles.size());
TermFieldMatchDataPosition pos;
pos.setElementWeight(index_weight);
- rankProgram->match_data().resolveTermField(fooHandles[i])->reset(docId);
- rankProgram->match_data().resolveTermField(fooHandles[i])->appendPosition(pos);
+ match_data->resolveTermField(fooHandles[i])->reset(docId);
+ match_data->resolveTermField(fooHandles[i])->appendPosition(pos);
}
void setBarWeight(uint32_t i, uint32_t docId, int32_t index_weight) {
ASSERT_LESS(i, barHandles.size());
TermFieldMatchDataPosition pos;
pos.setElementWeight(index_weight);
- rankProgram->match_data().resolveTermField(barHandles[i])->reset(docId);
- rankProgram->match_data().resolveTermField(barHandles[i])->appendPosition(pos);
+ match_data->resolveTermField(barHandles[i])->reset(docId);
+ match_data->resolveTermField(barHandles[i])->appendPosition(pos);
}
};
diff --git a/searchlib/src/tests/features/raw_score/raw_score_test.cpp b/searchlib/src/tests/features/raw_score/raw_score_test.cpp
index 52587c0d8f9..4aad6c6896e 100644
--- a/searchlib/src/tests/features/raw_score/raw_score_test.cpp
+++ b/searchlib/src/tests/features/raw_score/raw_score_test.cpp
@@ -45,13 +45,14 @@ struct FeatureDumpFixture : public IDumpFeatureVisitor {
struct RankFixture : BlueprintFactoryFixture, IndexFixture {
QueryEnvironment queryEnv;
RankSetup rankSetup;
- RankProgram::UP rankProgram;
MatchDataLayout mdl;
+ MatchData::UP match_data;
+ RankProgram::UP rankProgram;
std::vector<TermFieldHandle> fooHandles;
std::vector<TermFieldHandle> barHandles;
RankFixture(size_t fooCnt, size_t barCnt)
: queryEnv(&indexEnv), rankSetup(factory, indexEnv),
- rankProgram(), mdl(), fooHandles(), barHandles()
+ mdl(), match_data(), rankProgram(), fooHandles(), barHandles()
{
for (size_t i = 0; i < fooCnt; ++i) {
uint32_t fieldId = indexEnv.getFieldByName("foo")->id();
@@ -70,14 +71,15 @@ struct RankFixture : BlueprintFactoryFixture, IndexFixture {
rankSetup.setFirstPhaseRank(featureName);
rankSetup.setIgnoreDefaultRankFeatures(true);
ASSERT_TRUE(rankSetup.compile());
+ match_data = mdl.createMatchData();
rankProgram = rankSetup.create_first_phase_program();
- rankProgram->setup(mdl, queryEnv);
+ rankProgram->setup(*match_data, queryEnv);
}
feature_t getScore(uint32_t docId) {
return Utils::getScoreFeature(*rankProgram, docId);
}
void setScore(TermFieldHandle handle, uint32_t docId, feature_t score) {
- rankProgram->match_data().resolveTermField(handle)->setRawScore(docId, score);
+ match_data->resolveTermField(handle)->setRawScore(docId, score);
}
void setFooScore(uint32_t i, uint32_t docId, feature_t score) {
ASSERT_LESS(i, fooHandles.size());
diff --git a/searchlib/src/tests/features/subqueries/subqueries_test.cpp b/searchlib/src/tests/features/subqueries/subqueries_test.cpp
index 37e43b4d2b0..d275cf134c6 100644
--- a/searchlib/src/tests/features/subqueries/subqueries_test.cpp
+++ b/searchlib/src/tests/features/subqueries/subqueries_test.cpp
@@ -43,22 +43,24 @@ struct FeatureDumpFixture : public IDumpFeatureVisitor {
struct RankFixture : BlueprintFactoryFixture, IndexFixture {
QueryEnvironment queryEnv;
RankSetup rankSetup;
- RankProgram::UP rankProgram;
MatchDataLayout mdl;
+ MatchData::UP match_data;
+ RankProgram::UP rankProgram;
std::vector<TermFieldHandle> fooHandles;
std::vector<TermFieldHandle> barHandles;
RankFixture(size_t fooCnt, size_t barCnt,
std::string featureName = "subqueries(foo)")
: queryEnv(&indexEnv), rankSetup(factory, indexEnv),
- rankProgram(), mdl(), fooHandles(), barHandles()
+ mdl(), match_data(), rankProgram(), fooHandles(), barHandles()
{
fooHandles = addFields(fooCnt, indexEnv.getFieldByName("foo")->id());
barHandles = addFields(barCnt, indexEnv.getFieldByName("bar")->id());
rankSetup.setFirstPhaseRank(featureName);
rankSetup.setIgnoreDefaultRankFeatures(true);
ASSERT_TRUE(rankSetup.compile());
+ match_data = mdl.createMatchData();
rankProgram = rankSetup.create_first_phase_program();
- rankProgram->setup(mdl, queryEnv);
+ rankProgram->setup(*match_data, queryEnv);
}
std::vector<TermFieldHandle> addFields(size_t count, uint32_t fieldId) {
std::vector<TermFieldHandle> handles;
@@ -75,7 +77,7 @@ struct RankFixture : BlueprintFactoryFixture, IndexFixture {
}
void setSubqueries(TermFieldHandle handle, uint32_t docId,
uint64_t subqueries) {
- rankProgram->match_data().resolveTermField(handle)->setSubqueries(docId, subqueries);
+ match_data->resolveTermField(handle)->setSubqueries(docId, subqueries);
}
void setFooSubqueries(uint32_t i, uint32_t docId, uint64_t subqueries) {
ASSERT_LESS(i, fooHandles.size());
diff --git a/searchlib/src/tests/fef/featureoverride/featureoverride.cpp b/searchlib/src/tests/fef/featureoverride/featureoverride.cpp
index 0ff7e0899e9..e55bd7081a1 100644
--- a/searchlib/src/tests/fef/featureoverride/featureoverride.cpp
+++ b/searchlib/src/tests/fef/featureoverride/featureoverride.cpp
@@ -141,7 +141,8 @@ TEST("test overrides")
overrides.add("value(1,2,3).2", "6.0");
overrides.add("bogus(feature)", "10.0");
- rankProgram->setup(mdl, queryEnv, overrides);
+ MatchData::UP match_data = mdl.createMatchData();
+ rankProgram->setup(*match_data, queryEnv, overrides);
std::map<vespalib::string, feature_t> res = Utils::getAllFeatures(*rankProgram, 2);
diff --git a/searchlib/src/tests/fef/object_passing/object_passing_test.cpp b/searchlib/src/tests/fef/object_passing/object_passing_test.cpp
index 5c0f09a1203..3ae5932889a 100644
--- a/searchlib/src/tests/fef/object_passing/object_passing_test.cpp
+++ b/searchlib/src/tests/fef/object_passing/object_passing_test.cpp
@@ -7,6 +7,7 @@
#include <vespa/searchlib/fef/test/indexenvironment.h>
#include <vespa/searchlib/fef/test/queryenvironment.h>
#include <vespa/searchlib/fef/test/plugin/unbox.h>
+#include <vespa/searchlib/fef/matchdatalayout.h>
#include <vespa/searchlib/fef/rank_program.h>
#include <vespa/searchlib/fef/verify_feature.h>
#include <vespa/eval/eval/value_type.h>
@@ -87,10 +88,11 @@ struct Fixture {
return vespalib::eval::error_value;
}
MatchDataLayout mdl;
+ MatchData::UP md = mdl.createMatchData();
QueryEnvironment queryEnv(&indexEnv);
Properties overrides;
RankProgram program(resolver);
- program.setup(mdl, queryEnv, overrides);
+ program.setup(*md, queryEnv, overrides);
auto result = program.get_seeds();
EXPECT_EQUAL(1u, result.num_features());
EXPECT_TRUE(!result.is_object(0)); // verifies auto-unboxing
diff --git a/searchlib/src/tests/fef/rank_program/rank_program_test.cpp b/searchlib/src/tests/fef/rank_program/rank_program_test.cpp
index 8e23731e961..7e28178e5f7 100644
--- a/searchlib/src/tests/fef/rank_program/rank_program_test.cpp
+++ b/searchlib/src/tests/fef/rank_program/rank_program_test.cpp
@@ -6,6 +6,7 @@
#include <vespa/searchlib/features/rankingexpressionfeature.h>
#include <vespa/searchlib/fef/blueprintfactory.h>
#include <vespa/searchlib/fef/indexproperties.h>
+#include <vespa/searchlib/fef/matchdatalayout.h>
#include <vespa/searchlib/fef/test/indexenvironment.h>
#include <vespa/searchlib/fef/test/queryenvironment.h>
#include <vespa/searchlib/fef/test/plugin/sum.h>
@@ -69,10 +70,11 @@ struct Fixture {
IndexEnvironment indexEnv;
BlueprintResolver::SP resolver;
Properties overrides;
+ MatchData::UP match_data;
RankProgram program;
size_t track_cnt;
Fixture() : factory(), indexEnv(), resolver(new BlueprintResolver(factory, indexEnv)),
- overrides(), program(resolver), track_cnt(0)
+ overrides(), match_data(), program(resolver), track_cnt(0)
{
factory.addPrototype(Blueprint::SP(new BoxingBlueprint()));
factory.addPrototype(Blueprint::SP(new DocidBlueprint()));
@@ -107,7 +109,8 @@ struct Fixture {
ASSERT_TRUE(resolver->compile());
MatchDataLayout mdl;
QueryEnvironment queryEnv(&indexEnv);
- program.setup(mdl, queryEnv, overrides);
+ match_data = mdl.createMatchData();
+ program.setup(*match_data, queryEnv, overrides);
return *this;
}
double get(uint32_t docid = default_docid) {
diff --git a/searchlib/src/tests/fef/termfieldmodel/termfieldmodel_test.cpp b/searchlib/src/tests/fef/termfieldmodel/termfieldmodel_test.cpp
index 57fb19b7b23..01c72497246 100644
--- a/searchlib/src/tests/fef/termfieldmodel/termfieldmodel_test.cpp
+++ b/searchlib/src/tests/fef/termfieldmodel/termfieldmodel_test.cpp
@@ -207,4 +207,39 @@ TEST("Access subqueries") {
EXPECT_EQUAL(0ULL, state.f3->getSubqueries());
}
+TEST("require that TermFieldMatchData can be tagged as needed or not") {
+ TermFieldMatchData tfmd;
+ tfmd.setFieldId(123);
+ EXPECT_EQUAL(tfmd.getFieldId(),123u);
+ EXPECT_TRUE(!tfmd.isNotNeeded());
+ tfmd.tagAsNotNeeded();
+ EXPECT_EQUAL(tfmd.getFieldId(),123u);
+ EXPECT_TRUE(tfmd.isNotNeeded());
+ tfmd.tagAsNeeded();
+ EXPECT_EQUAL(tfmd.getFieldId(),123u);
+ EXPECT_TRUE(!tfmd.isNotNeeded());
+}
+
+TEST("require that MatchData soft_reset retains appropriate state") {
+ auto md = MatchData::makeTestInstance(10, 10);
+ md->set_termwise_limit(0.5);
+ auto *old_term = md->resolveTermField(7);
+ old_term->tagAsNotNeeded();
+ old_term->populate_fixed()->setElementWeight(21);
+ old_term->resetOnlyDocId(42);
+ EXPECT_EQUAL(md->get_termwise_limit(), 0.5);
+ EXPECT_TRUE(old_term->isNotNeeded());
+ EXPECT_EQUAL(old_term->getFieldId(), 7u);
+ EXPECT_EQUAL(old_term->getWeight(), 21);
+ EXPECT_EQUAL(old_term->getDocId(), 42u);
+ md->soft_reset();
+ auto *new_term = md->resolveTermField(7);
+ EXPECT_EQUAL(new_term, old_term);
+ EXPECT_EQUAL(md->get_termwise_limit(), 1.0);
+ EXPECT_TRUE(!new_term->isNotNeeded());
+ EXPECT_EQUAL(new_term->getFieldId(), 7u);
+ EXPECT_EQUAL(new_term->getWeight(), 21);
+ EXPECT_EQUAL(new_term->getDocId(), TermFieldMatchData::invalidId());
+}
+
TEST_MAIN() { TEST_RUN_ALL(); }
diff --git a/searchlib/src/tests/ranksetup/ranksetup_test.cpp b/searchlib/src/tests/ranksetup/ranksetup_test.cpp
index c2ef8f3a46b..7a26180eed2 100644
--- a/searchlib/src/tests/ranksetup/ranksetup_test.cpp
+++ b/searchlib/src/tests/ranksetup/ranksetup_test.cpp
@@ -87,6 +87,7 @@ private:
const RankEnvironment & _rankEnv;
MatchDataLayout _layout;
std::unique_ptr<RankSetup> _rs;
+ MatchData::UP _match_data;
RankProgram::UP _firstPhaseProgram;
RankProgram::UP _secondPhaseProgram;
@@ -100,7 +101,7 @@ public:
RankExecutor::RankExecutor(const vespalib::string &initRank, const vespalib::string &finalRank,
const RankEnvironment &rankEnv)
: _initRank(initRank), _finalRank(finalRank), _rankEnv(rankEnv), _layout(),
- _rs(), _firstPhaseProgram(), _secondPhaseProgram()
+ _rs(), _match_data(), _firstPhaseProgram(), _secondPhaseProgram()
{}
RankExecutor::~RankExecutor() {}
@@ -121,12 +122,13 @@ RankExecutor::setup()
if (!_rs->compile()) {
return false;
}
+ _match_data = _layout.createMatchData();
_firstPhaseProgram = _rs->create_first_phase_program();
- _firstPhaseProgram->setup(_layout, _rankEnv.queryEnvironment());
+ _firstPhaseProgram->setup(*_match_data, _rankEnv.queryEnvironment());
if (!_finalRank.empty()) {
_secondPhaseProgram = _rs->create_second_phase_program();
- _secondPhaseProgram->setup(_layout, _rankEnv.queryEnvironment());
+ _secondPhaseProgram->setup(*_match_data, _rankEnv.queryEnvironment());
}
return true;
}
@@ -154,6 +156,7 @@ private:
const RankEnvironment & _rankEnv;
RankSetup _setup;
MatchDataLayout _layout;
+ MatchData::UP _match_data;
RankProgram::UP _rankProgram;
public:
@@ -169,6 +172,7 @@ FeatureDumper::FeatureDumper(const RankEnvironment & rankEnv)
: _rankEnv(rankEnv),
_setup(_rankEnv.factory(), _rankEnv.indexEnvironment()),
_layout(),
+ _match_data(),
_rankProgram()
{}
FeatureDumper::~FeatureDumper() {}
@@ -191,8 +195,9 @@ FeatureDumper::setup()
return false;
}
+ _match_data = _layout.createMatchData();
_rankProgram = _setup.create_dump_program();
- _rankProgram->setup(_layout, _rankEnv.queryEnvironment());
+ _rankProgram->setup(*_match_data, _rankEnv.queryEnvironment());
return true;
}
@@ -780,12 +785,13 @@ RankSetupTest::testFeatureNormalization()
{ // RANK context
MatchDataLayout layout;
QueryEnvironment queryEnv;
+ MatchData::UP match_data = layout.createMatchData();
RankProgram::UP firstPhaseProgram = rankSetup.create_first_phase_program();
RankProgram::UP secondPhaseProgram = rankSetup.create_second_phase_program();
RankProgram::UP summaryProgram = rankSetup.create_summary_program();
- firstPhaseProgram->setup(layout, queryEnv);
- secondPhaseProgram->setup(layout, queryEnv);
- summaryProgram->setup(layout, queryEnv);
+ firstPhaseProgram->setup(*match_data, queryEnv);
+ secondPhaseProgram->setup(*match_data, queryEnv);
+ summaryProgram->setup(*match_data, queryEnv);
EXPECT_APPROX(2.0, Utils::getScoreFeature(*firstPhaseProgram, 1), 0.001);
EXPECT_APPROX(4.0, Utils::getScoreFeature(*secondPhaseProgram, 1), 0.001);
@@ -831,8 +837,9 @@ RankSetupTest::testFeatureNormalization()
{ // DUMP context
MatchDataLayout layout;
QueryEnvironment queryEnv;
+ MatchData::UP match_data = layout.createMatchData();
RankProgram::UP rankProgram = rankSetup.create_dump_program();
- rankProgram->setup(layout, queryEnv);
+ rankProgram->setup(*match_data, queryEnv);
{ // dump seed features
std::map<vespalib::string, feature_t> actual = Utils::getSeedFeatures(*rankProgram, 1);
diff --git a/searchlib/src/vespa/searchlib/features/attributematchfeature.cpp b/searchlib/src/vespa/searchlib/features/attributematchfeature.cpp
index bf3297f3e14..2ff53951d8b 100644
--- a/searchlib/src/vespa/searchlib/features/attributematchfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/attributematchfeature.cpp
@@ -253,7 +253,7 @@ AttributeMatchExecutor<T>::execute(uint32_t docId)
template <typename T>
void
-AttributeMatchExecutor<T>::handle_bind_match_data(MatchData &md)
+AttributeMatchExecutor<T>::handle_bind_match_data(const MatchData &md)
{
_cmp.bind_match_data(md);
}
diff --git a/searchlib/src/vespa/searchlib/features/attributematchfeature.h b/searchlib/src/vespa/searchlib/features/attributematchfeature.h
index 8396493b81d..707572abf9e 100644
--- a/searchlib/src/vespa/searchlib/features/attributematchfeature.h
+++ b/searchlib/src/vespa/searchlib/features/attributematchfeature.h
@@ -69,12 +69,12 @@ private:
feature_t getWeight() const;
feature_t getSignificance() const;
feature_t getImportance() const { return (getWeight() + getSignificance()) * 0.5; }
- void bind_match_data(fef::MatchData &md) { _md = &md; }
+ void bind_match_data(const fef::MatchData &md) { _md = &md; }
};
Computer _cmp;
- virtual void handle_bind_match_data(fef::MatchData &md) override;
+ virtual void handle_bind_match_data(const fef::MatchData &md) override;
public:
/**
diff --git a/searchlib/src/vespa/searchlib/features/element_completeness_feature.cpp b/searchlib/src/vespa/searchlib/features/element_completeness_feature.cpp
index c3bb87379ad..18988281cbd 100644
--- a/searchlib/src/vespa/searchlib/features/element_completeness_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/element_completeness_feature.cpp
@@ -73,7 +73,7 @@ ElementCompletenessExecutor::execute(uint32_t docId)
}
void
-ElementCompletenessExecutor::handle_bind_match_data(fef::MatchData &md)
+ElementCompletenessExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/element_completeness_feature.h b/searchlib/src/vespa/searchlib/features/element_completeness_feature.h
index 013efaedc22..1b835da45d2 100644
--- a/searchlib/src/vespa/searchlib/features/element_completeness_feature.h
+++ b/searchlib/src/vespa/searchlib/features/element_completeness_feature.h
@@ -82,7 +82,7 @@ private:
static bool nextElement(Item &item);
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
ElementCompletenessExecutor(const fef::IQueryEnvironment &env,
diff --git a/searchlib/src/vespa/searchlib/features/element_similarity_feature.cpp b/searchlib/src/vespa/searchlib/features/element_similarity_feature.cpp
index e3c0217be15..c9eab238f82 100644
--- a/searchlib/src/vespa/searchlib/features/element_similarity_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/element_similarity_feature.cpp
@@ -230,7 +230,7 @@ public:
bool isPure() override { return _terms.handles.empty(); }
- void handle_bind_match_data(fef::MatchData &md) override {
+ void handle_bind_match_data(const fef::MatchData &md) override {
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/fieldinfofeature.cpp b/searchlib/src/vespa/searchlib/features/fieldinfofeature.cpp
index 85f3fc9b652..960dcc7dfd1 100644
--- a/searchlib/src/vespa/searchlib/features/fieldinfofeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/fieldinfofeature.cpp
@@ -59,7 +59,7 @@ IndexFieldInfoExecutor::execute(uint32_t docId)
}
void
-IndexFieldInfoExecutor::handle_bind_match_data(fef::MatchData &md)
+IndexFieldInfoExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
@@ -98,7 +98,7 @@ AttrFieldInfoExecutor::execute(uint32_t docId)
}
void
-AttrFieldInfoExecutor::handle_bind_match_data(fef::MatchData &md)
+AttrFieldInfoExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/fieldinfofeature.h b/searchlib/src/vespa/searchlib/features/fieldinfofeature.h
index 08cf99ed32a..7ced5d5cb9f 100644
--- a/searchlib/src/vespa/searchlib/features/fieldinfofeature.h
+++ b/searchlib/src/vespa/searchlib/features/fieldinfofeature.h
@@ -18,7 +18,7 @@ private:
uint32_t _fieldHandle;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
IndexFieldInfoExecutor(feature_t type, feature_t isFilter,
@@ -35,7 +35,7 @@ private:
uint32_t _fieldHandle;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
AttrFieldInfoExecutor(feature_t type, uint32_t fieldHandle);
diff --git a/searchlib/src/vespa/searchlib/features/fieldlengthfeature.cpp b/searchlib/src/vespa/searchlib/features/fieldlengthfeature.cpp
index 0ec1f8bd038..d0680e8fc19 100644
--- a/searchlib/src/vespa/searchlib/features/fieldlengthfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/fieldlengthfeature.cpp
@@ -56,7 +56,7 @@ FieldLengthExecutor::execute(uint32_t docId)
}
void
-FieldLengthExecutor::handle_bind_match_data(MatchData &md)
+FieldLengthExecutor::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/fieldlengthfeature.h b/searchlib/src/vespa/searchlib/features/fieldlengthfeature.h
index a818a65d26e..4988bb97546 100644
--- a/searchlib/src/vespa/searchlib/features/fieldlengthfeature.h
+++ b/searchlib/src/vespa/searchlib/features/fieldlengthfeature.h
@@ -15,7 +15,7 @@ private:
std::vector<fef::TermFieldHandle> _fieldHandles;
const fef::MatchData *_md;
- virtual void handle_bind_match_data(fef::MatchData &md) override;
+ virtual void handle_bind_match_data(const fef::MatchData &md) override;
public:
/**
diff --git a/searchlib/src/vespa/searchlib/features/fieldmatchfeature.cpp b/searchlib/src/vespa/searchlib/features/fieldmatchfeature.cpp
index bea6e032e81..f5f8652461e 100644
--- a/searchlib/src/vespa/searchlib/features/fieldmatchfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/fieldmatchfeature.cpp
@@ -86,7 +86,7 @@ FieldMatchExecutor::execute(uint32_t docId)
}
void
-FieldMatchExecutor::handle_bind_match_data(fef::MatchData &md)
+FieldMatchExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_splitter.bind_match_data(md);
}
diff --git a/searchlib/src/vespa/searchlib/features/fieldmatchfeature.h b/searchlib/src/vespa/searchlib/features/fieldmatchfeature.h
index 287af60b021..71ac1023df6 100644
--- a/searchlib/src/vespa/searchlib/features/fieldmatchfeature.h
+++ b/searchlib/src/vespa/searchlib/features/fieldmatchfeature.h
@@ -19,7 +19,7 @@ private:
const fieldmatch::Params & _params;
fieldmatch::Computer _cmp;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
/**
diff --git a/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.cpp b/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.cpp
index 0f71d5dcbec..a7a00bee956 100644
--- a/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.cpp
@@ -63,7 +63,7 @@ FieldTermMatchExecutor::execute(uint32_t docId)
}
void
-FieldTermMatchExecutor::handle_bind_match_data(fef::MatchData &md)
+FieldTermMatchExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.h b/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.h
index 02dfa17ba33..0f89f7eeb75 100644
--- a/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.h
+++ b/searchlib/src/vespa/searchlib/features/fieldtermmatchfeature.h
@@ -27,7 +27,7 @@ private:
fef::TermFieldHandle _fieldHandle;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
};
/**
diff --git a/searchlib/src/vespa/searchlib/features/flow_completeness_feature.cpp b/searchlib/src/vespa/searchlib/features/flow_completeness_feature.cpp
index 5c93566698e..c46d602983d 100644
--- a/searchlib/src/vespa/searchlib/features/flow_completeness_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/flow_completeness_feature.cpp
@@ -234,7 +234,7 @@ FlowCompletenessExecutor::execute(uint32_t)
}
void
-FlowCompletenessExecutor::handle_bind_match_data(fef::MatchData &md)
+FlowCompletenessExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/flow_completeness_feature.h b/searchlib/src/vespa/searchlib/features/flow_completeness_feature.h
index cdfa0356307..db5c53beac8 100644
--- a/searchlib/src/vespa/searchlib/features/flow_completeness_feature.h
+++ b/searchlib/src/vespa/searchlib/features/flow_completeness_feature.h
@@ -62,7 +62,7 @@ private:
static bool nextElement(Item &item);
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
FlowCompletenessExecutor(const fef::IQueryEnvironment &env,
diff --git a/searchlib/src/vespa/searchlib/features/item_raw_score_feature.cpp b/searchlib/src/vespa/searchlib/features/item_raw_score_feature.cpp
index a7476185000..45baf646656 100644
--- a/searchlib/src/vespa/searchlib/features/item_raw_score_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/item_raw_score_feature.cpp
@@ -23,7 +23,7 @@ ItemRawScoreExecutor::execute(uint32_t docId)
}
void
-ItemRawScoreExecutor::handle_bind_match_data(MatchData &md)
+ItemRawScoreExecutor::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
@@ -42,7 +42,7 @@ SimpleItemRawScoreExecutor::execute(uint32_t docId)
}
void
-SimpleItemRawScoreExecutor::handle_bind_match_data(MatchData &md)
+SimpleItemRawScoreExecutor::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/item_raw_score_feature.h b/searchlib/src/vespa/searchlib/features/item_raw_score_feature.h
index 3aa1672b05d..d6a55f29632 100644
--- a/searchlib/src/vespa/searchlib/features/item_raw_score_feature.h
+++ b/searchlib/src/vespa/searchlib/features/item_raw_score_feature.h
@@ -16,7 +16,7 @@ private:
HandleVector _handles;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
ItemRawScoreExecutor(HandleVector handles)
@@ -30,7 +30,7 @@ private:
fef::TermFieldHandle _handle;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
SimpleItemRawScoreExecutor(fef::TermFieldHandle handle)
diff --git a/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.cpp b/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.cpp
index 3f4817d61d2..a5e3e2da5ba 100644
--- a/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.cpp
@@ -59,7 +59,7 @@ JaroWinklerDistanceExecutor::execute(uint32_t docId)
}
void
-JaroWinklerDistanceExecutor::handle_bind_match_data(fef::MatchData &md)
+JaroWinklerDistanceExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.h b/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.h
index 0497d2a2f73..a8d6eacf0eb 100644
--- a/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.h
+++ b/searchlib/src/vespa/searchlib/features/jarowinklerdistancefeature.h
@@ -43,7 +43,7 @@ private:
std::vector<fef::TermFieldHandle> _termFieldHandles; // The handles of all query terms.
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
};
/**
diff --git a/searchlib/src/vespa/searchlib/features/matchcountfeature.cpp b/searchlib/src/vespa/searchlib/features/matchcountfeature.cpp
index 51a7a952781..fd453e17eb1 100644
--- a/searchlib/src/vespa/searchlib/features/matchcountfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/matchcountfeature.cpp
@@ -36,7 +36,7 @@ MatchCountExecutor::execute(uint32_t docId)
}
void
-MatchCountExecutor::handle_bind_match_data(MatchData &md)
+MatchCountExecutor::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/matchcountfeature.h b/searchlib/src/vespa/searchlib/features/matchcountfeature.h
index c4123016e16..eadb5064c57 100644
--- a/searchlib/src/vespa/searchlib/features/matchcountfeature.h
+++ b/searchlib/src/vespa/searchlib/features/matchcountfeature.h
@@ -18,7 +18,7 @@ private:
std::vector<fef::TermFieldHandle> _handles;
const fef::MatchData *_md;
- virtual void handle_bind_match_data(fef::MatchData &md) override;
+ virtual void handle_bind_match_data(const fef::MatchData &md) override;
public:
MatchCountExecutor(uint32_t fieldId, const fef::IQueryEnvironment &env);
diff --git a/searchlib/src/vespa/searchlib/features/matchesfeature.cpp b/searchlib/src/vespa/searchlib/features/matchesfeature.cpp
index c2e36551648..f4788ee74c8 100644
--- a/searchlib/src/vespa/searchlib/features/matchesfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/matchesfeature.cpp
@@ -40,7 +40,7 @@ MatchesExecutor::execute(uint32_t docId)
}
void
-MatchesExecutor::handle_bind_match_data(MatchData &md)
+MatchesExecutor::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/matchesfeature.h b/searchlib/src/vespa/searchlib/features/matchesfeature.h
index fe82929d2a7..c6366349dd9 100644
--- a/searchlib/src/vespa/searchlib/features/matchesfeature.h
+++ b/searchlib/src/vespa/searchlib/features/matchesfeature.h
@@ -18,7 +18,7 @@ private:
std::vector<fef::TermFieldHandle> _handles;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
MatchesExecutor(uint32_t fieldId,
diff --git a/searchlib/src/vespa/searchlib/features/native_dot_product_feature.cpp b/searchlib/src/vespa/searchlib/features/native_dot_product_feature.cpp
index 6986703eaf6..cb8136e8b7f 100644
--- a/searchlib/src/vespa/searchlib/features/native_dot_product_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/native_dot_product_feature.cpp
@@ -35,7 +35,7 @@ NativeDotProductExecutor::execute(uint32_t docId)
}
void
-NativeDotProductExecutor::handle_bind_match_data(fef::MatchData &md)
+NativeDotProductExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/native_dot_product_feature.h b/searchlib/src/vespa/searchlib/features/native_dot_product_feature.h
index cc29ecd5e6b..a71d23f3158 100644
--- a/searchlib/src/vespa/searchlib/features/native_dot_product_feature.h
+++ b/searchlib/src/vespa/searchlib/features/native_dot_product_feature.h
@@ -17,7 +17,7 @@ private:
std::vector<Pair> _pairs;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
NativeDotProductExecutor(const fef::IQueryEnvironment &env, uint32_t fieldId);
diff --git a/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.cpp b/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.cpp
index 4243bb83e03..1e6423f9de8 100644
--- a/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.cpp
@@ -73,7 +73,7 @@ NativeAttributeMatchExecutorMulti::execute(uint32_t docId)
}
void
-NativeAttributeMatchExecutorMulti::handle_bind_match_data(MatchData &md)
+NativeAttributeMatchExecutorMulti::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
@@ -88,7 +88,7 @@ NativeAttributeMatchExecutorSingle::execute(uint32_t docId)
}
void
-NativeAttributeMatchExecutorSingle::handle_bind_match_data(MatchData &md)
+NativeAttributeMatchExecutorSingle::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.h b/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.h
index 4ebdc871bb5..e47cbed0344 100644
--- a/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.h
+++ b/searchlib/src/vespa/searchlib/features/nativeattributematchfeature.h
@@ -57,7 +57,7 @@ private:
std::vector<CachedTermData> _queryTermData;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
NativeAttributeMatchExecutorMulti(const Precomputed & setup) : _divisor(setup.second), _queryTermData(setup.first), _md(nullptr) { }
void execute(uint32_t docId) override;
@@ -69,7 +69,7 @@ private:
CachedTermData _queryTermData;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
NativeAttributeMatchExecutorSingle(const Precomputed & setup) :
diff --git a/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.cpp b/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.cpp
index e4b633d54dd..84ad17e1cb0 100644
--- a/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.cpp
@@ -83,7 +83,7 @@ NativeFieldMatchExecutor::execute(uint32_t docId)
}
void
-NativeFieldMatchExecutor::handle_bind_match_data(fef::MatchData &md)
+NativeFieldMatchExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.h b/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.h
index d47dece1654..cb676e5b775 100644
--- a/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.h
+++ b/searchlib/src/vespa/searchlib/features/nativefieldmatchfeature.h
@@ -73,7 +73,7 @@ private:
return table->get(index);
}
- virtual void handle_bind_match_data(fef::MatchData &md) override;
+ virtual void handle_bind_match_data(const fef::MatchData &md) override;
public:
NativeFieldMatchExecutor(const fef::IQueryEnvironment & env,
diff --git a/searchlib/src/vespa/searchlib/features/nativeproximityfeature.cpp b/searchlib/src/vespa/searchlib/features/nativeproximityfeature.cpp
index 3c92db6b937..a31d9207e05 100644
--- a/searchlib/src/vespa/searchlib/features/nativeproximityfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/nativeproximityfeature.cpp
@@ -97,7 +97,7 @@ NativeProximityExecutor::execute(uint32_t docId)
}
void
-NativeProximityExecutor::handle_bind_match_data(fef::MatchData &md)
+NativeProximityExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/nativeproximityfeature.h b/searchlib/src/vespa/searchlib/features/nativeproximityfeature.h
index cda6776d42e..f98fe2bfd22 100644
--- a/searchlib/src/vespa/searchlib/features/nativeproximityfeature.h
+++ b/searchlib/src/vespa/searchlib/features/nativeproximityfeature.h
@@ -62,7 +62,7 @@ private:
feature_t calculateScoreForField(const FieldSetup & fs, uint32_t docId);
feature_t calculateScoreForPair(const TermPair & pair, uint32_t fieldId, uint32_t docId);
- virtual void handle_bind_match_data(fef::MatchData &md) override;
+ virtual void handle_bind_match_data(const fef::MatchData &md) override;
public:
NativeProximityExecutor(const fef::IQueryEnvironment & env, const NativeProximityParams & params);
diff --git a/searchlib/src/vespa/searchlib/features/proximityfeature.cpp b/searchlib/src/vespa/searchlib/features/proximityfeature.cpp
index e6303ace100..f625e30f378 100644
--- a/searchlib/src/vespa/searchlib/features/proximityfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/proximityfeature.cpp
@@ -51,7 +51,7 @@ ProximityExecutor::execute(uint32_t docId)
}
void
-ProximityExecutor::handle_bind_match_data(fef::MatchData &md)
+ProximityExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/proximityfeature.h b/searchlib/src/vespa/searchlib/features/proximityfeature.h
index 5a46bafe869..14a80628004 100644
--- a/searchlib/src/vespa/searchlib/features/proximityfeature.h
+++ b/searchlib/src/vespa/searchlib/features/proximityfeature.h
@@ -41,7 +41,7 @@ private:
bool findBest(const fef::TermFieldMatchData &matchA,
const fef::TermFieldMatchData &matchB);
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
};
/**
diff --git a/searchlib/src/vespa/searchlib/features/querycompletenessfeature.cpp b/searchlib/src/vespa/searchlib/features/querycompletenessfeature.cpp
index af44f75e33f..b4b6a1b0eb4 100644
--- a/searchlib/src/vespa/searchlib/features/querycompletenessfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/querycompletenessfeature.cpp
@@ -62,7 +62,7 @@ QueryCompletenessExecutor::execute(uint32_t docId)
}
void
-QueryCompletenessExecutor::handle_bind_match_data(fef::MatchData &md)
+QueryCompletenessExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/querycompletenessfeature.h b/searchlib/src/vespa/searchlib/features/querycompletenessfeature.h
index 5e30563f6ec..a5e6bbdc2bb 100644
--- a/searchlib/src/vespa/searchlib/features/querycompletenessfeature.h
+++ b/searchlib/src/vespa/searchlib/features/querycompletenessfeature.h
@@ -38,7 +38,7 @@ private:
std::vector<fef::TermFieldHandle> _fieldHandles;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
};
/**
diff --git a/searchlib/src/vespa/searchlib/features/raw_score_feature.cpp b/searchlib/src/vespa/searchlib/features/raw_score_feature.cpp
index 2fa59111d00..02e44e781d4 100644
--- a/searchlib/src/vespa/searchlib/features/raw_score_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/raw_score_feature.cpp
@@ -35,7 +35,7 @@ RawScoreExecutor::execute(uint32_t docId)
}
void
-RawScoreExecutor::handle_bind_match_data(fef::MatchData &md)
+RawScoreExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/raw_score_feature.h b/searchlib/src/vespa/searchlib/features/raw_score_feature.h
index 8bb310fa18e..0eceba16ffe 100644
--- a/searchlib/src/vespa/searchlib/features/raw_score_feature.h
+++ b/searchlib/src/vespa/searchlib/features/raw_score_feature.h
@@ -14,7 +14,7 @@ private:
std::vector<fef::TermFieldHandle> _handles;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
RawScoreExecutor(const fef::IQueryEnvironment &env, uint32_t fieldId);
void execute(uint32_t docId) override;
diff --git a/searchlib/src/vespa/searchlib/features/reverseproximityfeature.cpp b/searchlib/src/vespa/searchlib/features/reverseproximityfeature.cpp
index 0f35e1becf8..c27936332d2 100644
--- a/searchlib/src/vespa/searchlib/features/reverseproximityfeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/reverseproximityfeature.cpp
@@ -90,7 +90,7 @@ ReverseProximityExecutor::execute(uint32_t docId)
}
void
-ReverseProximityExecutor::handle_bind_match_data(fef::MatchData &md)
+ReverseProximityExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/reverseproximityfeature.h b/searchlib/src/vespa/searchlib/features/reverseproximityfeature.h
index c16448435f2..776dde39f77 100644
--- a/searchlib/src/vespa/searchlib/features/reverseproximityfeature.h
+++ b/searchlib/src/vespa/searchlib/features/reverseproximityfeature.h
@@ -39,7 +39,7 @@ private:
fef::TermFieldHandle _termB; // Handle to the second query term.
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
};
/**
diff --git a/searchlib/src/vespa/searchlib/features/subqueries_feature.cpp b/searchlib/src/vespa/searchlib/features/subqueries_feature.cpp
index 4bbb7af9dfb..6c52b6edb76 100644
--- a/searchlib/src/vespa/searchlib/features/subqueries_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/subqueries_feature.cpp
@@ -37,7 +37,7 @@ void SubqueriesExecutor::execute(uint32_t docId) {
}
void
-SubqueriesExecutor::handle_bind_match_data(fef::MatchData &md)
+SubqueriesExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/subqueries_feature.h b/searchlib/src/vespa/searchlib/features/subqueries_feature.h
index 60bec51cb79..136d8d90f94 100644
--- a/searchlib/src/vespa/searchlib/features/subqueries_feature.h
+++ b/searchlib/src/vespa/searchlib/features/subqueries_feature.h
@@ -12,7 +12,7 @@ class SubqueriesExecutor : public fef::FeatureExecutor {
std::vector<fef::TermFieldHandle> _handles;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
SubqueriesExecutor(const fef::IQueryEnvironment &env, uint32_t fieldId);
void execute(uint32_t docId) override;
diff --git a/searchlib/src/vespa/searchlib/features/term_field_md_feature.cpp b/searchlib/src/vespa/searchlib/features/term_field_md_feature.cpp
index 97d267071cf..8b0eecd9cff 100644
--- a/searchlib/src/vespa/searchlib/features/term_field_md_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/term_field_md_feature.cpp
@@ -65,7 +65,7 @@ TermFieldMdExecutor::execute(uint32_t docId)
}
void
-TermFieldMdExecutor::handle_bind_match_data(MatchData &md)
+TermFieldMdExecutor::handle_bind_match_data(const MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/term_field_md_feature.h b/searchlib/src/vespa/searchlib/features/term_field_md_feature.h
index f819989fc08..df9ab982dcb 100644
--- a/searchlib/src/vespa/searchlib/features/term_field_md_feature.h
+++ b/searchlib/src/vespa/searchlib/features/term_field_md_feature.h
@@ -21,7 +21,7 @@ class TermFieldMdExecutor : public fef::FeatureExecutor {
const fef::MatchData *_md;
void execute(uint32_t docId) override;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
TermFieldMdExecutor(const fef::IQueryEnvironment &env, uint32_t fieldId);
};
diff --git a/searchlib/src/vespa/searchlib/features/termdistancefeature.cpp b/searchlib/src/vespa/searchlib/features/termdistancefeature.cpp
index a67c6a0be7c..0408f6d4b75 100644
--- a/searchlib/src/vespa/searchlib/features/termdistancefeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/termdistancefeature.cpp
@@ -42,7 +42,7 @@ TermDistanceExecutor::execute(uint32_t docId)
}
void
-TermDistanceExecutor::handle_bind_match_data(fef::MatchData &md)
+TermDistanceExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/termdistancefeature.h b/searchlib/src/vespa/searchlib/features/termdistancefeature.h
index 5c4542a1f0d..8e153072bc2 100644
--- a/searchlib/src/vespa/searchlib/features/termdistancefeature.h
+++ b/searchlib/src/vespa/searchlib/features/termdistancefeature.h
@@ -31,7 +31,7 @@ private:
QueryTerm _termB;
const fef::MatchData *_md;
- virtual void handle_bind_match_data(fef::MatchData &md) override;
+ virtual void handle_bind_match_data(const fef::MatchData &md) override;
public:
TermDistanceExecutor(const fef::IQueryEnvironment & env,
diff --git a/searchlib/src/vespa/searchlib/features/termeditdistancefeature.cpp b/searchlib/src/vespa/searchlib/features/termeditdistancefeature.cpp
index 05bdbb4475b..5990d62cb25 100644
--- a/searchlib/src/vespa/searchlib/features/termeditdistancefeature.cpp
+++ b/searchlib/src/vespa/searchlib/features/termeditdistancefeature.cpp
@@ -156,7 +156,7 @@ TermEditDistanceExecutor::execute(uint32_t docId)
}
void
-TermEditDistanceExecutor::handle_bind_match_data(fef::MatchData &md)
+TermEditDistanceExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/termeditdistancefeature.h b/searchlib/src/vespa/searchlib/features/termeditdistancefeature.h
index 07d625726e0..ea5974495ea 100644
--- a/searchlib/src/vespa/searchlib/features/termeditdistancefeature.h
+++ b/searchlib/src/vespa/searchlib/features/termeditdistancefeature.h
@@ -88,7 +88,7 @@ private:
*/
void logRow(const std::vector<TedCell> &row, size_t numCols);
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
private:
const TermEditDistanceConfig &_config; // The config for this executor.
diff --git a/searchlib/src/vespa/searchlib/features/text_similarity_feature.cpp b/searchlib/src/vespa/searchlib/features/text_similarity_feature.cpp
index 31876e4f1c9..a50f042b552 100644
--- a/searchlib/src/vespa/searchlib/features/text_similarity_feature.cpp
+++ b/searchlib/src/vespa/searchlib/features/text_similarity_feature.cpp
@@ -158,7 +158,7 @@ TextSimilarityExecutor::execute(uint32_t docId)
}
void
-TextSimilarityExecutor::handle_bind_match_data(fef::MatchData &md)
+TextSimilarityExecutor::handle_bind_match_data(const fef::MatchData &md)
{
_md = &md;
}
diff --git a/searchlib/src/vespa/searchlib/features/text_similarity_feature.h b/searchlib/src/vespa/searchlib/features/text_similarity_feature.h
index e948589fc97..a660f3e3e30 100644
--- a/searchlib/src/vespa/searchlib/features/text_similarity_feature.h
+++ b/searchlib/src/vespa/searchlib/features/text_similarity_feature.h
@@ -35,7 +35,7 @@ private:
vespalib::PriorityQueue<Item> _queue;
const fef::MatchData *_md;
- void handle_bind_match_data(fef::MatchData &md) override;
+ void handle_bind_match_data(const fef::MatchData &md) override;
public:
TextSimilarityExecutor(const fef::IQueryEnvironment &env, uint32_t field_id);
diff --git a/searchlib/src/vespa/searchlib/fef/featureexecutor.cpp b/searchlib/src/vespa/searchlib/fef/featureexecutor.cpp
index f8268d66e47..02d69fbc5ca 100644
--- a/searchlib/src/vespa/searchlib/fef/featureexecutor.cpp
+++ b/searchlib/src/vespa/searchlib/fef/featureexecutor.cpp
@@ -28,7 +28,7 @@ FeatureExecutor::handle_bind_outputs(vespalib::ArrayRef<NumberOrObject>)
}
void
-FeatureExecutor::handle_bind_match_data(MatchData &)
+FeatureExecutor::handle_bind_match_data(const MatchData &)
{
}
@@ -47,7 +47,7 @@ FeatureExecutor::bind_outputs(vespalib::ArrayRef<NumberOrObject> outputs)
}
void
-FeatureExecutor::bind_match_data(MatchData &md)
+FeatureExecutor::bind_match_data(const MatchData &md)
{
handle_bind_match_data(md);
}
diff --git a/searchlib/src/vespa/searchlib/fef/featureexecutor.h b/searchlib/src/vespa/searchlib/fef/featureexecutor.h
index 1d8e4bff0c4..c8219fada3b 100644
--- a/searchlib/src/vespa/searchlib/fef/featureexecutor.h
+++ b/searchlib/src/vespa/searchlib/fef/featureexecutor.h
@@ -98,7 +98,7 @@ private:
protected:
virtual void handle_bind_inputs(vespalib::ConstArrayRef<LazyValue> inputs);
virtual void handle_bind_outputs(vespalib::ArrayRef<NumberOrObject> outputs);
- virtual void handle_bind_match_data(MatchData &md);
+ virtual void handle_bind_match_data(const MatchData &md);
/**
* Execute this feature executor for the given document.
@@ -117,7 +117,7 @@ public:
// bind order per executor: inputs, outputs, match_data
void bind_inputs(vespalib::ConstArrayRef<LazyValue> inputs);
void bind_outputs(vespalib::ArrayRef<NumberOrObject> outputs);
- void bind_match_data(MatchData &md);
+ void bind_match_data(const MatchData &md);
const Inputs &inputs() const { return _inputs; }
const Outputs &outputs() const { return _outputs; }
diff --git a/searchlib/src/vespa/searchlib/fef/featureoverrider.cpp b/searchlib/src/vespa/searchlib/fef/featureoverrider.cpp
index ad623c65c07..5209065fd62 100644
--- a/searchlib/src/vespa/searchlib/fef/featureoverrider.cpp
+++ b/searchlib/src/vespa/searchlib/fef/featureoverrider.cpp
@@ -40,7 +40,7 @@ FeatureOverrider::execute(uint32_t docId)
}
void
-FeatureOverrider::handle_bind_match_data(MatchData &md)
+FeatureOverrider::handle_bind_match_data(const MatchData &md)
{
_executor.bind_match_data(md);
}
diff --git a/searchlib/src/vespa/searchlib/fef/featureoverrider.h b/searchlib/src/vespa/searchlib/fef/featureoverrider.h
index 3f2075ffbcb..89c397abfdb 100644
--- a/searchlib/src/vespa/searchlib/fef/featureoverrider.h
+++ b/searchlib/src/vespa/searchlib/fef/featureoverrider.h
@@ -24,7 +24,7 @@ private:
uint32_t _outputIdx;
feature_t _value;
- virtual void handle_bind_match_data(MatchData &md) override;
+ virtual void handle_bind_match_data(const MatchData &md) override;
virtual void handle_bind_inputs(vespalib::ConstArrayRef<LazyValue> inputs) override;
virtual void handle_bind_outputs(vespalib::ArrayRef<NumberOrObject> outputs) override;
diff --git a/searchlib/src/vespa/searchlib/fef/matchdata.cpp b/searchlib/src/vespa/searchlib/fef/matchdata.cpp
index 4ce2a7c9299..0c589749112 100644
--- a/searchlib/src/vespa/searchlib/fef/matchdata.cpp
+++ b/searchlib/src/vespa/searchlib/fef/matchdata.cpp
@@ -11,6 +11,15 @@ MatchData::MatchData(const Params &cparams)
{
}
+void
+MatchData::soft_reset()
+{
+ for (auto &tfmd: _termFields) {
+ tfmd.resetOnlyDocId(TermFieldMatchData::invalidId()).tagAsNeeded();
+ }
+ _termwise_limit = 1.0;
+}
+
MatchData::UP
MatchData::makeTestInstance(uint32_t numTermFields, uint32_t fieldIdLimit)
{
diff --git a/searchlib/src/vespa/searchlib/fef/matchdata.h b/searchlib/src/vespa/searchlib/fef/matchdata.h
index 472b34a823f..5b06f4d37e0 100644
--- a/searchlib/src/vespa/searchlib/fef/matchdata.h
+++ b/searchlib/src/vespa/searchlib/fef/matchdata.h
@@ -58,6 +58,15 @@ public:
**/
explicit MatchData(const Params &cparams);
+ /**
+ * Reset this match data in such a way that it can be re-used with
+ * either the same search iterator tree or with a new search
+ * iterator tree where the only difference in interaction with the
+ * match data is which terms are unpacked. Note that this will
+ * reset some properties, but not all. Use with caution.
+ **/
+ void soft_reset();
+
MatchData(const MatchData &rhs) = delete;
MatchData & operator=(const MatchData &rhs) = delete;
diff --git a/searchlib/src/vespa/searchlib/fef/rank_program.cpp b/searchlib/src/vespa/searchlib/fef/rank_program.cpp
index fa4582d46b7..33a65b48a1c 100644
--- a/searchlib/src/vespa/searchlib/fef/rank_program.cpp
+++ b/searchlib/src/vespa/searchlib/fef/rank_program.cpp
@@ -109,7 +109,7 @@ RankProgram::run_const(FeatureExecutor *executor)
}
void
-RankProgram::unbox(BlueprintResolver::FeatureRef seed)
+RankProgram::unbox(BlueprintResolver::FeatureRef seed, const MatchData &md)
{
FeatureExecutor *input_executor = _executors[seed.executor];
const NumberOrObject *input_value = input_executor->outputs().get_raw(seed.output);
@@ -122,7 +122,7 @@ RankProgram::unbox(BlueprintResolver::FeatureRef seed)
FeatureExecutor &unboxer = _hot_stash.create<UnboxingExecutor>();
unboxer.bind_inputs(inputs);
unboxer.bind_outputs(outputs);
- unboxer.bind_match_data(*_match_data);
+ unboxer.bind_match_data(md);
_unboxed_seeds.emplace(input_value, LazyValue(&outputs[0], &unboxer));
}
}
@@ -153,7 +153,6 @@ RankProgram::resolve(const BlueprintResolver::FeatureMap &features, bool unbox_s
RankProgram::RankProgram(BlueprintResolver::SP resolver)
: _resolver(resolver),
- _match_data(),
_hot_stash(32768),
_cold_stash(),
_executors(),
@@ -165,12 +164,11 @@ RankProgram::RankProgram(BlueprintResolver::SP resolver)
RankProgram::~RankProgram() {}
void
-RankProgram::setup(const MatchDataLayout &mdl_in,
+RankProgram::setup(const MatchData &md,
const IQueryEnvironment &queryEnv,
const Properties &featureOverrides)
{
assert(_executors.empty());
- _match_data = mdl_in.createMatchData();
std::vector<Override> overrides = prepare_overrides(_resolver->getFeatureMap(), featureOverrides);
auto override = overrides.begin();
auto override_end = overrides.end();
@@ -204,7 +202,7 @@ RankProgram::setup(const MatchDataLayout &mdl_in,
}
executor->bind_inputs(inputs);
executor->bind_outputs(outputs);
- executor->bind_match_data(*_match_data);
+ executor->bind_match_data(md);
_executors.push_back(executor);
if (is_const) {
run_const(executor);
@@ -213,7 +211,7 @@ RankProgram::setup(const MatchDataLayout &mdl_in,
for (const auto &seed_entry: _resolver->getSeedMap()) {
auto seed = seed_entry.second;
if (specs[seed.executor].output_types[seed.output]) {
- unbox(seed);
+ unbox(seed, md);
}
}
assert(_executors.size() == specs.size());
diff --git a/searchlib/src/vespa/searchlib/fef/rank_program.h b/searchlib/src/vespa/searchlib/fef/rank_program.h
index 7cf593e4a4e..3a92fc874a4 100644
--- a/searchlib/src/vespa/searchlib/fef/rank_program.h
+++ b/searchlib/src/vespa/searchlib/fef/rank_program.h
@@ -6,7 +6,6 @@
#include "featureexecutor.h"
#include "properties.h"
#include "matchdata.h"
-#include "matchdatalayout.h"
#include "feature_resolver.h"
#include <vespa/vespalib/stllike/string.h>
#include <vespa/vespalib/util/array.h>
@@ -21,11 +20,10 @@ namespace fef {
* values. In order to access (and thereby calculate) output features
* you typically use the get_seeds function to resolve the predefined
* set of output features. Each feature value will be wrapped in a
- * LazyValue object that can be realized for a specific docid. The
- * rank program also owns the MatchData used to store unpacked
- * term-field match information. Note that you need unpack any
- * relevant posting information into the MatchData object before
- * trying to resolve lazy values.
+ * LazyValue object that can be realized for a specific docid. Note
+ * that you need unpack any relevant posting information into the
+ * MatchData object passed to the setup function before trying to
+ * resolve lazy values.
**/
class RankProgram
{
@@ -37,7 +35,6 @@ private:
using ValueSet = std::set<const NumberOrObject *>;
BlueprintResolver::SP _resolver;
- MatchData::UP _match_data;
vespalib::Stash _hot_stash;
vespalib::Stash _cold_stash;
std::vector<FeatureExecutor *> _executors;
@@ -47,7 +44,7 @@ private:
bool check_const(const NumberOrObject *value) const { return (_is_const.count(value) == 1); }
bool check_const(FeatureExecutor *executor, const std::vector<BlueprintResolver::FeatureRef> &inputs) const;
void run_const(FeatureExecutor *executor);
- void unbox(BlueprintResolver::FeatureRef seed);
+ void unbox(BlueprintResolver::FeatureRef seed, const MatchData &md);
FeatureResolver resolve(const BlueprintResolver::FeatureMap &features, bool unbox_seeds) const;
public:
@@ -66,21 +63,13 @@ public:
/**
* Set up this rank program by creating the needed feature
* executors and wiring them together. This function will also
- * create the MatchData to be used for iterator unpacking as well
- * as pre-calculating all constant features.
+ * pre-calculate all constant features.
**/
- void setup(const MatchDataLayout &mdl,
+ void setup(const MatchData &md,
const IQueryEnvironment &queryEnv,
const Properties &featureOverrides = Properties());
/**
- * Expose the MatchData used when creating search iterators as it
- * is where all iterators should unpack their match information.
- **/
- MatchData &match_data() { return *_match_data; }
- const MatchData &match_data() const { return *_match_data; }
-
- /**
* Obtain the names and storage locations of all seed features for
* this rank program. Programs for ranking phases will only have a
* single seed while programs used for summary features or
diff --git a/searchlib/src/vespa/searchlib/fef/termfieldmatchdata.h b/searchlib/src/vespa/searchlib/fef/termfieldmatchdata.h
index 2d91764f7c9..018af889557 100644
--- a/searchlib/src/vespa/searchlib/fef/termfieldmatchdata.h
+++ b/searchlib/src/vespa/searchlib/fef/termfieldmatchdata.h
@@ -258,6 +258,13 @@ public:
}
/**
+ * Tag that this instance is used for ranking.
+ */
+ void tagAsNeeded() {
+ _fieldId = _fieldId & ~0x2000;
+ }
+
+ /**
* Special docId value indicating that no data has been saved yet.
* This should match (or be above) endId() in search::queryeval::SearchIterator.
*
diff --git a/searchlib/src/vespa/searchlib/fef/test/featuretest.cpp b/searchlib/src/vespa/searchlib/fef/test/featuretest.cpp
index 2652abf97a4..e5a6232ed54 100644
--- a/searchlib/src/vespa/searchlib/fef/test/featuretest.cpp
+++ b/searchlib/src/vespa/searchlib/fef/test/featuretest.cpp
@@ -22,6 +22,7 @@ FeatureTest::FeatureTest(BlueprintFactory &factory,
_layout(layout),
_overrides(overrides),
_resolver(new BlueprintResolver(factory, indexEnv)),
+ _match_data(_layout.createMatchData()),
_rankProgram(new RankProgram(_resolver)),
_doneSetup(false)
{
@@ -42,6 +43,7 @@ FeatureTest::FeatureTest(BlueprintFactory &factory,
_layout(layout),
_overrides(overrides),
_resolver(new BlueprintResolver(factory, indexEnv)),
+ _match_data(_layout.createMatchData()),
_rankProgram(new RankProgram(_resolver)),
_doneSetup(false)
{
@@ -68,7 +70,7 @@ FeatureTest::setup()
return false;
}
- _rankProgram->setup(_layout, _queryEnv, _overrides);
+ _rankProgram->setup(*_match_data, _queryEnv, _overrides);
_doneSetup = true;
return true;
}
@@ -77,7 +79,7 @@ MatchDataBuilder::UP
FeatureTest::createMatchDataBuilder()
{
if (_doneSetup) {
- return MatchDataBuilder::UP(new MatchDataBuilder(_queryEnv, _rankProgram->match_data()));
+ return MatchDataBuilder::UP(new MatchDataBuilder(_queryEnv, *_match_data));
}
LOG(warning, "Match data not initialized.");
return MatchDataBuilder::UP();
@@ -135,6 +137,7 @@ void
FeatureTest::clear()
{
_resolver = BlueprintResolver::SP(new BlueprintResolver(_factory, _indexEnv));
+ _match_data = _layout.createMatchData();
_rankProgram.reset(new RankProgram(_resolver));
_doneSetup = false;
}
diff --git a/searchlib/src/vespa/searchlib/fef/test/featuretest.h b/searchlib/src/vespa/searchlib/fef/test/featuretest.h
index 2bce63d357d..e354884c110 100644
--- a/searchlib/src/vespa/searchlib/fef/test/featuretest.h
+++ b/searchlib/src/vespa/searchlib/fef/test/featuretest.h
@@ -116,6 +116,7 @@ private:
MatchDataLayout &_layout;
const Properties &_overrides;
BlueprintResolver::SP _resolver;
+ MatchData::UP _match_data;
RankProgram::UP _rankProgram;
bool _doneSetup;
diff --git a/searchlib/src/vespa/searchlib/queryeval/termwise_search.cpp b/searchlib/src/vespa/searchlib/queryeval/termwise_search.cpp
index 13b1c8f1da6..e7df5aa6bae 100644
--- a/searchlib/src/vespa/searchlib/queryeval/termwise_search.cpp
+++ b/searchlib/src/vespa/searchlib/queryeval/termwise_search.cpp
@@ -11,17 +11,27 @@ template <bool IS_STRICT>
struct TermwiseSearch : public SearchIterator {
SearchIterator::UP search;
- BitVector::UP result;
+ BitVector::UP result;
+ uint32_t my_beginid;
+ uint32_t my_first_hit;
+
+ bool same_range(uint32_t beginid, uint32_t endid) const {
+ return ((beginid == my_beginid) && endid == getEndId());
+ }
TermwiseSearch(SearchIterator::UP search_in)
- : search(std::move(search_in)), result() {}
+ : search(std::move(search_in)), result(), my_beginid(0), my_first_hit(0) {}
Trinary is_strict() const override { return IS_STRICT ? Trinary::True : Trinary::False; }
void initRange(uint32_t beginid, uint32_t endid) override {
- SearchIterator::initRange(beginid, endid);
- search->initRange(beginid, endid);
- setDocId(std::max(getDocId(), search->getDocId()));
- result = search->get_hits(beginid);
+ if (!same_range(beginid, endid)) {
+ my_beginid = beginid;
+ SearchIterator::initRange(beginid, endid);
+ search->initRange(beginid, endid);
+ my_first_hit = std::max(getDocId(), search->getDocId());
+ result = search->get_hits(beginid);
+ }
+ setDocId(my_first_hit);
}
void doSeek(uint32_t docid) override {
if (__builtin_expect(isAtEnd(docid), false)) {
diff --git a/streamingvisitors/src/vespa/searchvisitor/rankprocessor.cpp b/streamingvisitors/src/vespa/searchvisitor/rankprocessor.cpp
index f0c2a8d4cc9..dc9cfdc7efd 100644
--- a/streamingvisitors/src/vespa/searchvisitor/rankprocessor.cpp
+++ b/streamingvisitors/src/vespa/searchvisitor/rankprocessor.cpp
@@ -89,6 +89,7 @@ RankProcessor::initQueryEnvironment()
terms[i].getTerm()->index().c_str(), terms[i].getTerm()->getTerm());
}
}
+ _match_data = _mdLayout.createMatchData();
}
void
@@ -100,7 +101,7 @@ RankProcessor::initHitCollector(size_t wantedHitCount)
void
RankProcessor::setupRankProgram(RankProgram &program)
{
- program.setup(_mdLayout, _queryEnv, search::fef::Properties());
+ program.setup(*_match_data, _queryEnv, search::fef::Properties());
}
void
@@ -137,6 +138,7 @@ RankProcessor::RankProcessor(RankManager::Snapshot::SP snapshot,
_query(query),
_queryEnv(location, snapshot->getIndexEnvironment(rankProfile), queryProperties, attrMgr),
_mdLayout(),
+ _match_data(),
_rankProgram(),
_docId(TermFieldMatchData::invalidId()),
_score(0.0),
@@ -182,13 +184,13 @@ copyTermFieldMatchData(const std::vector<search::fef::TermFieldMatchData> &src,
class RankProgramWrapper : public HitCollector::IRankProgram
{
private:
- RankProgram &_rankProgram;
+ MatchData &_match_data;
public:
- RankProgramWrapper(RankProgram &rankProgram) : _rankProgram(rankProgram) {}
+ RankProgramWrapper(MatchData &match_data) : _match_data(match_data) {}
virtual void run(uint32_t docid, const std::vector<search::fef::TermFieldMatchData> &matchData) override {
// Prepare the match data object used by the rank program with earlier unpacked match data.
- copyTermFieldMatchData(matchData, _rankProgram.match_data());
+ copyTermFieldMatchData(matchData, _match_data);
(void) docid;
}
};
@@ -202,7 +204,7 @@ RankProcessor::calculateFeatureSet()
RankProgram &rankProgram = *(_summaryProgram.get() != nullptr ? _summaryProgram : _rankProgram);
search::fef::FeatureResolver resolver(rankProgram.get_seeds());
LOG(debug, "Feature handles: numNames(%ld)", resolver.num_features());
- RankProgramWrapper wrapper(rankProgram);
+ RankProgramWrapper wrapper(*_match_data);
FeatureSet::SP sf = _hitCollector->getFeatureSet(wrapper, resolver);
LOG(debug, "Feature set: numFeatures(%u), numDocs(%u)", sf->numFeatures(), sf->numDocs());
return sf;
@@ -217,9 +219,8 @@ RankProcessor::fillSearchResult(vdslib::SearchResult & searchResult)
void
RankProcessor::unpackMatchData(uint32_t docId)
{
- MatchData &matchData = _rankProgram->match_data();
_docId = docId;
- unpackMatchData(matchData);
+ unpackMatchData(*_match_data);
}
void
diff --git a/streamingvisitors/src/vespa/searchvisitor/rankprocessor.h b/streamingvisitors/src/vespa/searchvisitor/rankprocessor.h
index f1fb917afcf..bab63c36522 100644
--- a/streamingvisitors/src/vespa/searchvisitor/rankprocessor.h
+++ b/streamingvisitors/src/vespa/searchvisitor/rankprocessor.h
@@ -29,6 +29,7 @@ private:
QueryEnvironment _queryEnv;
search::fef::MatchDataLayout _mdLayout;
+ search::fef::MatchData::UP _match_data;
search::fef::RankProgram::UP _rankProgram;
uint32_t _docId;
double _score;
@@ -67,7 +68,7 @@ public:
void runRankProgram(uint32_t docId);
search::FeatureSet::SP calculateFeatureSet();
void fillSearchResult(vdslib::SearchResult & searchResult);
- const search::fef::MatchData &getMatchData() const { return _rankProgram->match_data(); }
+ const search::fef::MatchData &getMatchData() const { return *_match_data; }
void setRankScore(double score) { _score = score; }
double getRankScore() const { return _score; }
HitCollector & getHitCollector() { return *_hitCollector; }