* Remove implicit addition of vector 0
Removes logic to add 0 vector implicitly. This is in preparation for
adding nodes from other graphs to initialize a new graph. Having the
implicit addition of node 0 complicates this logic.
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Enable out of order insertion of nodes in hnsw
Enables nodes to be added into OnHeapHnswGraph in out of order fashion.
To do so, additional operations have to be taken to resort the
nodesByLevel array. Optimizations have been made to avoid sorting
whenever possible.
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Add ability to initialize from graph
Adds method to initialize an HNSWGraphBuilder from another HNSWGraph.
Initialization can only happen when the builder's graph is empty.
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Utilize merge with graph init in HNSWWriter
Uses HNSWGraphBuilder initialization from graph functionality in
Lucene95HnswVectorsWriter. Selects the largest graph to initialize the
new graph produced by the HNSWGraphBuilder for merge.
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Minor modifications to Lucene95HnswVectorsWriter
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Use TreeMap for graph structure for levels > 0
Refactors OnHeapHnswGraph to use TreeMap to represent graph structure of
levels greater than 0. Refactors NodesIterator to support set
representation of nodes.
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Refactor initializer to be in static create method
Refeactors initialization from graph to be accessible via a create
static method in HnswGraphBuilder.
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Address review comments
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Add change log entry
Signed-off-by: John Mazanec <jmazane@amazon.com>
* Remove empty iterator for neighborqueue
Signed-off-by: John Mazanec <jmazane@amazon.com>
---------
Signed-off-by: John Mazanec <jmazane@amazon.com>
`KeywordField` is a combination of `StringField` and `SortedSetDocValuesField`,
similarly to how `LongField` is a combination of `LongPoint` and
`SortedNumericDocValuesField`. This makes it easier for users to create fields
that can be used for filtering, sorting and faceting.
* Optimize the common case that docs only have single values for the field
* In the multivalued case, terminate reading docvalues if they are > maximum set ordinal
* Implement ScorerSupplier, so that (potentially large) number of ordinal lookups aren't performed just to get the cost()
* Graduate to Sorted(Set)DocValuesField.newSlowSetQuery to complement newSlowRangeQuery, newSlowExactQuery
Like other slow queries in these classes, it's currently only recommended to use with points, e.g. IndexOrDocValuesQuery(new PointInSetQuery, newSlowSetQuery)
LongHashSet is used for the set of numbers, but it has some issues:
* tries to hard to extend AbstractSet, mostly for testing
* causes traps with boxing if you aren't careful
* complex hashcode/equals
Practically we should take advantage of the fact numbers come in sorted
order for multivalued fields: just like range queries do. So we use
min/max to our advantage, including termination of docvalues iteration
Actually it is generally a win to just check min/max even in the single-valued
case: these constant time comparisons are cheap and can avoid hashing,
etc.
In the worst-case, if all of your query Sets contain both the minimum and maximum
possible values, then it won't help, but it doesn't hurt either.
There's no need to make things abstract: DocValues does the right thing
Optimizing for where no docs for the field in the segment exist is easy, simple null check (replacing the existing one!)
Currently stored fields have to look at binaryValue(), stringValue() and
numericValue() to guess the type of the value and then store it. This has a few
issues:
- If there is a problem, e.g. all of these 3 methods return null, it's
currently discovered late, when we already passed the responsibility of
writing data from IndexingChain to the codec.
- numericValue() is used both for numeric doc values and storage. This makes
it impossible to implement a `double` field that is stored and doc-valued,
as numericValue() needs to return simultaneously a number that consists of
the double for storage, and the long bits of the double for doc values.
- binaryValue() is used both for sorted(_set) doc values and storage. This
makes it impossible to implement `keyword` fields that is stored and
doc-valued, as the field returns a non-null value for both binaryValue() and
stringValue() and stored fields no longer know which field to store.
This commit introduces `IndexableField#storedValue()`, which is used only for
stored fields. This addresses the above issues. IndexingChain passes the
storedValue() directly to the codec, so it's impossible for a stored fields
format to mistakenly use binaryValue()/stringValue()/numericValue() instead of
storedValue().
Sometimes the random search lucene test searcher will wrap the reader. Consequently, we need to make sure to use the reader provided by the test IndexSearcher or the reader may be different between creating the weight with the searcher vs. accessing the leaf context for the scorer.
While FeatureQuery is a powerful tool in the scoring case, there are scenarios when caching should be allowed and scoring disabled.
A particular case is when the FeatureQuery is used in conjunction with learned-sparse retrieval. It is useful to iterate and calculate the entire matching doc set when combined with various other queries.
related to: https://github.com/apache/lucene/issues/11799
Two of the methods (squareDistance and dotProduct) that take byte arrays return a float while
the variable used to store the value is an int. They can just return an int.
A recent test failure signaled that when the simple text codec was randomly selected, byte vectors could not be written.
This commit addressed that by adding support for writing byte vectors to SimpleTextKnnVectorsWriter.
Note that while support is added to the BufferingKnnVectorsWriter base class, 90, 91 and 92 writers don't need to support
byte vectors and will throw unsupported operation exception when attempting to do that.
Follow-up of #12105 to remove the deprecated classes for the next major version.
Removes KnnVectorField, KnnVectorQuery, VectorValues and LeafReader#getVectorValues.
When a field indexes numeric doc values, `MemoryIndex` does an unchecked cast
to `java.lang.Long`. However, the new `IntField` represents the value as a
`java.lang.Integer` so this cast fails. This commit aligns `MemoryIndex` with
`IndexingChain` by casting to `Number` and calling `Number#longValue` instead
of casting to `Long`.
We recently introduced KnnByteVectorField, KnnByteVectorQuery and ByteVectorValues. The corresponding float variants of the same classes don't follow the same naming convention: KnnVectorField, KnnVectoryQuery and VectorValues. Ideally their names would reflect that they are the float variant of the vector field, vector query and vector values.
This commit aims at clarifying this in the public facing API, by deprecating the current float classes in favour of new ones that are their exact copy but follow the same naming conventions as the byte ones.
As a result, LeafReader#getVectorValues is also deprecated in favour of newly introduced getFloatVectorValues method that returns FloatVectorValues.
Relates to #11963
The main classes involved are ByteVectorValues, KnnByteVectorField and KnnByteVectorQuery. It becomes quite natural to simplify things further and use byte[] in the following methods too: ByteVectorValues#vectorValue, KnnVectorReader#search, LeafReader#searchNearestVectors, HNSWGraphSearcher#search, VectorSimilarityFunction#compare, VectorUtil#cosine, VectorUtil#squareDistance, VectorUtil#dotProduct, VectorUtil#dotProductScore
This method tries to expose an encoded view of vectors, but we shouldn't have
this part of our user-facing API. With this change, the way vectors are encoded
is entirely on the codec.
Clean up this query a bit and support:
* NumericDocValuesField.newSlowSetQuery()
* SortedNumericDocValuesField.newSlowSetQuery()
This complements the existing docvalues-based range queries, with a set query.
Add ScorerSupplier/cost estimation support to PointInSetQuery
Add newSetQuery() to IntField/LongField/DoubleField/FloatField, that uses IndexOrDocValuesQuery
WeightedSpanTermExtractor will try to rewrite queries that it doesn't
know about, to see if they end up as something it does know about and
that it can extract terms from. To support field merging, it rewrites against
a delegating leaf reader that does not support getFieldInfos().
FieldExistsQuery uses getFieldInfos() in its rewrite, which means that
if one is passed to WeightedSpanTermExtractor, we get an
UnsupportedOperationException thrown.
This commit makes WeightedSpanTermExtractor aware of FieldExistsQuery,
so that it can just ignore it and avoid throwing an exception.
When sub collectors don't agree on their `ScoreMode`, `MultiCollector`
currently returns `COMPLETE`. This makes sense when assuming that there is
likely one collector computing top hits (`TOP_SCORES`) and another one
computing facets (`COMPLETE_NO_SCORES`) so `COMPLETE` makes sense. However it
is also possible to have one collector computing top hits by field (`TOP_DOCS`)
and another one doing facets (`COMPLETE_NO_SCORES`), and `MultiCollector`
shouldn't report that scores are needed in that case.
In the case when an index is sorted on a low-cardinality field, or the index
sort order correlates with the order in which documents get ingested, we can
optimize `SortedDocIDMerger` by doing a single comparison with the doc ID on
the next sub. This checks covers at the same time whether the priority queue
needs reordering and whether the current sub reached `NO_MORE_DOCS`.
On the NYC taxis dataset on my local machine, switching from
`Arrays#compareUnsigned` to `ArrayUtil#getUnsignedComparator` yielded a 15%
speedup of BKD merging.
When #672 was introduced, it added many nice rewrite optimizations. However, in the case when there are many multiple nested Boolean queries under a top level Boolean#filter clause, its runtime grows exponentially.
The key issue was how the BooleanQuery#rewriteNoScoring redirected yet again to the ConstantScoreQuery#rewrite. This causes BooleanQuery#rewrite to be called again recursively , even though it was previously called in ConstantScoreQuery#rewrite, and THEN BooleanQuery#rewriteNoScoring is called again, recursively.
This causes exponential growth in rewrite time based on query depth. The change here hopes to short-circuit that and only grow (near) linearly by calling BooleanQuery#rewriteNoScoring directly, instead if attempting to redirect through ConstantScoreQuery#rewrite.
closes: #12069
Today Lucene allows creating indexed binary fields, e.g. via
`StringField(String, BytesRef, Field.Store)`, but not reusing them: calling
`setBytesValue` on a `StringField` throws.
This commit removes the check that prevents reusing fields with binary values.
I considered an alternative that consisted of failing if calling
`setBytesValue` on a field that is indexed and tokenized, but we currently
don't have such checks e.g. on numeric values, so it did not feel consistent.
Doing this change would help improve the [nightly benchmarks for the NYC taxis
dataset](http://people.apache.org/~mikemccand/lucenebench/sparseResults.html)
by doing the String -> UTF-8 conversion only once for keywords, instead of once
for the `StringField` and one for the `SortedDocValuesField`, while still
reusing fields.
Today Lucene allows creating indexed binary fields, e.g. via
`StringField(String, BytesRef, Field.Store)`, but not reusing them: calling
`setBytesValue` on a `StringField` throws.
This commit removes the check that prevents reusing fields with binary values.
I considered an alternative that consisted of failing if calling
`setBytesValue` on a field that is indexed and tokenized, but we currently
don't have such checks e.g. on numeric values, so it did not feel consistent.
Doing this change would help improve the [nightly benchmarks for the NYC taxis
dataset](http://people.apache.org/~mikemccand/lucenebench/sparseResults.html)
by doing the String -> UTF-8 conversion only once for keywords, instead of once
for the `StringField` and one for the `SortedDocValuesField`, while still
reusing fields.
`ConcurrentMergeScheduler` uses the rate at which a merge writes bytes as a
proxy for CPU usage, in order to prevent merging from disrupting searches too
much. However creating compound files are lightweight CPU-wise and do not need
throttling.
Closes#12068
This iterates on #399 to also optimize the case when an index sort is
configured. When cutting over the NYC taxis benchmark to the new numeric
fields,
[flush times](http://people.apache.org/~mikemccand/lucenebench/sparseResults.html#flush_times)
stayed mostly the same when index sorting is disabled and increased by 7-8%
when index sorting is enabled. I expect this change to address this slowdown.
When flushing segments that have an index sort configured, postings lists get
loaded into arrays and get reordered according to the index sort.
This reordering is implemented with `TimSorter`, a variant of merge sort. Like
merge sort, an important part of `TimSorter` consists of merging two contiguous
sorted slices of the array into a combined sorted slice. This merging can be
done either with external memory, which is the classical approach, or in-place,
which still runs in linear time but with a much higher factor. Until now we
were allocating a fixed budget of `maxDoc/64` for doing these merges with
external memory. If this is not enough, sorted slices would be merged in place.
I've been looking at some profiles recently for an index where a non-negligible
chunk of the time was spent on in-place merges. So I would like to propose the
following change:
- Increase the maximum RAM budget to `maxDoc / 8`. This should help avoid
in-place merges for all postings up to `docFreq = maxDoc / 4`.
- Make this RAM budget lazily allocated, rather than eagerly like today. This
would help not allocate memory in O(maxDoc) for fields like primary keys
that only have a couple postings per term.
So overall memory usage would never be more than 50% higher than what it is
today, because `TimSorter` never needs more than X temporary slots if the
postings list doesn't have at least 2*X entries, and these 2*X entries already
get loaded into memory today. And for fields that have short postings, memory
usage should actually be lower.
They currently call `Arrays#sort`, which incurs a tiny bit of overhead due to
range checks and some logic to determine the optimal sorting algorithm to use
depending on the number of values. We can skip this overhead in the case when
there is a single value.
Drop 3.x compatibility (which was pickier at compile-time and prevented slow things from happening). Instead add paranoia to runtime tests, so that they fail if antlr would do something slow in the parsing. This is needed because antlrv4 is a big performance trap: https://github.com/antlr/antlr4/blob/master/doc/faq/general.md
"Q: What are the main design decisions in ANTLR4?
Ease-of-use over performance. I will worry about performance later."
It allows us to move forward with newer antlr but hopefully prevent the associated headaches.
Signed-off-by: Andriy Redko <andriy.redko@aiven.io>
Co-authored-by: Robert Muir <rmuir@apache.org>
Add new stored fields and termvectors interfaces: IndexReader.storedFields()
and IndexReader.termVectors(). Deprecate IndexReader.document() and IndexReader.getTermVector().
The new APIs do not rely upon ThreadLocal storage for each index segment, which can greatly
reduce RAM requirements when there are many threads and/or segments.
Co-authored-by: Adrien Grand <jpountz@gmail.com>
* Leverage DISI static factory methods more over custom DISI impl where possible.
* Assert points field is a single-dim.
* Bound cost estimate by the cost of the doc values field (for sparse fields).
`RandomAccessVectorValues` is internally used in our HNSW implementation to
provide random access to vectors, both at index and search time. In order to
better reflect this, this change does the following:
- `RandomAccessVectorValues` moves to `org.apache.lucene.util.hnsw`.
- `BufferingKnnVectorsWriter` no longer has a dependency on
`RandomAccessVectorValues` and moves to `org.apache.lucene.codecs` since
it's more of a utility class for KNN vector file formats than an index API.
Maybe we should think of moving it near each file format that uses it
instead.
- `SortingCodecReader` no longer has a dependency on
`RandomAccessVectorValues`.
Closes#10623
This generalizes #687 to indexes that are sorted in descending order. The main
challenge with descending sorts is that they require being able to compute the
last doc ID that matches a value, which would ideally require walking the BKD
tree in reverse order, but the API only support moving forward. This is worked
around by maintaining a stack of `PointTree` clones to perform the search.
PassageScorer uses a priority queue of size maxPassages to keep track of
which highlighted passages are worth returning to the user. Once all
passages have been collected, we go through and merge overlapping
passages together, but this reduction in the number of passages is not
compensated for by re-adding the highest-scoring passages that were pushed
out of the queue by passages which have been merged away.
This commit increases the size of the priority queue to try and account for
overlapping passages that will subsequently be merged together.
ExitableTerms should not iterate through the terms to retrieve min and max when the wrapped implementation has the values cached (e.g. FieldsReader, OrdsFieldReader)
OffsetsFromMatchIterator and OffsetsFromPositions both have package-
private constructors, which makes them difficult to use as components in a
separate highlighter implementation.
`VectorValues` have a `cost()` method that reports an approximate number of
documents that have a vector, but also a `size()` method that reports the
accurate number of vectors in the field. Since KNN vectors only support
single-valued fields we should enforce that `cost()` returns the `size()`.
QueryBuilder#newSynonymQuery takes an array of TermAndBoost objects as a
parameter and uses the field of the first term in the array as its field. However,
there are cases where this array may be empty, which will result in an
ArrayOutOfBoundsException.
This commit reworks QueryBuilder so that TermAndBoost contains plain
BytesRefs, and passes the field as a separate parameter. This guards against
accidental calls to newSynonymQuery with an empty list - in this case, an
empty synonym query is generated rather than an exception. It also
refactors SynonymQuery itself to hold BytesRefs rather than Terms, which
needlessly repeat the field for each entry.
Fixes#11864
This test intentionally does a ton of filesystem operations: currently
about 20% of the time you can get really unlucky and get virus checker
simulated, against a real filesystem, which makes things really slow.
Instead use a ByteBuffersDirectory for local runs so that it doesn't
take minutes. The test can still be pretty slow even with this
implementation, so tone down the runtime so that it takes ~ 1.5s
locally.
The default codec has a number of small and hot files, that actually used to be
fully loaded in memory before we moved them off-heap. In the general case,
these files are expected to fully fit into the page cache for things to work
well. Should we give control over preloading to codecs? This is what this
commit does for the following files:
- Terms index (`tip`)
- Points index (`kdi`)
- Stored fields index (`fdx`)
- Terms vector index (`tvx`)
This only has an effect on `MMapDirectory`.
Port generic exception handling from MemorySegmentIndexInput to ByteBufferIndexInput. This also adds the invalid position while seeking or reading to the exception message.
hunspell: introduce FragmentChecker to speed up ModifyingSuggester
add NGramFragmentChecker to quickly check whether insertions/replacements produce strings that are even possible in the language
Co-authored-by: Dawid Weiss <dawid.weiss@gmail.com>
This test often takes several minutes with normal runs (no NIGHTLY/multiplier/etc). Tone it down so that it isn't slow: CI builds can work it harder by passing those parameters
Adds createLatLonShapeDocValues and createXYShapeDocValues factory methods
to LatLonShape and XYShape factory classes, respectively.
Signed-off-by: Nicholas Walter Knize <nknize@apache.org>
When reading large segments, the vectors format can fail with a validation
error:
java.lang.IllegalStateException: Vector data length 3070061568 not matching
size=999369 * dim=768 * byteSize=4 = -1224905728
The problem is that we use an integer to represent the size, which is too small
to hold it. The bug snuck in during the work to enable int8 values, which
switched a long value to an int.
When running the `reindex` task with KnnGraphTester, exceptionally large
datasets can be used. Since mmap is used to read the data, we need to know the
buffer size. This size is limited to Integer.MAX_VALUE, which is inadequate for
larger datasets.
So, this commit adjusts the reading to only read a single vector at a time.
We recently introduced support for kNN vectors to `ExitableDirectoryReader`.
Previously, we checked for cancellation not only on sampled calls `advance`,
but on every single call to `vectorValue`. This can cause significant overhead
when a query scans many vector values (for example the case where you're doing
an exact scan and computing a vector similarity for every matching document).
This PR removes the cancellation checks on `vectorValue`, since having them on
`advance` is already enough.
Move minimum TieredMergePolicy delete percentage from 20% to 5%
and change deletePctAllowed default to 20%
Co-authored-by: Marc D'Mello <dmellomd@amazon.com>
Since QueryVisitor added the ability to signal multi-term queries, the query rewrite
call in UnifiedHighlighter has been essentially useless, and with more aggressive
rewriting this is now causing bugs like #11490. We can safely remove this call.
Fixes#11490
This uses Gradle's auto-provisioning to compile Java 19 classes and build a multi-release JAR from them. Please make sure to regenerate gradle.properties (delete it) or change "org.gradle.java.installations.auto-download" to "true"
* Upgrade several build dependencies.
* Update error prone rules (those are off but they do trigger warnings/ errors)
* A few corrections I made before I turned off new warnings. Let's do nother issue to fix them.
This method is recursive: to avoid eating too much stack we apply a
small limit. This means it can't really be used on any largish automata
without hitting exception.
But the benefit of knowing finite vs infinite in AutomatonTermsEnum is
minor: let's not auto-compute this. FuzzyQuery still gets the finite
optimization because its finite by definition. PrefixQuery is always
infinite. Wildcard/Regex just assume infinite which is safe to do.
Remove the auto-computation and the "trillean" Boolean parameter. If you
dont know that your automaton is finite, pass false to
CompiledAutomaton, it is safe.
Move this method to AutomatonTestUtil so we can still use it in test
asserts.
Closes#11809
The UnifiedHighlighter can throw exceptions when highlighting terms that are longer
than the maximum size the DaciukMihovAutomatonBuilder accepts. Rather than throwing
a confusing exception, we can instead filter out the long terms when building the
MemoryIndexOffsetStrategy. Very long terms are likely to be junk input in any case.
FieldExistsQuery checks if there are points for a certain field, and then retrieves the
corresponding point values. When all documents that had points for a certain field have
been deleted from a certain segments, as well as merged away, field info may report
that there are points yet the corresponding point values are null.
With this change we add a null check in FieldExistsQuery. Long term, we will likely want
to prevent this situation from happening.
Relates #11393
Introduction of dynamic pruning for string sorts (#11669) introduced a bug with
string sorts and ghost fields, triggering a `NullPointerException` because the
code assumes that `LeafReader#terms` is not null if the field is indexed
according to field infos.
This commit fixes the issue and adds tests for ghost fields across all sort
types.
Hopefully we can simplify and remove the null check in the future when we
improve handling of ghost fields (#11393).
IntervalBuilder.NO_INTERVALS should return -1 when unpositioned,
not NO_MORE_DOCS. This can trigger exceptions when an empty
IntervalQuery is combined in a conjunction.
Fixes#11759
This PR removes the recently added function on LeafReader to exhaustively search
through vectors, plus the helper function KnnVectorsReader#searchExhaustively.
Instead it performs the exact search within KnnVectorQuery, using a new helper
class called VectorScorer.
If ConcurrentMergeScheduler is used, and the merge hits fatal exception (such as disk full) after prepareCommit()'s ensureOpen() check, then startCommit() will throw IllegalStateException instead of AlreadyClosedException.
The test is currently not prepared to handle this: the logic is only geared around exceptions coming from addDocument()
Closes#11755
When indexing term vectors for a very large document, the automatic computation
of the dictionary size based on the overall size of the block might yield a
size that exceeds the maximum window size that is supported by LZ4. This commit
addresses the issue by automatically taking the minimum of the result of this
computation and the maximum window size (64kB).
* Remove usages of System.currentTimeMillis() from tests
- Use Random from `RandomizedRunner` to be able to use a Seed to
reproduce tests, instead of a seed coming from wall clock.
- Replace time based tests, using wall clock to determine periods
with counter of repetitions, to have a consistent reproduction.
Closes: #11459
* address comments
* tune iterations
* tune iterations for nightly
These internal versions only make sense within a codec definition, and aren't
meant to be exposed and compared across codecs. Since this method is only used
in tests, we can move the check to the test classes instead.
This change folds the `RandomAccessVectorValuesProducer` interface into
`RandomAccessVectorValues`. This reduces the number of interfaces and clarifies
the cloning/ copying behavior.
This is a small simplification related to LUCENE-9583, but does not address the
main issue.
The base spatial test case may create invalid self crossing polygons. These
polygons are cleaned by the tessellator which may result in an inconsistent
bounding box between the tessellated shape and the original, invalid, geometry.
This commit fixes the shape doc value test case to compute the bounding box from
the cleaned geometry instead of relying on the, potentially invalid, original
geometry.
Signed-off-by: Nicholas Walter Knize <nknize@apache.org>
We currently compute the partition point for a set of points by multiplying the number of nodes that needs to be on
the left of the BKD tree by the maxPointsInLeafNode. This multiplication is done on the integer space so if the partition point is bigger than Integer.MAX_VALUE it will overflow. This commit moves the multiplication to the long space so it doesn't overflow.
Adds new doc value field to support LatLonShape and XYShape doc values. The
implementation is inspired by ComponentTree. A binary tree of tessellated
components (point, line, or triangle) is created. This tree is then DFS
serialized to a variable compressed DataOutput buffer to keep the doc value
format as compact as possible.
DocValue queries are performed on the serialized tree using a similar component
relation logic as found in SpatialQuery for BKD indexed shapes. To make this
possible some of the relation logic is refactored to make it accessible to the
doc value query counterpart.
Note this does not support the following:
* Multi Geometries or Collections - This will be investigated by exploring
the addition of multi binary doc values.
* General Geometry Queries - This will be added in a follow on improvement.
Signed-off-by: Nicholas Walter Knize <nknize@apache.org>
* add Comment on Lev & pretty the toDot
* use auto generate scripts to add comment
* update checksum
* update checksum
* restore toDot
* add removeDeadStates in levAutomata
Co-authored-by: tangdonghai <tangdonghai@meituan.com>
Added a `prefilter` and `filterSelectivity` argument to KnnGraphTester to be
able to compare pre and post-filtering benchmarks.
`filterSelectivity` expresses the selectivity of a filter as proportion of
passing docs that are randomly selected. We store these in a FixedBitSet and
use this to calculate true KNN as well as in HNSW search.
In case of post-filter, we over-select results as `topK / filterSelectivity` to
get final hits close to actual requested `topK`. For pre-filter, we wrap the
FixedBitSet in a query and pass it as prefilter argument to KnnVectorQuery.
If there are multiple segments. KnnVectorQuery explain has a bug in locating
the doc ID. This is because the doc ID in explain is the docBase without the
segment. In KnnVectorQuery.DocAndScoreQuery docs docid is increased in each
segment of the docBase. So, in the 'DocAndScoreQuery.explain', needs to be
added with the segment's docBase.
Co-authored-by: Julie Tibshirani <julietibs@apache.org>
This test occasionally fails if knn search returns only 1 document
in the index, as we have an assertion that returned doc IDs from
sorted and unsorted index must be different.
This patch ensures that we have many documents in the index, so
that knn search always returns enough results.
Currently, when indexing knn vectors, we buffer them in memory and
on flush during a segment construction we build an HNSW graph.
As building an HNSW graph is very expensive, this makes flush
operation take a lot of time. This also makes overall indexing
performance quite unpredictable – some indexing operations return
almost instantly while others that trigger flush take a lot of time.
This happens because flushes are unpredictable and trigged
by memory used, presence of concurrent searches etc.
Building an HNSW graph as we index documents avoid these problems,
as the load of HNSW graph construction is spread evenly during indexing.
Co-authored-by: Adrien Grand <jpountz@gmail.com>
Abstract method copyBytes need to copy from input to a buffer and then write into ByteBuffersDataOutput, i think there is unnecessary, we can override it, copy directly from input into output
Fix error in comparing between bytes of candidates and bytes of max merge.
It's wrong to use candidateSize rather than currentCandidateBytes comparing with maxMergeBytes.
This method is called from `addIndexes` and should be synchronized so that it
would see consistent data structures in case of concurrent indexing that would
be introducing new fields.
I hit a rare test failure of `TestIndexRearranger` that I can only explain by this lack of locking:
```
15:40:14 > java.util.concurrent.ExecutionException: java.lang.NullPointerException: Cannot read field "numDimensions" because "props" is null
15:40:14 > at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
15:40:14 > at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
15:40:14 > at org.apache.lucene.misc.index.IndexRearranger.execute(IndexRearranger.java:98)
15:40:14 > at org.apache.lucene.misc.index.TestIndexRearranger.testRearrangeUsingBinaryDocValueSelector(TestIndexRearranger.java:97)
15:40:14 > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
15:40:14 > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
15:40:14 > at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
15:40:14 > at java.base/java.lang.reflect.Method.invoke(Method.java:568)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner.invoke(RandomizedRunner.java:1758)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner$8.evaluate(RandomizedRunner.java:946)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner$9.evaluate(RandomizedRunner.java:982)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner$10.evaluate(RandomizedRunner.java:996)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleSetupTeardownChained$1.evaluate(TestRuleSetupTeardownChained.java:44)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleThreadAndTestName$1.evaluate(TestRuleThreadAndTestName.java:45)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44)
15:40:14 > at junit@4.13.1/org.junit.rules.RunRules.evaluate(RunRules.java:20)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:390)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.ThreadLeakControl.forkTimeoutingTask(ThreadLeakControl.java:843)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.ThreadLeakControl$3.evaluate(ThreadLeakControl.java:490)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner.runSingleTest(RandomizedRunner.java:955)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner$5.evaluate(RandomizedRunner.java:840)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner$6.evaluate(RandomizedRunner.java:891)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.RandomizedRunner$7.evaluate(RandomizedRunner.java:902)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleStoreClassName$1.evaluate(TestRuleStoreClassName.java:38)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.NoShadowingOrOverridesOnMethodsRule$1.evaluate(NoShadowingOrOverridesOnMethodsRule.java:40)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleAssertionsRequired$1.evaluate(TestRuleAssertionsRequired.java:53)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.AbstractBeforeAfterRule$1.evaluate(AbstractBeforeAfterRule.java:43)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleMarkFailure$1.evaluate(TestRuleMarkFailure.java:44)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleIgnoreAfterMaxFailures$1.evaluate(TestRuleIgnoreAfterMaxFailures.java:60)
15:40:14 > at org.apache.lucene.test_framework@10.0.0-SNAPSHOT/org.apache.lucene.tests.util.TestRuleIgnoreTestSuites$1.evaluate(TestRuleIgnoreTestSuites.java:47)
15:40:14 > at junit@4.13.1/org.junit.rules.RunRules.evaluate(RunRules.java:20)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.rules.StatementAdapter.evaluate(StatementAdapter.java:36)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.ThreadLeakControl$StatementRunner.run(ThreadLeakControl.java:390)
15:40:14 > at randomizedtesting.runner@2.8.0/com.carrotsearch.randomizedtesting.ThreadLeakControl.lambda$forkTimeoutingTask$0(ThreadLeakControl.java:850)
15:40:14 > at java.base/java.lang.Thread.run(Thread.java:833)
15:40:14 >
15:40:14 > Caused by:
15:40:14 > java.lang.NullPointerException: Cannot read field "numDimensions" because "props" is null
15:40:14 > at org.apache.lucene.core@10.0.0-SNAPSHOT/org.apache.lucene.index.FieldInfos$FieldNumbers.verifySameSchema(FieldInfos.java:459)
15:40:14 > at org.apache.lucene.core@10.0.0-SNAPSHOT/org.apache.lucene.index.FieldInfos$FieldNumbers.verifyFieldInfo(FieldInfos.java:359)
15:40:14 > at org.apache.lucene.core@10.0.0-SNAPSHOT/org.apache.lucene.index.IndexWriter.addIndexes(IndexWriter.java:3149)
15:40:14 > at org.apache.lucene.misc.index.IndexRearranger.addOneSegment(IndexRearranger.java:139)
15:40:14 > at org.apache.lucene.misc.index.IndexRearranger.lambda$execute$0(IndexRearranger.java:92)
15:40:14 > at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
15:40:14 > at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
15:40:14 > at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
15:40:14 > ... 1 more
```
* Use merge policy and merge scheduler to run addIndexes merges
* wrapped reader does not see deletes - debug
* Partially fixed tests in TestAddIndexes
* Use writer object to invoke addIndexes merge
* Use merge object info
* Add javadocs for new methods
* TestAddIndexes passing
* verify field info schemas upfront from incoming readers
* rename flag to track pooled readers
* Keep addIndexes API transactional
* Maintain transactionality - register segments with iw after all merges complete
* fix checkstyle
* PR comments
* Fix pendingDocs - numDocs mismatch bug
* Tests with 1-1 merges and partial merge failures
* variable renaming and better comments
* add test for partial merge failures. change tests to use 1-1 findmerges
* abort pending merges gracefully
* test null and empty merge specs
* test interim files are deleted
* test with empty readers
* test cascading merges triggered
* remove nocommits
* gradle check errors
* remove unused line
* remove printf
* spotless apply
* update TestIndexWriterOnDiskFull to accept mergeException from failing addIndexes calls
* return singleton reader mergespec in NoMergePolicy
* rethrow exceptions seen in merge threads on failure
* spotless apply
* update test to new exception type thrown
* spotlessApply
* test for maxDoc limit in IndexWriter
* spotlessApply
* Use DocValuesIterator instead of DocValuesFieldExistsQuery for counting soft deletes
* spotless apply
* change exception message for closed IW
* remove non-essential comments
* update api doc string
* doc string update
* spotless
* Changes file entry
* simplify findMerges API, add 1-1 merges to MockRandomMergePolicy
* update merge policies to new api
* remove unused imports
* spotless apply
* move changes entry to end of list
* fix testAddIndicesWithSoftDeletes
* test with 1-1 merge policy always enabled
* please spotcheck
* tidy
* test - never use 1-1 merge policy
* use 1-1 merge policy randomly
* Remove concurrent addIndexes findMerges from MockRandomMergePolicy
* Bug Fix: RuntimeException in addIndexes
Aborted pending merges were slipping through the merge exception check in
API, and getting caught later in the RuntimeException check.
* tidy
* Rebase on main. Move changes to 10.0
* Synchronize IW.AddIndexesMergeSource on outer class IW object
* tidy
I noticed some minor bugs in the original PR #927 that this PR should fix:
- When a timeout is set, we would no longer catch
`CollectionTerminatedException`.
- I added randomization to `LuceneTestCase` to randomly set a timeout, it
would have caught the above bug.
- Fixed visibility of `TimeLimitingBulkScorer`.
Before we were checking the number of vectors in the segment against the total
number of documents in IndexReader. This meant FieldExistsQuery would not
rewrite to MatchAllDocsQuery when there were multiple segments.
Instead of collecting hit-by-hit using a `LeafCollector`, we break down the
search by instantiating a weight, creating scorers, and checking the underlying
iterator. If it is backed by a `BitSet`, we directly update the reference (as
we won't be editing the `Bits`). Else we can create a new `BitSet` from the
iterator using `BitSet.of`.
The test used to leave hanging threads behind following a failure. Also one method was executing two different tests. I split the existing method into two and I am now leveraging setup and teardown to properly close all the resources both when the tests succeed as well as whey they fail.