Upgrade to Lucene 10.1.0 (#119308)

This commit upgrades to Lucene 10.1.0.
This commit is contained in:
Chris Hegarty 2025-01-30 13:41:02 +00:00 committed by GitHub
parent cdc16120fd
commit 4baffe4de1
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
50 changed files with 437 additions and 326 deletions

View file

@ -1,5 +1,5 @@
elasticsearch = 9.0.0
lucene = 10.0.0
lucene = 10.1.0
bundled_jdk_vendor = openjdk
bundled_jdk = 23+37@3c5b90190c68498b986a97f276efd28a

View file

@ -1,8 +1,8 @@
include::{docs-root}/shared/versions/stack/{source_branch}.asciidoc[]
:lucene_version: 10.0.0
:lucene_version_path: 10_0_0
:lucene_version: 10.1.0
:lucene_version_path: 10_1_0
:jdk: 11.0.2
:jdk_major: 11
:build_type: tar

View file

@ -0,0 +1,5 @@
pr: 119308
summary: Upgrade to Lucene 10.1.0
area: Search
type: upgrade
issues: []

View file

@ -176,7 +176,7 @@ The API returns the following result:
"time_in_nanos": 775274,
"children" : [
{
"name": "SimpleTopScoreDocCollector",
"name": "TopScoreDocCollector",
"reason": "search_top_hits",
"time_in_nanos": 775274
}
@ -537,7 +537,7 @@ Looking at the previous example:
"time_in_nanos": 775274,
"children" : [
{
"name": "SimpleTopScoreDocCollector",
"name": "TopScoreDocCollector",
"reason": "search_top_hits",
"time_in_nanos": 775274
}
@ -551,7 +551,7 @@ Looking at the previous example:
We see a top-level collector named `QueryPhaseCollector` which holds a child
`SimpleTopScoreDocCollector`. `SimpleTopScoreDocCollector` is the default
`TopScoreDocCollector`. `TopScoreDocCollector` is the default
"scoring and sorting" `Collector` used by {es}. The `reason` field attempts
to give a plain English description of the class name. The `time_in_nanos`
is similar to the time in the Query tree: a wall-clock time inclusive of all
@ -751,7 +751,7 @@ The API returns the following result:
"time_in_nanos": 1945072,
"children": [
{
"name": "SimpleTopScoreDocCollector",
"name": "TopScoreDocCollector",
"reason": "search_top_hits",
"time_in_nanos": 22577
},
@ -788,7 +788,7 @@ major portions of the query are represented:
2. The second `TermQuery` (message:search) represents the `post_filter` query.
The Collector tree is fairly straightforward, showing how a single
QueryPhaseCollector that holds the normal scoring SimpleTopScoreDocCollector
QueryPhaseCollector that holds the normal scoring TopScoreDocCollector
used to collect top hits, as well as BucketCollectorWrapper to run all scoped
aggregations.
@ -1332,7 +1332,7 @@ One of the `dfs.knn` sections for a shard looks like the following:
"rewrite_time" : 1275732,
"collector" : [
{
"name" : "SimpleTopScoreDocCollector",
"name" : "TopScoreDocCollector",
"reason" : "search_top_hits",
"time_in_nanos" : 17163
}

View file

@ -2920,129 +2920,179 @@
<sha256 value="015d5c229f3cd5c0ebf175c1da08d596d94043362ae9d92637d88848c90537c8" origin="Generated by Gradle"/>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-common" version="10.0.0">
<artifact name="lucene-analysis-common-10.0.0.jar">
<sha256 value="af703beb0898514a1c2d8dd58ac94fa14a6275398288482e1b9cd85f07d56912" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-common" version="10.1.0">
<artifact name="lucene-analysis-common-10.1.0.jar">
<sha256 value="31556c9a364c345677c5651532de4341cbf92663725a977df002e83adba2e91d" origin="Generated by Gradle">
<also-trust value="6e3730816710f1caa50342034c18fbd705a5f3ac88909279fa1f7c5e1e04405a"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-icu" version="10.0.0">
<artifact name="lucene-analysis-icu-10.0.0.jar">
<sha256 value="0c69ea85db9be3daf6ed692f01e8467ab9b8e8fcc3740169bfbb601e7c5d55e5" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-icu" version="10.1.0">
<artifact name="lucene-analysis-icu-10.1.0.jar">
<sha256 value="8fbb8f1506daf91ab9d6b8f504036cedd82eff8dba2e6a0124806a86f6776c45" origin="Generated by Gradle">
<also-trust value="a5702e4a8608e5ade6338103d8f3b264887e955b5429944acbdb9eb58f1aacdf"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-kuromoji" version="10.0.0">
<artifact name="lucene-analysis-kuromoji-10.0.0.jar">
<sha256 value="c12cb66f91752c5aa4969ad1a7775d268e5dec3d5e6e78b107fad2a02e897aac" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-kuromoji" version="10.1.0">
<artifact name="lucene-analysis-kuromoji-10.1.0.jar">
<sha256 value="7f0123efbe713a9f1fa27df64a4af884719057975eb0676667e327aac80a6add" origin="Generated by Gradle">
<also-trust value="b4b57c41e96f3c5ac528d9ead6e10d6f9888d0f03259275a5c68be6faa320745"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-morfologik" version="10.0.0">
<artifact name="lucene-analysis-morfologik-10.0.0.jar">
<sha256 value="99aa16783155c122833f61c84726eedce41b1b0778a861e84d853a3e5397fc6d" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-morfologik" version="10.1.0">
<artifact name="lucene-analysis-morfologik-10.1.0.jar">
<sha256 value="9f20bcb2ed691b375c40a15752e9f09906c4dac1d36a309472fd6705b3515639" origin="Generated by Gradle">
<also-trust value="96eed0039370bd402f5c73a611bdfcbce38d7a6b012bb99f8ae62099d8925f9b"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-nori" version="10.0.0">
<artifact name="lucene-analysis-nori-10.0.0.jar">
<sha256 value="5e80fa011ef360f2093fec157959d63af7766c13490f5d32fa3e82260edcf54e" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-nori" version="10.1.0">
<artifact name="lucene-analysis-nori-10.1.0.jar">
<sha256 value="c5779e7d987166fc926d17d9b3ded95fbbbfc08f2f0a08ab1ed1675cb8c48649" origin="Generated by Gradle">
<also-trust value="1e199f1fac7f839dd8946ea231b1d072ef985da2267b0f50648f2b023bc76d90"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-phonetic" version="10.0.0">
<artifact name="lucene-analysis-phonetic-10.0.0.jar">
<sha256 value="901320e21ce7a15548323be1628df9dc96d83ebca4e0bfa71b3d771bfff5a7bb" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-phonetic" version="10.1.0">
<artifact name="lucene-analysis-phonetic-10.1.0.jar">
<sha256 value="a5481e024ee7a7b240e880e6e8fc1af15a65ab835d8286d7daf4a34c6d5cad80" origin="Generated by Gradle">
<also-trust value="01cbba2cae0d2ec099a86124dd09dd4bea15eacb5e73c6f90caa2d02b5aa15e9"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-smartcn" version="10.0.0">
<artifact name="lucene-analysis-smartcn-10.0.0.jar">
<sha256 value="4d41a52e24f16fb7e9d96371362bf5bba5ae308c2740753fd1ff5e938800c6b3" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-smartcn" version="10.1.0">
<artifact name="lucene-analysis-smartcn-10.1.0.jar">
<sha256 value="bc46f95e6ba5dc95f6d546d9dea60ec9938a660617a49cd59dacd2a7abd5a52b" origin="Generated by Gradle">
<also-trust value="14411b9b5f4592d7f62c13fb0f97c07100cfc5e58e2b6ea66fe213fcf5e7945b"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-analysis-stempel" version="10.0.0">
<artifact name="lucene-analysis-stempel-10.0.0.jar">
<sha256 value="723ca9d879d341283cf2dbe3c6bdf5a13875165aaba6a197dc669152bf0c2047" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-analysis-stempel" version="10.1.0">
<artifact name="lucene-analysis-stempel-10.1.0.jar">
<sha256 value="ffd2571c296da3e82b06124ac43896d3828490256038f3243d26b1baefb2d451" origin="Generated by Gradle">
<also-trust value="03601385dbd511c67145e5923976942dc692d44ef3e8c385b78947de445b002c"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-backward-codecs" version="10.0.0">
<artifact name="lucene-backward-codecs-10.0.0.jar">
<sha256 value="a5439e70d0e0dbc6745354b30b3168eb39f5ea452cc7187a81d30f080f4e87e4" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-backward-codecs" version="10.1.0">
<artifact name="lucene-backward-codecs-10.1.0.jar">
<sha256 value="1c5aa8086d13a589dee4bbf79bd2cda6571fccd6fed6b7a56847b1bcc20231e5" origin="Generated by Gradle">
<also-trust value="4c5729631bfda4561277f493076c085889ecbf91c9e7e058fc3bbd658e78b3e8"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-codecs" version="10.0.0">
<artifact name="lucene-codecs-10.0.0.jar">
<sha256 value="517bd716e6773346b2a8e9f32a72c10b006c63ff6c27733d4808deb41cf01bda" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-codecs" version="10.1.0">
<artifact name="lucene-codecs-10.1.0.jar">
<sha256 value="5f48b0a529482b5007d60cb4bbfeb5dcc24f37bae9ae783f8998def63866e4a6" origin="Generated by Gradle">
<also-trust value="cee89f831ba1fb8af6f24aa1dd2018020daa585b48c7d7ab02f8609cdc5bdb59"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-core" version="10.0.0">
<artifact name="lucene-core-10.0.0.jar">
<sha256 value="ab187a10018d1e889624ef9e60b6fff0349c6315965b5087bdad0243a735f941" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-core" version="10.1.0">
<artifact name="lucene-core-10.1.0.jar">
<sha256 value="c9bb8d28327b21f31c7441bba1c96978231f877622c8a5a449f14254af0050e7" origin="Generated by Gradle">
<also-trust value="dfe584dbe6808942667e6f1bd558b3ab1dbc27b318fa1c53fb242f3289a7adc5"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-expressions" version="10.0.0">
<artifact name="lucene-expressions-10.0.0.jar">
<sha256 value="4b1b9ffaf2ca65ec57e4af52b76fcbfc975b31d5c836ba343ab471f7d884963e" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-expressions" version="10.1.0">
<artifact name="lucene-expressions-10.1.0.jar">
<sha256 value="8f4d6a5750f80173b48dde86ba80a75cb06c200888144e2a81b53c61f194467e" origin="Generated by Gradle">
<also-trust value="ab44f9e274843d55650eaf24dbb40e1b2062e241d07fa64a6db610db3d5b7d74"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-facet" version="10.0.0">
<artifact name="lucene-facet-10.0.0.jar">
<sha256 value="a760708d9ba1da12803d69a3027e5960b0cf8d6c05f7649716015dfe2a49ea77" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-facet" version="10.1.0">
<artifact name="lucene-facet-10.1.0.jar">
<sha256 value="6f8e4b58b858ec845f5bf02703e55bba7fdbc9c57d8c74d8e4f45437272fc9ea" origin="Generated by Gradle">
<also-trust value="f2c525ca5fb7341afd4a7ab346534bd5b201514365aeba5b62e908fb2e70acf0"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-grouping" version="10.0.0">
<artifact name="lucene-grouping-10.0.0.jar">
<sha256 value="311b875f88cebcdc93a4104d51302eb8c1c278dbcda8a4ae79467a634d4272dc" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-grouping" version="10.1.0">
<artifact name="lucene-grouping-10.1.0.jar">
<sha256 value="92620829184e82b45b9cdb53d28bdabd46a9fad8ee9bec683af10b9cff0b8815" origin="Generated by Gradle">
<also-trust value="cad8b601a9ce85999d34395b79b8e06b35942b11469e8ecee065be7f01e6d220"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-highlighter" version="10.0.0">
<artifact name="lucene-highlighter-10.0.0.jar">
<sha256 value="74a2e01e94b040dcaf4b5e989a0a262f2a6b41f41bc2b17e11c9bca3e2b239bb" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-highlighter" version="10.1.0">
<artifact name="lucene-highlighter-10.1.0.jar">
<sha256 value="674c89120f533ba518043600b7ad2da665be649c76d7fdfd92f85d25060366b9" origin="Generated by Gradle">
<also-trust value="c2b4a2e2dff74a9f9cd740b7c800a89ecbac3ce203cb67e2ead620fa1b70e72d"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-join" version="10.0.0">
<artifact name="lucene-join-10.0.0.jar">
<sha256 value="19b8326e56693824bbb1335d29e378d3288cd97add5c61538b502fb8666dd033" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-join" version="10.1.0">
<artifact name="lucene-join-10.1.0.jar">
<sha256 value="6ab3faee6c953dd7f8c972610cefa0250d815bf99fad37e435474067c25316b9" origin="Generated by Gradle">
<also-trust value="7ee2ed9a811db5b9f440bc5fc4c0ecae63da753bdba03fb12316661aae8f2ed5"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-memory" version="10.0.0">
<artifact name="lucene-memory-10.0.0.jar">
<sha256 value="40eb3a15025bd54ac6715a7f0f68ad3d0130d8a1226398b2d8fd2c7b82538cb5" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-memory" version="10.1.0">
<artifact name="lucene-memory-10.1.0.jar">
<sha256 value="2242750f139d41f9d311f806787f59308990161f8882218663322ae46d847427" origin="Generated by Gradle">
<also-trust value="810c208fda40863e3f692365acd96cdb0054eeefbb3bced8a77726cccb035291"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-misc" version="10.0.0">
<artifact name="lucene-misc-10.0.0.jar">
<sha256 value="ef3217c4fe4a294b98a122d6e252eb1d43801201369850345d6d995f1fc7a7b7" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-misc" version="10.1.0">
<artifact name="lucene-misc-10.1.0.jar">
<sha256 value="58a3bed8b66fba748c02ea8a537544493a2864a7705b500bddcee611d208fd64" origin="Generated by Gradle">
<also-trust value="d1faab3d7542e8f22bd212109ed242dd72240ec02833ebe7bbf72e05e9ef37af"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-queries" version="10.0.0">
<artifact name="lucene-queries-10.0.0.jar">
<sha256 value="2b0372c2d19dac6fb0eb7bc9ccf2dfe7211954d7c82a77c00d15bb5c2ae4196d" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-queries" version="10.1.0">
<artifact name="lucene-queries-10.1.0.jar">
<sha256 value="e009f809b585b736d09aacaf10a6455ba4a2a55f539569b348aee7c54602b67b" origin="Generated by Gradle">
<also-trust value="6200da82653f2cceee9bb93f7458b326172ed4fb74fe36d3d4d03c228320e1ce"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-queryparser" version="10.0.0">
<artifact name="lucene-queryparser-10.0.0.jar">
<sha256 value="29aff63cd8cd8ce6e8db1eeadf4b9634d5688ad1591866e23d6d9059afaf27a0" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-queryparser" version="10.1.0">
<artifact name="lucene-queryparser-10.1.0.jar">
<sha256 value="7de6ddeeb901eccc8233427d705562392fca21ea4f88f3dc1667aacdbd37366c" origin="Generated by Gradle">
<also-trust value="84888c96bcbee568a8f2187d7c4f0ef296c42fb697d27164e2d95d35e7306bb9"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-sandbox" version="10.0.0">
<artifact name="lucene-sandbox-10.0.0.jar">
<sha256 value="58bd8fb5f700b270b9971762269ef209f4c6b9eeadc7ff907b7e15cf6463740a" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-sandbox" version="10.1.0">
<artifact name="lucene-sandbox-10.1.0.jar">
<sha256 value="3525794a2c601ef8cbec472b5b75ce6fabf78eee85bd289b0dfd2eae12d2d59c" origin="Generated by Gradle">
<also-trust value="d96aae12b1c3f399bbfd104599c46493b206cea54d6ff0389a64b19cd0d063ab"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-spatial-extras" version="10.0.0">
<artifact name="lucene-spatial-extras-10.0.0.jar">
<sha256 value="e6e7708f51bd5a9ccd37774170a420478011c22e57b262242e207010962e8f01" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-spatial-extras" version="10.1.0">
<artifact name="lucene-spatial-extras-10.1.0.jar">
<sha256 value="511108f060da0f8cffebc98e7f12e869f7985aaa72e4dd81cc62287559e16084" origin="Generated by Gradle">
<also-trust value="b316782f8b0f5d4bc8709a193d17ca8c67c89a468be258af4b47de393a11956e"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-spatial3d" version="10.0.0">
<artifact name="lucene-spatial3d-10.0.0.jar">
<sha256 value="594738cc1b97dfc7dcd856fb10d7ebbeacf9ef14aa658555e4298ae4b6794978" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-spatial3d" version="10.1.0">
<artifact name="lucene-spatial3d-10.1.0.jar">
<sha256 value="8129fbdc884bf959267592bf0b6b2b2f7679740811d8f59e9f27106a615b5129" origin="Generated by Gradle">
<also-trust value="9f654dacef5d2ac262a023de5ecd7c4c37867de9dae48a709296f8c72b42dcf0"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-suggest" version="10.0.0">
<artifact name="lucene-suggest-10.0.0.jar">
<sha256 value="662851cf788fd76d846d89c1c9243e956af767402275924bb538468e7aeb35bc" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-suggest" version="10.1.0">
<artifact name="lucene-suggest-10.1.0.jar">
<sha256 value="bdad4d70408f26a66ee1fbaf79147188142e31b92aad1868bb2944980530691d" origin="Generated by Gradle">
<also-trust value="b0c6c8080f3bd4e1ba7c68cf6ed783749d719addef149fdcbad188a8590ab0b4"/>
</sha256>
</artifact>
</component>
<component group="org.apache.lucene" name="lucene-test-framework" version="10.0.0">
<artifact name="lucene-test-framework-10.0.0.jar">
<sha256 value="8bd7bd94136f5879a9748ab12c2b263a5519d8ce813bfc40d40f5bf086a7bebd" origin="Generated by Gradle"/>
<component group="org.apache.lucene" name="lucene-test-framework" version="10.1.0">
<artifact name="lucene-test-framework-10.1.0.jar">
<sha256 value="d778febde38dd14c1c4c898233d200724547d3d97f9926387842786b5d26fc73" origin="Generated by Gradle">
<also-trust value="9f68130df15d2a3c96918090d9ca747cd5dc4ab7b0928b19d7ab9097477d13b4"/>
</sha256>
</artifact>
</component>
<component group="org.apache.maven" name="maven-api-meta" version="4.0.0-alpha-9">

View file

@ -12,6 +12,7 @@ package org.elasticsearch.index.mapper.extras;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermStates;
import org.apache.lucene.index.memory.MemoryIndex;
@ -23,7 +24,6 @@ import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafSimScorer;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Matches;
@ -214,7 +214,6 @@ public final class SourceConfirmedTextQuery extends Query {
// No need to ever look at the _source for non-scoring term queries
return in.createWeight(searcher, scoreMode, boost);
}
// We use a LinkedHashSet here to preserve the ordering of terms to ensure that
// later summing of float scores per term is consistent
final Set<Term> terms = new LinkedHashSet<>();
@ -267,6 +266,7 @@ public final class SourceConfirmedTextQuery extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
NumericDocValues norms = context.reader().getNormValues(field);
RuntimePhraseScorer scorer = (RuntimePhraseScorer) scorerSupplier(context).get(0);
if (scorer == null) {
return Explanation.noMatch("No matching phrase");
@ -277,8 +277,7 @@ public final class SourceConfirmedTextQuery extends Query {
}
float phraseFreq = scorer.freq();
Explanation freqExplanation = Explanation.match(phraseFreq, "phraseFreq=" + phraseFreq);
final LeafSimScorer leafSimScorer = new LeafSimScorer(simScorer, context.reader(), field, scoreMode.needsScores());
Explanation scoreExplanation = leafSimScorer.explain(doc, freqExplanation);
Explanation scoreExplanation = simScorer.explain(freqExplanation, getNormValue(norms, doc));
return Explanation.match(
scoreExplanation.getValue(),
"weight(" + getQuery() + " in " + doc + ") [" + searcher.getSimilarity().getClass().getSimpleName() + "], result of:",
@ -297,9 +296,9 @@ public final class SourceConfirmedTextQuery extends Query {
public Scorer get(long leadCost) throws IOException {
final Scorer approximationScorer = approximationSupplier.get(leadCost);
final DocIdSetIterator approximation = approximationScorer.iterator();
final LeafSimScorer leafSimScorer = new LeafSimScorer(simScorer, context.reader(), field, scoreMode.needsScores());
final CheckedIntFunction<List<Object>, IOException> valueFetcher = valueFetcherProvider.apply(context);
return new RuntimePhraseScorer(approximation, leafSimScorer, valueFetcher, field, in);
NumericDocValues norms = context.reader().getNormValues(field);
return new RuntimePhraseScorer(approximation, simScorer, norms, valueFetcher, field, in);
}
@Override
@ -335,12 +334,23 @@ public final class SourceConfirmedTextQuery extends Query {
};
}
private static long getNormValue(NumericDocValues norms, int doc) throws IOException {
if (norms != null) {
boolean found = norms.advanceExact(doc);
assert found;
return norms.longValue();
} else {
return 1L; // default norm
}
}
private class RuntimePhraseScorer extends Scorer {
private final LeafSimScorer scorer;
private final SimScorer scorer;
private final CheckedIntFunction<List<Object>, IOException> valueFetcher;
private final String field;
private final Query query;
private final TwoPhaseIterator twoPhase;
private final NumericDocValues norms;
private final MemoryIndexEntry cacheEntry = new MemoryIndexEntry();
@ -349,12 +359,14 @@ public final class SourceConfirmedTextQuery extends Query {
private RuntimePhraseScorer(
DocIdSetIterator approximation,
LeafSimScorer scorer,
SimScorer scorer,
NumericDocValues norms,
CheckedIntFunction<List<Object>, IOException> valueFetcher,
String field,
Query query
) {
this.scorer = scorer;
this.norms = norms;
this.valueFetcher = valueFetcher;
this.field = field;
this.query = query;
@ -386,12 +398,12 @@ public final class SourceConfirmedTextQuery extends Query {
@Override
public float getMaxScore(int upTo) throws IOException {
return scorer.getSimScorer().score(Float.MAX_VALUE, 1L);
return scorer.score(Float.MAX_VALUE, 1L);
}
@Override
public float score() throws IOException {
return scorer.score(docID(), freq());
return scorer.score(freq(), getNormValue(norms, doc));
}
@Override

View file

@ -57,6 +57,8 @@ tasks.named("precommit").configure {
tasks.named("yamlRestCompatTestTransform").configure ({ task ->
task.replaceValueInMatch("profile.shards.0.dfs.knn.0.query.0.description", "DocAndScoreQuery[0,...][0.009673266,...],0.009673266", "dfs knn vector profiling")
task.replaceValueInMatch("profile.shards.0.dfs.knn.0.query.0.description", "DocAndScoreQuery[0,...][0.009673266,...],0.009673266", "dfs knn vector profiling with vector_operations_count")
task.replaceValueInMatch("profile.shards.0.dfs.knn.0.collector.0.name", "TopScoreDocCollector", "dfs knn vector profiling")
task.replaceValueInMatch("profile.shards.0.dfs.knn.0.collector.0.name", "TopScoreDocCollector", "dfs knn vector profiling with vector_operations_count")
task.skipTest("cat.aliases/10_basic/Deprecated local parameter", "CAT APIs not covered by compatibility policy")
task.skipTest("cat.shards/10_basic/Help", "sync_id is removed in 9.0")
task.skipTest("search/500_date_range/from, to, include_lower, include_upper deprecated", "deprecated parameters are removed in 9.0")

View file

@ -230,7 +230,6 @@ dfs knn vector profiling:
- gt: { profile.shards.0.dfs.knn.0.query.0.breakdown.create_weight: 0 }
- gt: { profile.shards.0.dfs.knn.0.query.0.breakdown.create_weight_count: 0 }
- gt: { profile.shards.0.dfs.knn.0.rewrite_time: 0 }
- match: { profile.shards.0.dfs.knn.0.collector.0.name: "SimpleTopScoreDocCollector" }
- match: { profile.shards.0.dfs.knn.0.collector.0.reason: "search_top_hits" }
- gt: { profile.shards.0.dfs.knn.0.collector.0.time_in_nanos: 0 }
@ -275,6 +274,47 @@ dfs knn vector profiling description:
- match: { hits.total.value: 1 }
- match: { profile.shards.0.dfs.knn.0.query.0.description: "DocAndScoreQuery[0,...][0.009673266,...],0.009673266" }
---
dfs knn vector profiling collector name:
- requires:
cluster_features: "lucene_10_1_upgrade"
reason: collector name changed with lucene 10.1
- do:
indices.create:
index: images
body:
settings:
index.number_of_shards: 1
mappings:
properties:
image:
type: "dense_vector"
dims: 3
index: true
similarity: "l2_norm"
- do:
index:
index: images
id: "1"
refresh: true
body:
image: [1, 5, -20]
- do:
search:
index: images
body:
profile: true
knn:
field: "image"
query_vector: [-5, 9, -12]
k: 1
num_candidates: 100
- match: { profile.shards.0.dfs.knn.0.collector.0.name: "TopScoreDocCollector" }
---
dfs knn vector profiling with vector_operations_count:
- requires:
@ -335,7 +375,6 @@ dfs knn vector profiling with vector_operations_count:
- gt: { profile.shards.0.dfs.knn.0.query.0.breakdown.create_weight: 0 }
- gt: { profile.shards.0.dfs.knn.0.query.0.breakdown.create_weight_count: 0 }
- gt: { profile.shards.0.dfs.knn.0.rewrite_time: 0 }
- match: { profile.shards.0.dfs.knn.0.collector.0.name: "SimpleTopScoreDocCollector" }
- match: { profile.shards.0.dfs.knn.0.collector.0.reason: "search_top_hits" }
- gt: { profile.shards.0.dfs.knn.0.collector.0.time_in_nanos: 0 }

View file

@ -105,7 +105,7 @@ public class FieldUsageStatsIT extends ESIntegTestCase {
assertTrue(stats.hasField("field2"));
// positions because of span query
assertEquals(
Set.of(UsageContext.TERMS, UsageContext.POSTINGS, UsageContext.FREQS, UsageContext.POSITIONS),
Set.of(UsageContext.TERMS, UsageContext.POSTINGS, UsageContext.FREQS, UsageContext.POSITIONS, UsageContext.NORMS),
stats.get("field2").keySet()
);
assertEquals(1L * numShards, stats.get("field2").getTerms());

View file

@ -461,7 +461,8 @@ module org.elasticsearch.server {
with
org.elasticsearch.index.codec.Elasticsearch814Codec,
org.elasticsearch.index.codec.Elasticsearch816Codec,
org.elasticsearch.index.codec.Elasticsearch900Codec;
org.elasticsearch.index.codec.Elasticsearch900Codec,
org.elasticsearch.index.codec.Elasticsearch900Lucene101Codec;
provides org.apache.logging.log4j.core.util.ContextDataProvider with org.elasticsearch.common.logging.DynamicContextDataProvider;

View file

@ -13,6 +13,7 @@ import org.apache.logging.log4j.Logger;
import org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat;
import org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat;
import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer;
@ -21,7 +22,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.ByteVectorValues;
import org.apache.lucene.index.DirectoryReader;
@ -306,6 +307,9 @@ final class IndexDiskUsageAnalyzer {
private static BlockTermState getBlockTermState(TermsEnum termsEnum, BytesRef term) throws IOException {
if (term != null && termsEnum.seekExact(term)) {
final TermState termState = termsEnum.termState();
if (termState instanceof final Lucene101PostingsFormat.IntBlockTermState blockTermState) {
return new BlockTermState(blockTermState.docStartFP, blockTermState.posStartFP, blockTermState.payStartFP);
}
if (termState instanceof final Lucene912PostingsFormat.IntBlockTermState blockTermState) {
return new BlockTermState(blockTermState.docStartFP, blockTermState.posStartFP, blockTermState.payStartFP);
}

View file

@ -70,7 +70,6 @@ import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.index.IndexVersion;
import org.elasticsearch.index.IndexVersions;
import org.elasticsearch.index.analysis.AnalyzerScope;
import org.elasticsearch.index.analysis.NamedAnalyzer;
@ -89,10 +88,9 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.apache.lucene.util.Version.LUCENE_10_0_0;
public class Lucene {
public static final String LATEST_CODEC = "Lucene100";
public static final String LATEST_CODEC = "Lucene101";
public static final String SOFT_DELETES_FIELD = "__soft_deletes";
@ -156,25 +154,7 @@ public class Lucene {
* Reads the segments infos from the given segments file name, failing if it fails to load
*/
private static SegmentInfos readSegmentInfos(String segmentsFileName, Directory directory) throws IOException {
// TODO Use readCommit(Directory directory, String segmentFileName, int minSupportedMajorVersion) once Lucene 10.1 is available
// and remove the try-catch block for IndexFormatTooOldException
assert IndexVersion.current().luceneVersion().equals(LUCENE_10_0_0) : "remove the try-catch block below";
try {
return SegmentInfos.readCommit(directory, segmentsFileName);
} catch (IndexFormatTooOldException e) {
try {
// Temporary workaround until Lucene 10.1 is available: try to leverage min. read-only compatibility to read the last commit
// and then check if this is the commit we want. This should always work for the case we are interested in (archive and
// searchable snapshots indices in N-2 version) as no newer commit should be ever written.
var segmentInfos = readSegmentInfos(directory);
if (segmentsFileName.equals(segmentInfos.getSegmentsFileName())) {
return segmentInfos;
}
} catch (Exception suppressed) {
e.addSuppressed(suppressed);
}
throw e;
}
return SegmentInfos.readCommit(directory, segmentsFileName, IndexVersions.MINIMUM_READONLY_COMPATIBLE.luceneVersion().major);
}
/**

View file

@ -144,6 +144,7 @@ public class IndexVersions {
public static final IndexVersion LOGSB_OPTIONAL_SORTING_ON_HOST_NAME = def(9_006_00_0, Version.LUCENE_10_0_0);
public static final IndexVersion SOURCE_MAPPER_MODE_ATTRIBUTE_NOOP = def(9_007_00_0, Version.LUCENE_10_0_0);
public static final IndexVersion HOSTNAME_DOC_VALUES_SPARSE_INDEX = def(9_008_00_0, Version.LUCENE_10_0_0);
public static final IndexVersion UPGRADE_TO_LUCENE_10_1_0 = def(9_009_00_0, Version.LUCENE_10_1_0);
/*
* STOP! READ THIS FIRST! No, really,
* ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _

View file

@ -12,7 +12,7 @@ package org.elasticsearch.index.codec;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.FeatureFlag;
import org.elasticsearch.core.Nullable;
@ -46,7 +46,7 @@ public class CodecService implements CodecProvider {
public CodecService(@Nullable MapperService mapperService, BigArrays bigArrays) {
final var codecs = new HashMap<String, Codec>();
Codec legacyBestSpeedCodec = new LegacyPerFieldMapperCodec(Lucene100Codec.Mode.BEST_SPEED, mapperService, bigArrays);
Codec legacyBestSpeedCodec = new LegacyPerFieldMapperCodec(Lucene101Codec.Mode.BEST_SPEED, mapperService, bigArrays);
if (ZSTD_STORED_FIELDS_FEATURE_FLAG.isEnabled()) {
codecs.put(DEFAULT_CODEC, new PerFieldMapperCodec(Zstd814StoredFieldsFormat.Mode.BEST_SPEED, mapperService, bigArrays));
} else {
@ -58,7 +58,7 @@ public class CodecService implements CodecProvider {
BEST_COMPRESSION_CODEC,
new PerFieldMapperCodec(Zstd814StoredFieldsFormat.Mode.BEST_COMPRESSION, mapperService, bigArrays)
);
Codec legacyBestCompressionCodec = new LegacyPerFieldMapperCodec(Lucene100Codec.Mode.BEST_COMPRESSION, mapperService, bigArrays);
Codec legacyBestCompressionCodec = new LegacyPerFieldMapperCodec(Lucene101Codec.Mode.BEST_COMPRESSION, mapperService, bigArrays);
codecs.put(LEGACY_BEST_COMPRESSION_CODEC, legacyBestCompressionCodec);
codecs.put(LUCENE_DEFAULT_CODEC, Codec.getDefault());

View file

@ -10,12 +10,12 @@
package org.elasticsearch.index.codec;
import org.apache.lucene.backward_codecs.lucene912.Lucene912Codec;
import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;

View file

@ -9,13 +9,13 @@
package org.elasticsearch.index.codec;
import org.apache.lucene.backward_codecs.lucene100.Lucene100Codec;
import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
@ -23,8 +23,8 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.elasticsearch.index.codec.zstd.Zstd814StoredFieldsFormat;
/**
* Elasticsearch codec as of 9.0. This extends the Lucene 10.0 codec to compressed stored fields with ZSTD instead of LZ4/DEFLATE. See
* {@link Zstd814StoredFieldsFormat}.
* Elasticsearch codec as of 9.0-snapshot relying on Lucene 10.0. This extends the Lucene 10.0 codec to compressed stored fields
* with ZSTD instead of LZ4/DEFLATE. See {@link Zstd814StoredFieldsFormat}.
*/
public class Elasticsearch900Codec extends CodecService.DeduplicateFieldInfosCodec {

View file

@ -0,0 +1,131 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.index.codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.elasticsearch.index.codec.zstd.Zstd814StoredFieldsFormat;
/**
* Elasticsearch codec as of 9.0 relying on Lucene 10.1. This extends the Lucene 10.1 codec to compressed
* stored fields with ZSTD instead of LZ4/DEFLATE. See {@link Zstd814StoredFieldsFormat}.
*/
public class Elasticsearch900Lucene101Codec extends CodecService.DeduplicateFieldInfosCodec {
private final StoredFieldsFormat storedFieldsFormat;
private final PostingsFormat defaultPostingsFormat;
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Elasticsearch900Lucene101Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat defaultDVFormat;
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Elasticsearch900Lucene101Codec.this.getDocValuesFormatForField(field);
}
};
private final KnnVectorsFormat defaultKnnVectorsFormat;
private final KnnVectorsFormat knnVectorsFormat = new PerFieldKnnVectorsFormat() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return Elasticsearch900Lucene101Codec.this.getKnnVectorsFormatForField(field);
}
};
/** Public no-arg constructor, needed for SPI loading at read-time. */
public Elasticsearch900Lucene101Codec() {
this(Zstd814StoredFieldsFormat.Mode.BEST_SPEED);
}
/**
* Constructor. Takes a {@link Zstd814StoredFieldsFormat.Mode} that describes whether to optimize for retrieval speed at the expense of
* worse space-efficiency or vice-versa.
*/
public Elasticsearch900Lucene101Codec(Zstd814StoredFieldsFormat.Mode mode) {
super("Elasticsearch900Lucene101", new Lucene101Codec());
this.storedFieldsFormat = mode.getFormat();
this.defaultPostingsFormat = new Lucene101PostingsFormat();
this.defaultDVFormat = new Lucene90DocValuesFormat();
this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat();
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return storedFieldsFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final KnnVectorsFormat knnVectorsFormat() {
return knnVectorsFormat;
}
/**
* Returns the postings format that should be used for writing new segments of <code>field</code>.
*
* <p>The default implementation always returns "Lucene912".
*
* <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
* future version of Lucene are only guaranteed to be able to read the default implementation,
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultPostingsFormat;
}
/**
* Returns the docvalues format that should be used for writing new segments of <code>field</code>
* .
*
* <p>The default implementation always returns "Lucene912".
*
* <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
* future version of Lucene are only guaranteed to be able to read the default implementation.
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
/**
* Returns the vectors format that should be used for writing new segments of <code>field</code>
*
* <p>The default implementation always returns "Lucene912".
*
* <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
* future version of Lucene are only guaranteed to be able to read the default implementation.
*/
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return defaultKnnVectorsFormat;
}
}

View file

@ -13,7 +13,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.mapper.MapperService;
@ -22,11 +22,11 @@ import org.elasticsearch.index.mapper.MapperService;
* Legacy version of {@link PerFieldMapperCodec}. This codec is preserved to give an escape hatch in case we encounter issues with new
* changes in {@link PerFieldMapperCodec}.
*/
public final class LegacyPerFieldMapperCodec extends Lucene100Codec {
public final class LegacyPerFieldMapperCodec extends Lucene101Codec {
private final PerFieldFormatSupplier formatSupplier;
public LegacyPerFieldMapperCodec(Lucene100Codec.Mode compressionMode, MapperService mapperService, BigArrays bigArrays) {
public LegacyPerFieldMapperCodec(Lucene101Codec.Mode compressionMode, MapperService mapperService, BigArrays bigArrays) {
super(compressionMode);
this.formatSupplier = new PerFieldFormatSupplier(mapperService, bigArrays);
// If the below assertion fails, it is a sign that Lucene released a new codec. You must create a copy of the current Elasticsearch

View file

@ -60,18 +60,18 @@ public class PerFieldFormatSupplier {
if (mapperService != null) {
Mapper mapper = mapperService.mappingLookup().getMapper(field);
if (mapper instanceof CompletionFieldMapper) {
return PostingsFormatHolder.POSTINGS_FORMAT;
return CompletionPostingsFormatHolder.POSTINGS_FORMAT;
}
}
// return our own posting format using PFOR
return es812PostingsFormat;
}
private static class PostingsFormatHolder {
private static final PostingsFormat POSTINGS_FORMAT = getPostingsFormat();
private static class CompletionPostingsFormatHolder {
private static final PostingsFormat POSTINGS_FORMAT = getCompletionPostingsFormat();
private static PostingsFormat getPostingsFormat() {
String defaultName = "Completion912"; // Caution: changing this name will result in exceptions if a field is created during a
private static PostingsFormat getCompletionPostingsFormat() {
String defaultName = "Completion101"; // Caution: changing this name will result in exceptions if a field is created during a
// rolling upgrade and the new codec (specified by the name) is not available on all nodes in the cluster.
String codecName = ExtensionLoader.loadSingleton(ServiceLoader.load(CompletionsPostingsFormatExtension.class))
.map(CompletionsPostingsFormatExtension::getFormatName)

View file

@ -26,7 +26,7 @@ import org.elasticsearch.index.mapper.MapperService;
* per index in real time via the mapping API. If no specific postings format or vector format is
* configured for a specific field the default postings or vector format is used.
*/
public final class PerFieldMapperCodec extends Elasticsearch900Codec {
public final class PerFieldMapperCodec extends Elasticsearch900Lucene101Codec {
private final PerFieldFormatSupplier formatSupplier;

View file

@ -126,7 +126,7 @@ public abstract class IntervalBuilder {
if (maxGaps == 0 && ordered) {
return Intervals.phrase(sourcesArray);
}
IntervalsSource inner = ordered ? XIntervals.ordered(sourcesArray) : XIntervals.unordered(sourcesArray);
IntervalsSource inner = ordered ? Intervals.ordered(sourcesArray) : Intervals.unordered(sourcesArray);
if (maxGaps == -1) {
return inner;
}

View file

@ -1,106 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.index.query;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.queries.intervals.IntervalIterator;
import org.apache.lucene.queries.intervals.IntervalMatchesIterator;
import org.apache.lucene.queries.intervals.Intervals;
import org.apache.lucene.queries.intervals.IntervalsSource;
import org.apache.lucene.search.QueryVisitor;
import java.io.IOException;
import java.util.Collection;
import java.util.Objects;
/**
* Copy of {@link Intervals} that exposes versions of {@link Intervals#ordered} and {@link Intervals#unordered}
* that preserve their inner gaps.
* NOTE: Remove this hack when a version of Lucene with https://github.com/apache/lucene/pull/13819 is used (10.1.0).
*/
public final class XIntervals {
/**
* Create an ordered {@link IntervalsSource}
*
* <p>Returns intervals in which the subsources all appear in the given order
*
* @param subSources an ordered set of {@link IntervalsSource} objects
*/
public static IntervalsSource ordered(IntervalsSource... subSources) {
return new DelegateIntervalsSource(Intervals.ordered(subSources));
}
/**
* Create an ordered {@link IntervalsSource}
*
* <p>Returns intervals in which the subsources all appear in the given order
*
* @param subSources an ordered set of {@link IntervalsSource} objects
*/
public static IntervalsSource unordered(IntervalsSource... subSources) {
return new DelegateIntervalsSource(Intervals.unordered(subSources));
}
/**
* Wraps a source to avoid aggressive flattening of the ordered and unordered sources.
* The flattening modifies the final gap and is removed in the latest unreleased version of Lucene (10.1).
*/
private static class DelegateIntervalsSource extends IntervalsSource {
private final IntervalsSource delegate;
private DelegateIntervalsSource(IntervalsSource delegate) {
this.delegate = delegate;
}
@Override
public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
return delegate.intervals(field, ctx);
}
@Override
public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
return delegate.matches(field, ctx, doc);
}
@Override
public void visit(String field, QueryVisitor visitor) {
delegate.visit(field, visitor);
}
@Override
public int minExtent() {
return delegate.minExtent();
}
@Override
public Collection<IntervalsSource> pullUpDisjunctions() {
return delegate.pullUpDisjunctions();
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DelegateIntervalsSource that = (DelegateIntervalsSource) o;
return Objects.equals(delegate, that.delegate);
}
@Override
public int hashCode() {
return Objects.hash(delegate);
}
@Override
public String toString() {
return delegate.toString();
}
}
}

View file

@ -17,10 +17,11 @@ import java.util.Set;
public final class SearchFeatures implements FeatureSpecification {
public static final NodeFeature LUCENE_10_0_0_UPGRADE = new NodeFeature("lucene_10_upgrade");
public static final NodeFeature LUCENE_10_1_0_UPGRADE = new NodeFeature("lucene_10_1_upgrade");
@Override
public Set<NodeFeature> getFeatures() {
return Set.of(LUCENE_10_0_0_UPGRADE);
return Set.of(LUCENE_10_0_0_UPGRADE, LUCENE_10_1_0_UPGRADE);
}
public static final NodeFeature RETRIEVER_RESCORER_ENABLED = new NodeFeature("search.retriever.rescorer.enabled");

View file

@ -462,7 +462,7 @@ public abstract class FiltersAggregator extends BucketsAggregator {
final int filterOrd;
FilterMatchingDisiWrapper(Scorer scorer, int ord) {
super(scorer);
super(scorer, false);
this.filterOrd = ord;
}

View file

@ -1,3 +1,4 @@
org.elasticsearch.index.codec.Elasticsearch814Codec
org.elasticsearch.index.codec.Elasticsearch816Codec
org.elasticsearch.index.codec.Elasticsearch900Codec
org.elasticsearch.index.codec.Elasticsearch900Lucene101Codec

View file

@ -12,7 +12,7 @@ package org.elasticsearch.action.admin.indices.diskusage;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@ -55,12 +55,11 @@ import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.ScorerSupplier;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.suggest.document.Completion912PostingsFormat;
import org.apache.lucene.search.suggest.document.Completion101PostingsFormat;
import org.apache.lucene.search.suggest.document.CompletionPostingsFormat;
import org.apache.lucene.search.suggest.document.SuggestField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.tests.geo.GeoTestUtil;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.util.BitSetIterator;
@ -328,11 +327,11 @@ public class IndexDiskUsageAnalyzerTests extends ESTestCase {
public void testCompletionField() throws Exception {
IndexWriterConfig config = new IndexWriterConfig().setCommitOnClose(true)
.setUseCompoundFile(false)
.setCodec(new Lucene100Codec(Lucene100Codec.Mode.BEST_SPEED) {
.setCodec(new Lucene101Codec(Lucene101Codec.Mode.BEST_SPEED) {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if (field.startsWith("suggest_")) {
return new Completion912PostingsFormat(randomFrom(CompletionPostingsFormat.FSTLoadMode.values()));
return new Completion101PostingsFormat(randomFrom(CompletionPostingsFormat.FSTLoadMode.values()));
} else {
return super.postingsFormat();
}
@ -415,25 +414,25 @@ public class IndexDiskUsageAnalyzerTests extends ESTestCase {
enum CodecMode {
BEST_SPEED {
@Override
Lucene100Codec.Mode mode() {
return Lucene100Codec.Mode.BEST_SPEED;
Lucene101Codec.Mode mode() {
return Lucene101Codec.Mode.BEST_SPEED;
}
},
BEST_COMPRESSION {
@Override
Lucene100Codec.Mode mode() {
return Lucene100Codec.Mode.BEST_COMPRESSION;
Lucene101Codec.Mode mode() {
return Lucene101Codec.Mode.BEST_COMPRESSION;
}
};
abstract Lucene100Codec.Mode mode();
abstract Lucene101Codec.Mode mode();
}
static void indexRandomly(Directory directory, CodecMode codecMode, int numDocs, Consumer<Document> addFields) throws IOException {
IndexWriterConfig config = new IndexWriterConfig().setCommitOnClose(true)
.setUseCompoundFile(randomBoolean())
.setCodec(new Lucene100Codec(codecMode.mode()));
.setCodec(new Lucene101Codec(codecMode.mode()));
try (IndexWriter writer = new IndexWriter(directory, config)) {
for (int i = 0; i < numDocs; i++) {
final Document doc = new Document();
@ -641,7 +640,7 @@ public class IndexDiskUsageAnalyzerTests extends ESTestCase {
try (DirectoryReader reader = DirectoryReader.open(source)) {
IndexWriterConfig config = new IndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD)
.setUseCompoundFile(randomBoolean())
.setCodec(new Lucene100Codec(mode.mode()) {
.setCodec(new Lucene101Codec(mode.mode()) {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return new ES812PostingsFormat();
@ -688,7 +687,7 @@ public class IndexDiskUsageAnalyzerTests extends ESTestCase {
final String[] files;
final Directory directory;
if (sis.getUseCompoundFile()) {
directory = sis.getCodec().compoundFormat().getCompoundReader(reader.directory(), sis, IOContext.DEFAULT);
directory = sis.getCodec().compoundFormat().getCompoundReader(reader.directory(), sis);
files = directory.listAll();
} else {
directory = reader.directory();

View file

@ -50,7 +50,7 @@ public class CodecTests extends ESTestCase {
assumeTrue("Only when zstd_stored_fields feature flag is enabled", CodecService.ZSTD_STORED_FIELDS_FEATURE_FLAG.isEnabled());
CodecService codecService = createCodecService();
assertThat(codecService.codec("default"), instanceOf(PerFieldMapperCodec.class));
assertThat(codecService.codec("default"), instanceOf(Elasticsearch900Codec.class));
assertThat(codecService.codec("default"), instanceOf(Elasticsearch900Lucene101Codec.class));
}
public void testDefault() throws Exception {

View file

@ -11,7 +11,7 @@ package org.elasticsearch.index.codec.vectors;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase;
import org.elasticsearch.common.logging.LogConfigurator;
@ -24,7 +24,7 @@ public class ES813FlatVectorFormatTests extends BaseKnnVectorsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES813FlatVectorFormat();

View file

@ -11,7 +11,7 @@ package org.elasticsearch.index.codec.vectors;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase;
import org.elasticsearch.common.logging.LogConfigurator;
@ -24,7 +24,7 @@ public class ES813Int8FlatVectorFormatTests extends BaseKnnVectorsFormatTestCase
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES813Int8FlatVectorFormat();

View file

@ -11,7 +11,7 @@ package org.elasticsearch.index.codec.vectors;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.KnnFloatVectorField;
@ -42,7 +42,7 @@ public class ES814HnswScalarQuantizedVectorsFormatTests extends BaseKnnVectorsFo
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES814HnswScalarQuantizedVectorsFormat();

View file

@ -11,7 +11,7 @@ package org.elasticsearch.index.codec.vectors;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.index.VectorSimilarityFunction;
import org.junit.Before;
@ -19,7 +19,7 @@ public class ES815BitFlatVectorFormatTests extends BaseKnnBitVectorsFormatTestCa
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES815BitFlatVectorFormat();

View file

@ -11,7 +11,7 @@ package org.elasticsearch.index.codec.vectors;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.index.VectorSimilarityFunction;
import org.junit.Before;
@ -19,7 +19,7 @@ public class ES815HnswBitVectorsFormatTests extends BaseKnnBitVectorsFormatTestC
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES815HnswBitVectorsFormat();

View file

@ -22,7 +22,7 @@ package org.elasticsearch.index.codec.vectors.es816;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.KnnFloatVectorField;
import org.apache.lucene.index.DirectoryReader;
@ -60,7 +60,7 @@ public class ES816BinaryQuantizedVectorsFormatTests extends BaseKnnVectorsFormat
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES816BinaryQuantizedRWVectorsFormat();

View file

@ -22,7 +22,7 @@ package org.elasticsearch.index.codec.vectors.es816;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.KnnFloatVectorField;
@ -56,7 +56,7 @@ public class ES816HnswBinaryQuantizedVectorsFormatTests extends BaseKnnVectorsFo
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES816HnswBinaryQuantizedRWVectorsFormat();

View file

@ -22,7 +22,7 @@ package org.elasticsearch.index.codec.vectors.es818;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.KnnFloatVectorField;
import org.apache.lucene.index.DirectoryReader;
@ -60,7 +60,7 @@ public class ES818BinaryQuantizedVectorsFormatTests extends BaseKnnVectorsFormat
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES818BinaryQuantizedVectorsFormat();

View file

@ -22,7 +22,7 @@ package org.elasticsearch.index.codec.vectors.es818;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.KnnVectorsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.KnnFloatVectorField;
@ -56,7 +56,7 @@ public class ES818HnswBinaryQuantizedVectorsFormatTests extends BaseKnnVectorsFo
@Override
protected Codec getCodec() {
return new Lucene100Codec() {
return new Lucene101Codec() {
@Override
public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
return new ES818HnswBinaryQuantizedVectorsFormat();

View file

@ -10,7 +10,7 @@
package org.elasticsearch.index.codec.zstd;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DirectoryReader;
@ -35,13 +35,13 @@ public class StoredFieldCodecDuelTests extends ESTestCase {
private static final String DOUBLE_FIELD = "double_field_5";
public void testDuelBestSpeed() throws IOException {
var baseline = new LegacyPerFieldMapperCodec(Lucene100Codec.Mode.BEST_SPEED, null, BigArrays.NON_RECYCLING_INSTANCE);
var baseline = new LegacyPerFieldMapperCodec(Lucene101Codec.Mode.BEST_SPEED, null, BigArrays.NON_RECYCLING_INSTANCE);
var contender = new PerFieldMapperCodec(Zstd814StoredFieldsFormat.Mode.BEST_SPEED, null, BigArrays.NON_RECYCLING_INSTANCE);
doTestDuel(baseline, contender);
}
public void testDuelBestCompression() throws IOException {
var baseline = new LegacyPerFieldMapperCodec(Lucene100Codec.Mode.BEST_COMPRESSION, null, BigArrays.NON_RECYCLING_INSTANCE);
var baseline = new LegacyPerFieldMapperCodec(Lucene101Codec.Mode.BEST_COMPRESSION, null, BigArrays.NON_RECYCLING_INSTANCE);
var contender = new PerFieldMapperCodec(Zstd814StoredFieldsFormat.Mode.BEST_COMPRESSION, null, BigArrays.NON_RECYCLING_INSTANCE);
doTestDuel(baseline, contender);
}

View file

@ -11,11 +11,11 @@ package org.elasticsearch.index.codec.zstd;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase;
import org.elasticsearch.index.codec.Elasticsearch900Codec;
import org.elasticsearch.index.codec.Elasticsearch900Lucene101Codec;
public class Zstd814BestCompressionStoredFieldsFormatTests extends BaseStoredFieldsFormatTestCase {
private final Codec codec = new Elasticsearch900Codec(Zstd814StoredFieldsFormat.Mode.BEST_COMPRESSION);
private final Codec codec = new Elasticsearch900Lucene101Codec(Zstd814StoredFieldsFormat.Mode.BEST_COMPRESSION);
@Override
protected Codec getCodec() {

View file

@ -11,11 +11,11 @@ package org.elasticsearch.index.codec.zstd;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase;
import org.elasticsearch.index.codec.Elasticsearch900Codec;
import org.elasticsearch.index.codec.Elasticsearch900Lucene101Codec;
public class Zstd814BestSpeedStoredFieldsFormatTests extends BaseStoredFieldsFormatTestCase {
private final Codec codec = new Elasticsearch900Codec(Zstd814StoredFieldsFormat.Mode.BEST_SPEED);
private final Codec codec = new Elasticsearch900Lucene101Codec(Zstd814StoredFieldsFormat.Mode.BEST_SPEED);
@Override
protected Codec getCodec() {

View file

@ -9,12 +9,12 @@
package org.elasticsearch.index.engine;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene100.Lucene100Codec;
import org.apache.lucene.codecs.lucene101.Lucene101Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.search.suggest.document.Completion912PostingsFormat;
import org.apache.lucene.search.suggest.document.Completion101PostingsFormat;
import org.apache.lucene.search.suggest.document.SuggestField;
import org.apache.lucene.store.Directory;
import org.elasticsearch.ElasticsearchException;
@ -44,8 +44,8 @@ public class CompletionStatsCacheTests extends ESTestCase {
public void testCompletionStatsCache() throws IOException, InterruptedException {
final IndexWriterConfig indexWriterConfig = newIndexWriterConfig();
final PostingsFormat postingsFormat = new Completion912PostingsFormat();
indexWriterConfig.setCodec(new Lucene100Codec() {
final PostingsFormat postingsFormat = new Completion101PostingsFormat();
indexWriterConfig.setCodec(new Lucene101Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postingsFormat; // all fields are suggest fields

View file

@ -16,7 +16,7 @@ import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.suggest.document.Completion912PostingsFormat;
import org.apache.lucene.search.suggest.document.Completion101PostingsFormat;
import org.apache.lucene.search.suggest.document.CompletionAnalyzer;
import org.apache.lucene.search.suggest.document.ContextSuggestField;
import org.apache.lucene.search.suggest.document.FuzzyCompletionQuery;
@ -148,21 +148,19 @@ public class CompletionFieldMapperTests extends MapperTestCase {
}
public void testPostingsFormat() throws IOException {
final Class<?> latestLuceneCPClass = Completion101PostingsFormat.class;
MapperService mapperService = createMapperService(fieldMapping(this::minimalMapping));
CodecService codecService = new CodecService(mapperService, BigArrays.NON_RECYCLING_INSTANCE);
Codec codec = codecService.codec("default");
if (CodecService.ZSTD_STORED_FIELDS_FEATURE_FLAG.isEnabled()) {
assertThat(codec, instanceOf(PerFieldMapperCodec.class));
assertThat(((PerFieldMapperCodec) codec).getPostingsFormatForField("field"), instanceOf(Completion912PostingsFormat.class));
assertThat(((PerFieldMapperCodec) codec).getPostingsFormatForField("field"), instanceOf(latestLuceneCPClass));
} else {
if (codec instanceof CodecService.DeduplicateFieldInfosCodec deduplicateFieldInfosCodec) {
codec = deduplicateFieldInfosCodec.delegate();
}
assertThat(codec, instanceOf(LegacyPerFieldMapperCodec.class));
assertThat(
((LegacyPerFieldMapperCodec) codec).getPostingsFormatForField("field"),
instanceOf(Completion912PostingsFormat.class)
);
assertThat(((LegacyPerFieldMapperCodec) codec).getPostingsFormatForField("field"), instanceOf(latestLuceneCPClass));
}
}

View file

@ -129,7 +129,6 @@ public class DocumentParserContextTests extends ESTestCase {
var resultFromParserContext = parserContext.createDynamicMapperBuilderContext();
assertEquals("foo.hey", resultFromParserContext.buildFullName("hey"));
assertTrue(resultFromParserContext.isSourceSynthetic());
assertTrue(resultFromParserContext.isDataStream());
assertTrue(resultFromParserContext.parentObjectContainsDimensions());
assertEquals(ObjectMapper.Defaults.DYNAMIC, resultFromParserContext.getDynamic());

View file

@ -102,10 +102,6 @@ public class SparseVectorFieldMapperTests extends MapperTestCase {
List<IndexableField> fields = doc1.rootDoc().getFields("field");
assertEquals(2, fields.size());
if (IndexVersion.current().luceneVersion().major == 10) {
// TODO: Update to use Lucene's FeatureField after upgrading to Lucene 10.1.
assertThat(IndexVersion.current().luceneVersion().minor, equalTo(0));
}
assertThat(fields.get(0), Matchers.instanceOf(XFeatureField.class));
XFeatureField featureField1 = null;
XFeatureField featureField2 = null;

View file

@ -46,7 +46,7 @@ public class IntervalBuilderTests extends ESTestCase {
CannedTokenStream ts = new CannedTokenStream(new Token("term1", 1, 2), new Token("term2", 3, 4), new Token("term3", 5, 6));
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, true);
IntervalsSource expected = XIntervals.ordered(Intervals.term("term1"), Intervals.term("term2"), Intervals.term("term3"));
IntervalsSource expected = Intervals.ordered(Intervals.term("term1"), Intervals.term("term2"), Intervals.term("term3"));
assertEquals(expected, source);
@ -57,7 +57,7 @@ public class IntervalBuilderTests extends ESTestCase {
CannedTokenStream ts = new CannedTokenStream(new Token("term1", 1, 2), new Token("term2", 3, 4), new Token("term3", 5, 6));
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, false);
IntervalsSource expected = XIntervals.unordered(Intervals.term("term1"), Intervals.term("term2"), Intervals.term("term3"));
IntervalsSource expected = Intervals.unordered(Intervals.term("term1"), Intervals.term("term2"), Intervals.term("term3"));
assertEquals(expected, source);
@ -101,7 +101,7 @@ public class IntervalBuilderTests extends ESTestCase {
);
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, true);
IntervalsSource expected = XIntervals.ordered(
IntervalsSource expected = Intervals.ordered(
Intervals.term("term1"),
Intervals.or(Intervals.term("term2"), Intervals.term("term4")),
Intervals.term("term3")
@ -122,7 +122,7 @@ public class IntervalBuilderTests extends ESTestCase {
);
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, true);
IntervalsSource expected = XIntervals.ordered(
IntervalsSource expected = Intervals.ordered(
Intervals.term("term1"),
Intervals.extend(Intervals.or(Intervals.term("term2"), Intervals.term("term3"), Intervals.term("term4")), 1, 0),
Intervals.term("term5")
@ -143,7 +143,7 @@ public class IntervalBuilderTests extends ESTestCase {
);
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, true);
IntervalsSource expected = XIntervals.ordered(
IntervalsSource expected = Intervals.ordered(
Intervals.term("term1"),
Intervals.or(Intervals.term("term2"), Intervals.phrase("term3", "term4")),
Intervals.term("term5")
@ -166,7 +166,7 @@ public class IntervalBuilderTests extends ESTestCase {
);
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, true);
IntervalsSource expected = XIntervals.ordered(
IntervalsSource expected = Intervals.ordered(
Intervals.term("term1"),
Intervals.or(
Intervals.extend(Intervals.term("term2"), 1, 0),
@ -190,7 +190,7 @@ public class IntervalBuilderTests extends ESTestCase {
);
IntervalsSource source = BUILDER.analyzeText(new CachingTokenFilter(ts), -1, true);
IntervalsSource expected = XIntervals.ordered(
IntervalsSource expected = Intervals.ordered(
Intervals.term("term1"),
Intervals.or(Intervals.term("term2"), Intervals.phrase("term3", "term4")),
Intervals.extend(Intervals.term("term5"), 1, 0)

View file

@ -203,7 +203,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
}""", TEXT_FIELD_NAME);
IntervalQueryBuilder builder = (IntervalQueryBuilder) parseQuery(json);
Query expected = new IntervalQuery(TEXT_FIELD_NAME, XIntervals.unordered(Intervals.term("hello"), Intervals.term("world")));
Query expected = new IntervalQuery(TEXT_FIELD_NAME, Intervals.unordered(Intervals.term("hello"), Intervals.term("world")));
assertEquals(expected, builder.toQuery(createSearchExecutionContext()));
@ -222,7 +222,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
builder = (IntervalQueryBuilder) parseQuery(json);
expected = new IntervalQuery(
TEXT_FIELD_NAME,
Intervals.maxgaps(40, XIntervals.unordered(Intervals.term("hello"), Intervals.term("world")))
Intervals.maxgaps(40, Intervals.unordered(Intervals.term("hello"), Intervals.term("world")))
);
assertEquals(expected, builder.toQuery(createSearchExecutionContext()));
@ -241,7 +241,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
builder = (IntervalQueryBuilder) parseQuery(json);
expected = new BoostQuery(
new IntervalQuery(TEXT_FIELD_NAME, XIntervals.ordered(Intervals.term("hello"), Intervals.term("world"))),
new IntervalQuery(TEXT_FIELD_NAME, Intervals.ordered(Intervals.term("hello"), Intervals.term("world"))),
2
);
assertEquals(expected, builder.toQuery(createSearchExecutionContext()));
@ -263,7 +263,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
builder = (IntervalQueryBuilder) parseQuery(json);
expected = new IntervalQuery(
TEXT_FIELD_NAME,
Intervals.maxgaps(10, XIntervals.ordered(Intervals.term("Hello"), Intervals.term("world")))
Intervals.maxgaps(10, Intervals.ordered(Intervals.term("Hello"), Intervals.term("world")))
);
assertEquals(expected, builder.toQuery(createSearchExecutionContext()));
@ -285,7 +285,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
builder = (IntervalQueryBuilder) parseQuery(json);
expected = new IntervalQuery(
TEXT_FIELD_NAME,
Intervals.fixField(MASKED_FIELD, Intervals.maxgaps(10, XIntervals.ordered(Intervals.term("Hello"), Intervals.term("world"))))
Intervals.fixField(MASKED_FIELD, Intervals.maxgaps(10, Intervals.ordered(Intervals.term("Hello"), Intervals.term("world"))))
);
assertEquals(expected, builder.toQuery(createSearchExecutionContext()));
@ -314,7 +314,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
expected = new IntervalQuery(
TEXT_FIELD_NAME,
Intervals.containing(
Intervals.maxgaps(10, XIntervals.ordered(Intervals.term("Hello"), Intervals.term("world"))),
Intervals.maxgaps(10, Intervals.ordered(Intervals.term("Hello"), Intervals.term("world"))),
Intervals.term("blah")
)
);
@ -426,7 +426,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
Intervals.containedBy(
Intervals.maxgaps(
30,
XIntervals.ordered(Intervals.term("one"), XIntervals.unordered(Intervals.term("two"), Intervals.term("three")))
Intervals.ordered(Intervals.term("one"), Intervals.unordered(Intervals.term("two"), Intervals.term("three")))
),
Intervals.term("SENTENCE")
)
@ -486,7 +486,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
Intervals.notContainedBy(
Intervals.maxgaps(
30,
XIntervals.ordered(Intervals.term("atmosphere"), Intervals.or(Intervals.term("cold"), Intervals.term("outside")))
Intervals.ordered(Intervals.term("atmosphere"), Intervals.or(Intervals.term("cold"), Intervals.term("outside")))
),
Intervals.term("freeze")
)

View file

@ -86,14 +86,14 @@ public class DfsPhaseTests extends ESTestCase {
List<QueryProfileShardResult> queryProfileShardResult = searchProfileDfsPhaseResult.getQueryProfileShardResult();
assertNotNull(queryProfileShardResult);
CollectorResult collectorResult = queryProfileShardResult.get(0).getCollectorResult();
assertEquals("SimpleTopScoreDocCollector", (collectorResult.getName()));
assertEquals("TopScoreDocCollector", (collectorResult.getName()));
assertEquals("search_top_hits", (collectorResult.getReason()));
assertTrue(collectorResult.getTime() > 0);
List<CollectorResult> children = collectorResult.getChildrenResults();
if (children.size() > 0) {
long totalTime = 0L;
for (CollectorResult child : children) {
assertEquals("SimpleTopScoreDocCollector", (child.getName()));
assertEquals("TopScoreDocCollector", (child.getName()));
assertEquals("search_top_hits", (child.getReason()));
totalTime += child.getTime();
}

View file

@ -135,7 +135,7 @@ public class ProfileCollectorManagerTests extends ESTestCase {
assertEquals(numDocs, topDocs.totalHits.value());
CollectorResult result = profileCollectorManager.getCollectorTree();
assertEquals("profiler_reason", result.getReason());
assertEquals("SimpleTopScoreDocCollector", result.getName());
assertEquals("TopScoreDocCollector", result.getName());
assertTrue(result.getTime() > 0);
}
}

View file

@ -216,9 +216,7 @@ public class SourceOnlySnapshot {
Codec codec = si.getCodec();
Directory sourceDir = si.dir;
if (si.getUseCompoundFile()) {
sourceDir = new LinkedFilesDirectory.CloseMePleaseWrapper(
codec.compoundFormat().getCompoundReader(sourceDir, si, IOContext.DEFAULT)
);
sourceDir = new LinkedFilesDirectory.CloseMePleaseWrapper(codec.compoundFormat().getCompoundReader(sourceDir, si));
toClose = sourceDir;
}
final String segmentSuffix = "";

View file

@ -51,7 +51,7 @@ public class QueryFeatureExtractor implements FeatureExtractor {
}
Scorer scorer = weight.scorer(segmentContext);
if (scorer != null) {
disiPriorityQueue.add(new DisiWrapper(scorer));
disiPriorityQueue.add(new DisiWrapper(scorer, false));
}
scorers.add(scorer);
}

View file

@ -570,11 +570,11 @@ public class WildcardFieldMapperTests extends MapperTestCase {
{ "(maynotexist)?foobar", "+eoo +ooa +oaa +aaq +aq_ +q__" },
{ ".*/etc/passw.*", "+\\/es +esc +sc\\/ +c\\/o +\\/oa +oas +ass +ssw" },
{ ".*etc/passwd", " +esc +sc\\/ +c\\/o +\\/oa +oas +ass +ssw +swc +wc_ +c__" },
{ "(http|ftp)://foo.*", "+((+gss +sso) eso) +(+\\/\\/\\/ +\\/\\/e +\\/eo +eoo)" },
{ "(http|ftp)://foo.*", "+\\/\\/\\/ +\\/\\/e +\\/eo +eoo +((+gss +sso) eso)" },
{
"[Pp][Oo][Ww][Ee][Rr][Ss][Hh][Ee][Ll][Ll]\\.[Ee][Xx][Ee]",
"+_oo +oow +owe +weq +eqs +qsg +sge +gek +ekk +kk\\/ +k\\/e +\\/ew +ewe +we_ +e__" },
{ "foo<1-100>bar", "+(+_eo +eoo) +(+aaq +aq_ +q__)" },
{ "foo<1-100>bar", "+_eo +eoo +aaq +aq_ +q__" },
{ "(aaa.+&.+bbb)cat", "+cas +as_ +s__" },
{ ".a", "a__" } };
for (String[] test : acceleratedTests) {