Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Honor after value for skipping documents even if queue is not full for PagingFieldCollector #12334

Merged
merged 1 commit into from
May 31, 2023

Conversation

gashutos
Copy link
Contributor

@gashutos gashutos commented May 25, 2023

Description

As mentioned in issue #12333, if after value is out of range for a field in a segment, PagingFieldCollector is not able to put any item in the FieldhitQueue. Due to which queueFull always stays false and it wont ever update PointValues based competitiveIterator to skip non-competitive hits.

I think, it makes sense to update competitiveIterator even if queue is not full and the query is after query because we have at least topValue in after query always. That itself should be capable enough to skip non-competitive hits.

It does not only help when after is out of range, also when after is within the range but filtering only based on the after value significantly reduces the number of hits to evaluate.

@gashutos gashutos changed the title fix searchafer high latency when after value is out of range for segment Fix searchafer high latency when after value is out of range for segment May 25, 2023
@gashutos gashutos changed the title Fix searchafer high latency when after value is out of range for segment Fix searchafter high latency when after value is out of range for segment May 25, 2023
@gashutos gashutos changed the title Fix searchafter high latency when after value is out of range for segment Fix searchafter query high latency when after value is out of range for segment May 25, 2023
Copy link
Contributor

@gsmiller gsmiller left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems reasonable to be able to skip due to deep paging before the queue fills. Nice find! It'd be great to see a test that reproduces this if possible. Not sure how hard that would be?

@@ -204,13 +200,21 @@ private void updateCompetitiveIterator() throws IOException {
return;
}
if (reverse == false) {
encodeBottom(maxValueAsBytes);
if (queueFull) { // bottom is avilable only when queue is full
maxValueAsBytes = maxValueAsBytes == null ? new byte[bytesCount] : maxValueAsBytes;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need the lazy initialization? I thought topValueSet would already be set before the NumericLeafComparator gets constructed. Maybe I'm misunderstanding that?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We dont know upfront at the time of construction (where currently initialization is done) that would we be needing maxValueAsBytes & minValuesAsBytes both. Like about the case, where we dont have any competitive hit collected in queue hence no bottom but has after value so the topValue.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies in advance if I'm misunderstanding, but as the code is currently written, we also don't know if we'll ever need these arrays. If the queue never fills, we could unnecessarily have allocated one of them. I think we still have enough information upfront though to eagerly allocate these like we do today? Is it just a question of being eager vs. lazy with these?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we also don't know if we'll ever need these arrays. If the queue never fills, we could unnecessarily have allocated one of them.

if queueFull is false, that mean we will be breaching this condition || (queueFull == false && topValueSet == false)) return; only if its after query where topValueSet is set to true. We dont allocate unnecessary here, i.e we initiliaze min/max values only if we are to call encodeTop or encodeBottom for that.

I gave explanation w.r.t code in current PR, but if you were talking in context of existing code, yes, we are allocating unnecessary in case queueFull is always false.

I think we still have enough information upfront though to eagerly allocate these like we do today? Is it just a question of being eager vs. lazy with these?

There is a problem if we follow the same approach, check this code,

} else {
        if (queueFull) { // bottom is avilable only when queue is full
          minValueAsBytes = minValueAsBytes == null ? new byte[bytesCount] : minValueAsBytes;
          encodeBottom(minValueAsBytes);
        }
        if (topValueSet) {
          maxValueAsBytes = maxValueAsBytes == null ? new byte[bytesCount] : maxValueAsBytes;
          encodeTop(maxValueAsBytes);
        }
      }

if queueFUll is always false & there is a topValueSet, we will have minValueAsBytes set as [0, 0, 0, 0, 0, 0, 0, 0, 0] instead null. This will result in incorrect minValueAsBytes in further calculations.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does that clarify ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I see. Yes, that helps clarify. Thanks! The difference I missed is that we never start updating the competitive iterator until the queue fills in the current code, so it doesn't matter that these byte arrays are initialized as zeros, but now it does matter because null has meaning in the case of the competitive iterator update. OK got it. Thanks for walking me through it :)

