* Hunspell: reduce suggestion set dependency on the hash table order
When adding words to a dictionary, suggestions for other words shouldn't change unless they're directly related to the added words.
But before, GeneratingSuggester selected 100 best first matches from the hash table, whose order can change significantly after adding any unrelated word.
That resulted in unexpected suggestion changes on seemingly unrelated dictionary edits.
if we only have custom-case uART and capitalized UART, we shouldn't accept StandUart as a compound (although we keep hidden "Uart" dictionary entries for internal purposes)
After upgrading Elasticsearch to a recent Lucene snapshot, we observed a few
indexing slowdowns when indexing with low numbers of cores. This appears to be
due to the fact that we lost too much of the bias towards larger DWPTs in
apache/lucene#12199. This change tries to add back more ordering by adjusting
the concurrency of `DWPTPool` to the number of cores that are available on the
local node.
Given an input text 'A B A C A B C' and search ORDERED(A, B, C), we should
retrieve hits [0,3] and [4,6]; currently [4,6] is skipped.
After finding the first interval [0, 3], the subintervals will become A[0,0], B[1,1],
C[3,3]; then the algorithm will try to minimize it and the subintervals will
become: A:[2,2], B:[5,5], C:[3,3] (after finding 5 > 3 it breaks the minimization)
And when finding next interval, it will do advance(B) before checking whether
it is after A(the do-while loop), so subintervals will become A[2,2], B[inf, inf],
C[3,3] and return NO_MORE_INTERVAL.
This commit instead continues advancing subintervals from where the last
`nextInterval` call stopped, rather than always advancing all subintervals.
Currently we're only half reusing postings enums when flushing sorted indexes
as we still create new wrapper instances every time, which can be costly with
fields that have many terms.
Obtaining a DWPT and putting it back into the pool is subject to contention.
This change reduces contention by using 8 sub pools that are tried sequentially.
When applied on top of #12198, this reduces the time to index geonames with 20
threads from ~19s to ~16-17s.
This switches to LSBRadixSorter instead of TimSorter to sort postings whose
index options are `DOCS`. On a synthetic benchmark this yielded barely any
difference in the case when the index order is the same as the sort order, or
reverse, but almost a 3x speedup for writing postings in the case when the
index order is mostly random.
lucene-util's `IndexGeoNames` benchmark is heavily contended when running with
many indexing threads, 20 in my case. The main offender is
`DocumentsWriterFlushControl#doAfterDocument`, which runs after every index
operation to update doc and RAM accounting.
This change reduces contention by only updating RAM accounting if the amount of
RAM consumption that has not been committed yet by a single DWPT is at least
0.1% of the total RAM buffer size. This effectively batches updates to RAM
accounting, similarly to what happens when using `IndexWriter#addDocuments` to
index multiple documents at once. Since updates to RAM accounting may be
batched, `FlushPolicy` can no longer distinguish between inserts, updates and
deletes, so all 3 methods got merged into a single one.
With this change, `IndexGeoNames` goes from ~22s to ~19s and the main offender
for contention is now `DocumentsWriterPerThreadPool#getAndLock`.
Co-authored-by: Simon Willnauer <simonw@apache.org>
In Lucene 5.4.0 62313b83ba9c69379e1f84dffc881a361713ce9 introduced some changes for immutability of queries. setBoost() function was replaced with new BoostQuery(), but BoostQuery is not handled in setSlop function. This commit adds the handling of BoostQuery in setSlop() function.
GH#11744 deprecated LongValueFacetCounts#getTopChildrenSortByCount in favor
of the standard Facets#getTopChildren. The issue is that #getTopChildrenSortByCount
didn't do any input validation and allowed for topN == 0, while #getTopChildren
does input validation. Randomized testing could produce topN values of 0, which
resulted in falied tests. This addresses the tests.
* Define inputs and outputs for task validateJarLicenses
* Lazily configure validateJarLicenses
* Move functionality from copyTestResources task into processTestResources task
* Lazily configure processTestResources
* Altered TestCustomAnalyzer.testStopWordsFromFile() to find resources in updated location
* Resolve "overlapping output" issue preventing processTestResources from being cached
* Provide system properties from CommandLineArgumentProviders
* Configure certain system properties as inputs to take advantage of UP-TO-DATE checking
* Applies the correct pathing strategies to take full advantage of caching even if builds are executed from different locations on disk
* Make validateSourcePatterns task cacheable by removing .gradle directory from its input
- 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>
This change introduces `MultiTermQuery#CONSTANT_SCORE_BLENDED_REWRITE`, a new rewrite method
meant to be used in place of `MultiTermQuery#CONSTANT_SCORE_REWRITE` as the default for multi-term
queries that act as a filter. Currently, multi-term queries with a filter rewrite internally rewrite to a
disjunction if 16 terms or less match the query. Otherwise postings lists of
matching terms are collected into a `DocIdSetBuilder`. This change replaces the
latter with a mixed approach where a disjunction is created between the 16
terms that have the highest document frequency and an iterator produced from
the `DocIdSetBuilder` that collects all other terms. On fields that have a
zipfian distribution, it's quite likely that no high-frequency terms make it to
the `DocIdSetBuilder`. This provides two main benefits:
- Queries are less likely to allocate a FixedBitSet of size `maxDoc`.
- Queries are better at skipping or early terminating.
On the other hand, queries that need to consume most or all matching documents may get a slowdown, so
users can still opt-in to the "full filter rewrite" functionality by overriding the rewrite method. This is the new
default for PrefixQuery, WildcardQuery and TermRangeQuery.
Co-authored-by: Adrien Grand <jpountz@gmail.com> / Greg Miller <gsmiller@gmail.com>
The default implementation of merging doc values resolves the ordinal of a
document in `nextDoc()`. But sometimes, doc values iterators are consumed
without retrieving ordinals, e.g. to write the set of documents that have a
value, so this may be wasteful.
With this change, ordinals get resolved lazily upon `ordValue()`.
`LogMergePolicy` has this boundary at the floor level that prevents merging
segments above the minimum segment size with segments below this size. I cannot
see a benefit from doing this, and no tests fail if I remove it, while this
boundary has the downside of not running merges that seem legit to me. Should
we remove this boundary check?
Indexing simple keywords through a `TokenStream` abstraction introduces a bit
of overhead due to attribute management. Not much, but indexing keywords boils
down to adding to a hash map and appending to a postings list, which is quite
cheap too so even some low overhead can significantly impact indexing speed.
The two minor performance improvements are around count and Weight#scorer.
segmentStarts is a monotonically increasing start for each scored document indexed by leaf-segment ordinal. Consequently, if the upper and lower segments are equivalent, that means no docs match for this segment.
Count is similarly calculated by the difference between upper and lower segmentStarts according to the segment ordinal.
Similar to use of ScorerSupplier in #12129, implement it here too,
because creation of a Scorer requires lookupTerm() operations in the DV
terms dictionary. This results in wasted effort/random accesses, if, based on the cost(),
IndexOrDocValuesQuery decides not to use this query.
The helper class DocAndScoreQuery implements advanceShallow to help skip
non-competitive documents. This method doesn't actually keep track of where it
has advanced, which means it can do extra work.
Overall the complexity here didn't seem worth it, given the low cost of
collecting matching kNN docs. This PR switches to a simpler approach, which uses
a fixed upper bound on the max score.
This change adjusts the cache policy to ensure that all segments in the
max tier to be cached. Before, we cache segments that have more than 3%
of the total documents in the index; now cache segments have more than
half of the average documents per leave of the index.
Closes#12140
The test assumes a single segment is created (because only one scorer is created from the leaf contexts).
But, force merging wasn't done before getting the reader. Forcemerge to a single segment before getting the reader.
In TestUnifiedHighlighterTermVec, we have a special reader which counts the
number of times term vectors are accessed, so that we can assert that caching
works correctly here. There is some special logic in place to skip the check
when the test framework wraps readers with CheckIndex or ParallelReader;
however, this logic no longer works with ParallelReader in particular, because
term vectors are now accessed through an anonymous class.
A simpler solution here is to call newSearcher(reader, false), which disables
wrapping, meaning that we can remove this extra logic entirely.
Fixes#12115
* 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