@gashutos gashutos marked this pull request as ready for review May 26, 2023 15:23
Copy link
Contributor

@gsmiller gsmiller left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good. Thanks @gashutos! I'll get this merged.

@gsmiller gsmiller merged commit d44be24 into apache:main May 31, 2023
4 checks passed
@gsmiller gsmiller added this to the 9.7.0 milestone May 31, 2023
@jpountz
Copy link
Contributor

jpountz commented Jun 14, 2023

We just upgraded Elasticsearch to a Lucene snapshot that has this change, and this triggered major speedups on some queries. In my opinion, the PR title and description don't do justice to this change since it does not only help when after is out of range, also when after is within the range but filtering only based on the after value significantly reduces the number of hits to evaluate.

@gashutos
Copy link
Contributor Author

We just upgraded Elasticsearch to a Lucene snapshot that has this change, and this triggered major speedups on some queries. In my opinion, the PR title and description don't do justice to this change since it does not only help when after is out of range, also when after is within the range but filtering only based on the after value significantly reduces the number of hits to evaluate.

@jpountz Agreed !

@jpountz
Copy link
Contributor

jpountz commented Jun 14, 2023

@gashutos I think we should make users aware of this optimization, would you be up for opening another PR that adds a CHANGES entry?

@jpountz
Copy link
Contributor

jpountz commented Jun 14, 2023

Let's also update the title/description of this PR?

@gashutos gashutos changed the title Fix searchafter query high latency when after value is out of range for segment Honor after value for skipping documents even if queue is not full for PagingFieldCollector Jun 14, 2023
gashutos added a commit to gashutos/lucene that referenced this pull request Jun 14, 2023
…ents even if queue is not full for PagingFieldCollector
gashutos added a commit to gashutos/lucene that referenced this pull request Jun 14, 2023
…ents even if queue is not full for PagingFieldCollector

Signed-off-by: gashutos <gashutos@amazon.com>
@gashutos
Copy link
Contributor Author

gashutos commented Jun 14, 2023

Sure, changed title/description, LMK if looks good.
CHANGES.txt PR #12368

@jpountz
Copy link
Contributor

jpountz commented Jun 14, 2023

Looks great, thanks!

jpountz pushed a commit that referenced this pull request Jun 14, 2023
…ven if queue is not full for PagingFieldCollector (#12368)

Signed-off-by: gashutos <gashutos@amazon.com>
jpountz pushed a commit that referenced this pull request Jun 14, 2023
…ven if queue is not full for PagingFieldCollector (#12368)

Signed-off-by: gashutos <gashutos@amazon.com>
hiteshk25 pushed a commit to cowpaths/lucene that referenced this pull request Jul 18, 2023
…dc8ca633e8bcf`) (#20)

* Add next minor version 9.7.0

* Fix SynonymQuery equals implementation (apache#12260)

The term member of TermAndBoost used to be a Term instance and became a
BytesRef with apache#11941, which means its equals impl won't take the field
name into account. The SynonymQuery equals impl needs to be updated
accordingly to take the field into account as well, otherwise synonym
queries with same term and boost across different fields are equal which
is a bug.

* Fix MMapDirectory documentation for Java 20 (apache#12265)

* Don't generate stacktrace in CollectionTerminatedException (apache#12270)

CollectionTerminatedException is always caught and never exposed to users so there's no point in filling
in a stack-trace for it.

* add missing changelog entry for apache#12260

* Add missing author to changelog entry for apache#12220

* Make query timeout members final in ExitableDirectoryReader (apache#12274)

There's a couple of places in the Exitable wrapper classes where
queryTimeout is set within the constructor and never modified. This
commit makes such members final.

* Update javadocs for QueryTimeout (apache#12272)

QueryTimeout was introduced together with ExitableDirectoryReader but is
now also optionally set to the IndexSearcher to wrap the bulk scorer
with a TimeLimitingBulkScorer. Its javadocs needs updating.

* Make TimeExceededException members final (apache#12271)

TimeExceededException has three members that are set within its constructor and never modified. They can be made final.

* DOAP changes for release 9.6.0

* Add back-compat indices for 9.6.0

* `ToParentBlockJoinQuery` Explain Support Score Mode (apache#12245) (apache#12283)

* `ToParentBlockJoinQuery` Explain Support Score Mode

---------

Co-authored-by: Marcus <marcuseagan@gmail.com>

* Simplify SliceExecutor and QueueSizeBasedExecutor (apache#12285)

The only behaviour that QueueSizeBasedExecutor overrides from SliceExecutor is when to execute on the caller thread. There is no need to override the whole invokeAll method for that. Instead, this commit introduces a shouldExecuteOnCallerThread method that can be overridden.

* [Backport] GITHUB-11838 Add api to allow concurrent query rewrite (apache#12197)

* GITHUB-11838 Change API to allow concurrent query rewrite (apache#11840)

Replace Query#rewrite(IndexReader) with Query#rewrite(IndexSearcher)

Co-authored-by: Patrick Zhai <zhaih@users.noreply.github.com>
Co-authored-by: Adrien Grand <jpountz@gmail.com>

Backport of apache#11840

Changes from original:
 - Query keeps `rewrite(IndexReader)`, but it is now deprecated
 - VirtualMethod is used to correct delegate to the overridden methods
 - The changes to `RewriteMethod` type classes are reverted, this increased the backwards compatibility impact. 

------------------------------

### Description
Issue: apache#11838 

#### Updated Proposal
 * Change signature of rewrite to `rewrite(IndexSearcher)`
 * How did I migrate the usage:
   * Use Intellij to do preliminary refactoring for me
   * For test usage, use searcher whenever is available, otherwise create one using `newSearcher(reader)`
   * For very few non-test classes which doesn't have IndexSearcher available but called rewrite, create a searcher using `new IndexSearcher(reader)`, tried my best to avoid creating it recurrently (Especially in `FieldQuery`)
   * For queries who have implemented the rewrite and uses some part of reader's functionality, use shortcut method when possible, otherwise pull out the reader from indexSearcher.

* Backport: Concurrent rewrite for KnnVectorQuery (apache#12160) (apache#12288)

* Concurrent rewrite for KnnVectorQuery (apache#12160)


- Reduce overhead of non-concurrent search by preserving original execution
- Improve readability by factoring into separate functions

---------

Co-authored-by: Kaival Parikh <kaivalp2000@gmail.com>

* adjusting for backport

---------

Co-authored-by: Kaival Parikh <46070017+kaivalnp@users.noreply.github.com>
Co-authored-by: Kaival Parikh <kaivalp2000@gmail.com>

* toposort use iterator to avoid stackoverflow (apache#12286)

Co-authored-by: tangdonghai <tangdonghai@meituan.com>
# Conflicts:
#	lucene/CHANGES.txt

* Fix test to compile with Java 11 after backport of apache#12286

* Update Javadoc for topoSortStates method after apache#12286 (apache#12292)

* Optimize HNSW diversity calculation (apache#12235)

* Minor cleanup and improvements to DaciukMihovAutomatonBuilder (apache#12305)

* GITHUB-12291: Skip blank lines from stopwords list. (apache#12299)

* Wrap Query rewrite backwards layer with AccessController (apache#12308)

* Make sure APIJAR reproduces with different timezone (unfortunately java encodes the date using local timezone) (apache#12315)

* Add multi-thread searchability to OnHeapHnswGraph (apache#12257)

* Fix backport error

* [MINOR] Update javadoc in Query class (apache#12233)

- add a few missing full stops
- update wording in the description of Query#equals method

* [Backport] Integrate the Incubating Panama Vector API apache#12311 (apache#12327)

Leverage accelerated vector hardware instructions in Vector Search.

Lucene already has a mechanism that enables the use of non-final JDK APIs, currently used for the Previewing Pamana Foreign API. This change expands this mechanism to include the Incubating Pamana Vector API. When the jdk.incubator.vector module is present at run time the Panamaized version of the low-level primitives used by Vector Search is enabled. If not present, the default scalar version of these low-level primitives is used (as it was previously).

Currently, we're only targeting support for JDK 20. A subsequent PR should evaluate JDK 21.
---------

Co-authored-by: Uwe Schindler <uschindler@apache.org>
Co-authored-by: Robert Muir <rmuir@apache.org>

* Parallelize knn query rewrite across slices rather than segments (apache#12325)

The concurrent query rewrite for knn vectory query introduced with apache#12160
requests one thread per segment to the executor. To align this with the
IndexSearcher parallel behaviour, we should rather parallelize across
slices. Also, we can reuse the same slice executor instance that the
index searcher already holds, in that way we are using a
QueueSizeBasedExecutor when a thread pool executor is provided.

* Optimize ConjunctionDISI.createConjunction (apache#12328)

This method is showing up as a little hot when profiling some queries.
Almost all the time spent in this method is just burnt on ceremony
around stream indirections that don't inline.
Moving this to iterators, simplifying the check for same doc id and also saving one iteration (for the min
cost) makes this method far cheaper and easier to read.

* Update changes to be correct with ARM (it is called NEON there)

* GH#12321: Marked DaciukMihovAutomatonBuilder as deprecated (apache#12332)

Preparing to reduce visibility of this class in a future release

* add BitSet.clear() (apache#12268)

# Conflicts:
#	lucene/CHANGES.txt

* Clenaup and update changes and synchronize with 9.x

* Update TestVectorUtilProviders.java (apache#12338)

* Don't generate stacktrace for TimeExceededException (apache#12335)

The exception is package private and never rethrown, we can avoid
generating a stacktrace for it.

* Introduced the Word2VecSynonymFilter (apache#12169)

Co-authored-by: Alessandro Benedetti <a.benedetti@sease.io>

* Word2VecSynonymFilter constructor null check (apache#12169)

* Use thread-safe search version of HnswGraphSearcher (apache#12246)

Addressing comment received in the PR apache#12246

* Word2VecSynonymProvider to use standard Integer max value for hnsw searches (apache#12235)
We observed this change was not ported previously from main in an old cherry-pick

* Fix searchafter high latency when after value is out of range for segment (apache#12334)

* Make memory fence in `ByteBufferGuard` explicit (apache#12290)

* Add "direct to binary" option for DaciukMihovAutomatonBuilder and use it in TermInSetQuery#visit (apache#12320)

* Add updateDocuments API which accept a query (reopen) (apache#12346)

* GITHUB#11350: Handle backward compatibility when merging segments with different FieldInfo

This commits restores Lucene 9's ability to handle indices created with Lucene 8 where there are discrepancies in FieldInfos, such as different IndexOptions

* [Tessellator] Improve the checks that validate the diagonal between two polygon nodes (apache#12353)

# Conflicts:
#	lucene/CHANGES.txt

* feat: soft delete optimize (apache#12339)

* Better paging when random reads go backwards (apache#12357)

When reading data from outside the buffer, BufferedIndexInput always resets
its buffer to start at the new read position. If we are reading backwards (for example,
using an OffHeapFSTStore for a terms dictionary) then this can have the effect of
re-reading the same data over and over again.

This commit changes BufferedIndexInput to use paging when reading backwards,
so that if we ask for a byte that is before the current buffer, we read a block of data
of bufferSize that ends at the previous buffer start.

Fixes apache#12356

* Work around SecurityManager issues during initialization of vector api (JDK-8309727) (apache#12362)

* Restrict GraphTokenStreamFiniteStrings#articulationPointsRecurse recursion depth (apache#12249)

* Implement MMapDirectory with Java 21 Project Panama Preview API (apache#12294)
Backport incl JDK21 apijar file with java.util.Objects regenerated

* remove relic in apijar folder caused by vector additions

* Speed up IndexedDISI Sparse #AdvanceExactWithinBlock for tiny step advance (apache#12324)

* Add checks in KNNVectorField / KNNVectorQuery to only allow non-null, non-empty and finite vectors (apache#12281)


---------

Co-authored-by: Uwe Schindler <uschindler@apache.org>

* Implement VectorUtilProvider with Java 21 Project Panama Vector API (apache#12363) (apache#12365)

This commit enables the Panama Vector API for Java 21. The version of
VectorUtilPanamaProvider for Java 21 is identical to that of Java 20.
As such, there is no specific 21 version - the Java 20 version will be
loaded from the MRJAR.

* Add CHANGES.txt for apache#12334 Honor after value for skipping documents even if queue is not full for PagingFieldCollector (apache#12368)

Signed-off-by: gashutos <gashutos@amazon.com>

* Move TermAndBoost back to its original location. (apache#12366)

PR apache#12169 accidentally moved the `TermAndBoost` class to a different location,
which would break custom sub-classes of `QueryBuilder`. This commit moves it
back to its original location.

* GITHUB-12252: Add function queries for computing similarity scores between knn vectors (apache#12253)

Co-authored-by: Alessandro Benedetti <a.benedetti@sease.io>

* hunspell (minor): reduce allocations when processing compound rules (apache#12316)

(cherry picked from commit a454388)

* hunspell (minor): reduce allocations when reading the dictionary's morphological data (apache#12323)

there can be many entries with morph data, so we'd better avoid compiling and matching regexes and even stream allocation

(cherry picked from commit 4bf1b94)

* TestHunspell: reduce the flakiness probability (apache#12351)

* TestHunspell: reduce the flakiness probability

We need to check how the timeout interacts with custom exception-throwing checkCanceled.
The default timeout seems not enough for some CI agents, so let's increase it.

Co-authored-by: Dawid Weiss <dawid.weiss@gmail.com>
(cherry picked from commit 5b63a18)

* This allows VectorUtilProvider tests to be executed although hardware may not fully support vectorization or if C2 is not enabled (apache#12376)

---------

Signed-off-by: gashutos <gashutos@amazon.com>
Co-authored-by: Alan Woodward <romseygeek@apache.org>
Co-authored-by: Luca Cavanna <javanna@apache.org>
Co-authored-by: Uwe Schindler <uschindler@apache.org>
Co-authored-by: Armin Braun <me@obrown.io>
Co-authored-by: Mikhail Khludnev <mkhludnev@users.noreply.github.com>
Co-authored-by: Marcus <marcuseagan@gmail.com>
Co-authored-by: Benjamin Trent <ben.w.trent@gmail.com>
Co-authored-by: Kaival Parikh <46070017+kaivalnp@users.noreply.github.com>
Co-authored-by: Kaival Parikh <kaivalp2000@gmail.com>
Co-authored-by: tang donghai <tangdhcs@gmail.com>
Co-authored-by: Patrick Zhai <zhaih@users.noreply.github.com>
Co-authored-by: Greg Miller <gsmiller@gmail.com>
Co-authored-by: Jerry Chin <metrxqin@gmail.com>
Co-authored-by: Patrick Zhai <zhai7631@gmail.com>
Co-authored-by: Andrey Bozhko <andybozhko@gmail.com>
Co-authored-by: Chris Hegarty <62058229+ChrisHegarty@users.noreply.github.com>
Co-authored-by: Robert Muir <rmuir@apache.org>
Co-authored-by: Jonathan Ellis <jbellis@datastax.com>
Co-authored-by: Daniele Antuzi <daniele.antuzi@gmail.com>
Co-authored-by: Alessandro Benedetti <a.benedetti@sease.io>
Co-authored-by: Chaitanya Gohel <104654647+gashutos@users.noreply.github.com>
Co-authored-by: Petr Portnov | PROgrm_JARvis <pportnov@ozon.ru>
Co-authored-by: Tomas Eduardo Fernandez Lobbe <tflobbe@apache.org>
Co-authored-by: Ignacio Vera <ivera@apache.org>
Co-authored-by: fudongying <30896830+fudongyingluck@users.noreply.github.com>
Co-authored-by: Chris Fournier <chris.fournier@shopify.com>
Co-authored-by: gf2121 <52390227+gf2121@users.noreply.github.com>
Co-authored-by: Adrien Grand <jpountz@gmail.com>
Co-authored-by: Elia Porciani <e.porciani@sease.io>
Co-authored-by: Peter Gromov <peter@jetbrains.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants