Convert IndexVersion.CURRENT to a method with a pluggable interface (#97132)

This commit is contained in:
Simon Cooper 2023-06-27 14:47:32 +01:00 committed by GitHub
parent 936dbc0379
commit a873e26cf7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
116 changed files with 315 additions and 289 deletions

View File

@ -148,7 +148,7 @@ public class ScriptScoreBenchmark {
private Query scriptScoreQuery(ScoreScript.Factory factory) {
ScoreScript.LeafFactory leafFactory = factory.newFactory(Map.of(), lookup);
return new ScriptScoreQuery(new MatchAllDocsQuery(), null, leafFactory, lookup, null, "test", 0, IndexVersion.CURRENT);
return new ScriptScoreQuery(new MatchAllDocsQuery(), null, leafFactory, lookup, null, "test", 0, IndexVersion.current());
}
private ScoreScript.Factory bareMetalScript() {

View File

@ -242,7 +242,7 @@ public class DistanceFunctionBenchmark {
@Override
public void execute(Consumer<Object> consumer) {
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.CURRENT).dotProduct(queryVector);
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.current()).dotProduct(queryVector);
}
}
@ -290,7 +290,7 @@ public class DistanceFunctionBenchmark {
@Override
public void execute(Consumer<Object> consumer) {
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.CURRENT).cosineSimilarity(queryVector, false);
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.current()).cosineSimilarity(queryVector, false);
}
}
@ -338,7 +338,7 @@ public class DistanceFunctionBenchmark {
@Override
public void execute(Consumer<Object> consumer) {
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.CURRENT).l1Norm(queryVector);
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.current()).l1Norm(queryVector);
}
}
@ -386,7 +386,7 @@ public class DistanceFunctionBenchmark {
@Override
public void execute(Consumer<Object> consumer) {
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.CURRENT).l1Norm(queryVector);
new BinaryDenseVector(docFloatVector, docVector, dims, IndexVersion.current()).l1Norm(queryVector);
}
}

View File

@ -220,7 +220,7 @@ public class DataStreamGetWriteIndexTests extends ESTestCase {
null,
ScriptCompiler.NONE,
false,
IndexVersion.CURRENT
IndexVersion.current()
).build(MapperBuilderContext.root(false));
RootObjectMapper.Builder root = new RootObjectMapper.Builder("_doc", ObjectMapper.Defaults.SUBOBJECTS);
root.add(
@ -230,7 +230,7 @@ public class DataStreamGetWriteIndexTests extends ESTestCase {
DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER,
ScriptCompiler.NONE,
true,
IndexVersion.CURRENT
IndexVersion.current()
)
);
MetadataFieldMapper dtfm = DataStreamTestHelper.getDataStreamTimestampFieldMapper();

View File

@ -170,7 +170,7 @@ public class DataStreamTimestampFieldMapperTests extends MetadataMapperTestCase
Settings indexSettings = Settings.builder().put(FieldMapper.IGNORE_MALFORMED_SETTING.getKey(), true).build();
Exception e = expectThrows(
IllegalArgumentException.class,
() -> createMapperService(IndexVersion.CURRENT, indexSettings, () -> true, timestampMapping(true, b -> {
() -> createMapperService(IndexVersion.current(), indexSettings, () -> true, timestampMapping(true, b -> {
b.startObject("@timestamp");
b.field("type", "date");
b.endObject();
@ -181,7 +181,7 @@ public class DataStreamTimestampFieldMapperTests extends MetadataMapperTestCase
equalTo("data stream timestamp field [@timestamp] has disallowed [ignore_malformed] attribute specified")
);
MapperService mapperService = createMapperService(IndexVersion.CURRENT, indexSettings, () -> true, timestampMapping(true, b -> {
MapperService mapperService = createMapperService(IndexVersion.current(), indexSettings, () -> true, timestampMapping(true, b -> {
b.startObject("@timestamp");
b.field("type", "date");
b.field("ignore_malformed", false);

View File

@ -293,7 +293,7 @@ public class GeoWKTShapeParserTests extends BaseGeoParsingTestCase {
XContentParser parser = createParser(xContentBuilder);
parser.nextToken();
final LegacyGeoShapeFieldMapper mapperBuilder = new LegacyGeoShapeFieldMapper.Builder("test", IndexVersion.CURRENT, false, true)
final LegacyGeoShapeFieldMapper mapperBuilder = new LegacyGeoShapeFieldMapper.Builder("test", IndexVersion.current(), false, true)
.build(MapperBuilderContext.root(false));
// test store z disabled
@ -376,7 +376,7 @@ public class GeoWKTShapeParserTests extends BaseGeoParsingTestCase {
final LegacyGeoShapeFieldMapper coercingMapperBuilder = new LegacyGeoShapeFieldMapper.Builder(
"test",
IndexVersion.CURRENT,
IndexVersion.current(),
false,
true
).coerce(true).build(MapperBuilderContext.root(false));

View File

@ -98,7 +98,7 @@ public class MatchOnlyTextFieldMapper extends FieldMapper {
private final TextParams.Analyzers analyzers;
public Builder(String name, IndexAnalyzers indexAnalyzers) {
this(name, IndexVersion.CURRENT, indexAnalyzers);
this(name, IndexVersion.current(), indexAnalyzers);
}
public Builder(String name, IndexVersion indexCreatedVersion, IndexAnalyzers indexAnalyzers) {

View File

@ -368,7 +368,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
ScoreMode.None
);
LateParsingQuery query = (LateParsingQuery) hasChildQueryBuilder.toQuery(searchExecutionContext);
Similarity expected = SimilarityService.BUILT_IN.get(similarity).apply(Settings.EMPTY, IndexVersion.CURRENT, null);
Similarity expected = SimilarityService.BUILT_IN.get(similarity).apply(Settings.EMPTY, IndexVersion.current(), null);
assertThat(((PerFieldSimilarityWrapper) query.getSimilarity()).get("custom_string"), instanceOf(expected.getClass()));
}

View File

@ -827,7 +827,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
Collections.singletonList(new BytesArray("{}")),
percolateSearcher,
false,
IndexVersion.CURRENT
IndexVersion.current()
);
TopDocs topDocs = shardSearcher.search(query, 10, new Sort(SortField.FIELD_DOC));
assertEquals(3L, topDocs.totalHits.value);
@ -866,7 +866,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
Collections.singletonList(new BytesArray("{}")),
percolateSearcher,
false,
IndexVersion.CURRENT
IndexVersion.current()
);
TopDocs topDocs = shardSearcher.search(query, 10, new Sort(SortField.FIELD_DOC));
assertEquals(2L, topDocs.totalHits.value);
@ -895,7 +895,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
IndexSearcher shardSearcher = newSearcher(directoryReader);
shardSearcher.setQueryCache(null);
IndexVersion v = IndexVersion.CURRENT;
IndexVersion v = IndexVersion.current();
try (Directory directory = new ByteBuffersDirectory()) {
try (IndexWriter iw = new IndexWriter(directory, newIndexWriterConfig())) {
@ -1018,7 +1018,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
IndexSearcher shardSearcher = newSearcher(directoryReader);
shardSearcher.setQueryCache(null);
IndexVersion v = IndexVersion.CURRENT;
IndexVersion v = IndexVersion.current();
List<BytesReference> sources = Collections.singletonList(new BytesArray("{}"));
MemoryIndex memoryIndex = new MemoryIndex();
@ -1052,7 +1052,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
IndexSearcher shardSearcher = newSearcher(directoryReader);
shardSearcher.setQueryCache(null);
IndexVersion v = IndexVersion.CURRENT;
IndexVersion v = IndexVersion.current();
List<BytesReference> sources = Collections.singletonList(new BytesArray("{}"));
MemoryIndex memoryIndex = new MemoryIndex();
@ -1101,7 +1101,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
IndexSearcher shardSearcher = newSearcher(directoryReader);
shardSearcher.setQueryCache(null);
IndexVersion v = IndexVersion.CURRENT;
IndexVersion v = IndexVersion.current();
List<BytesReference> sources = Collections.singletonList(new BytesArray("{}"));
Document document = new Document();
@ -1125,7 +1125,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
Collections.singletonList(new BytesArray("{}")),
percolateSearcher,
false,
IndexVersion.CURRENT
IndexVersion.current()
);
Query query = requireScore ? percolateQuery : new ConstantScoreQuery(percolateQuery);
TopDocs topDocs = shardSearcher.search(query, 100);
@ -1211,7 +1211,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase {
Collections.singletonList(new BytesArray("{}")),
percolateSearcher,
false,
IndexVersion.CURRENT
IndexVersion.current()
);
return shardSearcher.search(percolateQuery, 10);
}

View File

@ -75,7 +75,7 @@ public class PercolatorMatchedSlotSubFetchPhaseTests extends ESTestCase {
when(sc.query()).thenReturn(percolateQuery);
SearchExecutionContext sec = mock(SearchExecutionContext.class);
when(sc.getSearchExecutionContext()).thenReturn(sec);
when(sec.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(sec.indexVersionCreated()).thenReturn(IndexVersion.current());
FetchSubPhaseProcessor processor = phase.getProcessor(sc);
assertNotNull(processor);
@ -106,7 +106,7 @@ public class PercolatorMatchedSlotSubFetchPhaseTests extends ESTestCase {
when(sc.query()).thenReturn(percolateQuery);
SearchExecutionContext sec = mock(SearchExecutionContext.class);
when(sc.getSearchExecutionContext()).thenReturn(sec);
when(sec.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(sec.indexVersionCreated()).thenReturn(IndexVersion.current());
FetchSubPhaseProcessor processor = phase.getProcessor(sc);
assertNotNull(processor);
@ -136,7 +136,7 @@ public class PercolatorMatchedSlotSubFetchPhaseTests extends ESTestCase {
when(sc.query()).thenReturn(percolateQuery);
SearchExecutionContext sec = mock(SearchExecutionContext.class);
when(sc.getSearchExecutionContext()).thenReturn(sec);
when(sec.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(sec.indexVersionCreated()).thenReturn(IndexVersion.current());
FetchSubPhaseProcessor processor = phase.getProcessor(sc);
assertNotNull(processor);

View File

@ -70,7 +70,7 @@ public class QueryBuilderStoreTests extends ESTestCase {
queryBuilders[i] = new TermQueryBuilder(randomAlphaOfLength(4), randomAlphaOfLength(8));
DocumentParserContext documentParserContext = new TestDocumentParserContext();
PercolatorFieldMapper.createQueryBuilderField(
IndexVersion.CURRENT,
IndexVersion.current(),
TransportVersion.current(),
fieldMapper,
queryBuilders[i],
@ -81,7 +81,7 @@ public class QueryBuilderStoreTests extends ESTestCase {
}
SearchExecutionContext searchExecutionContext = mock(SearchExecutionContext.class);
when(searchExecutionContext.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(searchExecutionContext.indexVersionCreated()).thenReturn(IndexVersion.current());
when(searchExecutionContext.getWriteableRegistry()).thenReturn(writableRegistry());
when(searchExecutionContext.getParserConfig()).thenReturn(parserConfig());
when(searchExecutionContext.getForField(fieldMapper.fieldType(), fielddataOperation)).thenReturn(

View File

@ -29,7 +29,7 @@ public class AnnotatedTextFieldTypeTests extends FieldTypeTestCase {
}
public void testFetchSourceValue() throws IOException {
MappedFieldType fieldType = new AnnotatedTextFieldMapper.Builder("field", IndexVersion.CURRENT, createDefaultIndexAnalyzers())
MappedFieldType fieldType = new AnnotatedTextFieldMapper.Builder("field", IndexVersion.current(), createDefaultIndexAnalyzers())
.build(MapperBuilderContext.root(false))
.fieldType();

View File

@ -959,7 +959,7 @@ public class FullClusterRestartIT extends ParameterizedFullClusterRestartTestCas
assertTrue("expected to find a primary but didn't\n" + recoveryResponse, foundPrimary);
assertEquals("mismatch while checking for translog recovery\n" + recoveryResponse, shouldHaveTranslog, restoredFromTranslog);
String currentLuceneVersion = IndexVersion.CURRENT.luceneVersion().toString();
String currentLuceneVersion = IndexVersion.current().luceneVersion().toString();
String bwcLuceneVersion = getOldClusterVersion().luceneVersion().toString();
String minCompatibleBWCVersion = Version.CURRENT.minimumCompatibilityVersion().luceneVersion().toString();
if (shouldHaveTranslog && false == currentLuceneVersion.equals(bwcLuceneVersion)) {

View File

@ -110,7 +110,7 @@ public class SplitIndexIT extends ESIntegTestCase {
useRoutingPartition = randomBoolean();
}
if (useRouting && useMixedRouting == false && useRoutingPartition) {
int numRoutingShards = MetadataCreateIndexService.calculateNumRoutingShards(secondSplitShards, IndexVersion.CURRENT) - 1;
int numRoutingShards = MetadataCreateIndexService.calculateNumRoutingShards(secondSplitShards, IndexVersion.current()) - 1;
settings.put("index.routing_partition_size", randomIntBetween(1, numRoutingShards));
if (useNested) {
createInitialIndex.setMapping("_routing", "required=true", "nested1", "type=nested");

View File

@ -1683,7 +1683,7 @@ public class IndexRecoveryIT extends AbstractIndexRecoveryIntegTestCase {
final Query query = new BooleanQuery.Builder().add(
LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, commitLocalCheckpoint + 1, Long.MAX_VALUE),
BooleanClause.Occur.MUST
).add(Queries.newNonNestedFilter(IndexVersion.CURRENT), BooleanClause.Occur.MUST).build();
).add(Queries.newNonNestedFilter(IndexVersion.current()), BooleanClause.Occur.MUST).build();
final Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1.0f);
for (LeafReaderContext leaf : directoryReader.leaves()) {
final Scorer scorer = weight.scorer(leaf);

View File

@ -193,7 +193,7 @@ public class ShardSnapshotsServiceIT extends ESIntegTestCase {
assertThat(commitVersion, is(equalTo(Version.CURRENT)));
final org.apache.lucene.util.Version commitLuceneVersion = shardSnapshotData.getCommitLuceneVersion();
assertThat(commitLuceneVersion, is(notNullValue()));
assertThat(commitLuceneVersion, is(equalTo(IndexVersion.CURRENT.luceneVersion())));
assertThat(commitLuceneVersion, is(equalTo(IndexVersion.current().luceneVersion())));
assertThat(shardSnapshotInfo.getShardId(), is(equalTo(shardId)));
assertThat(shardSnapshotInfo.getSnapshot().getSnapshotId().getName(), is(equalTo(snapshotName)));

View File

@ -340,10 +340,10 @@ class Elasticsearch {
}
static void checkLucene() {
if (IndexVersion.CURRENT.luceneVersion().equals(org.apache.lucene.util.Version.LATEST) == false) {
if (IndexVersion.current().luceneVersion().equals(org.apache.lucene.util.Version.LATEST) == false) {
throw new AssertionError(
"Lucene version mismatch this version of Elasticsearch requires lucene version ["
+ IndexVersion.CURRENT.luceneVersion()
+ IndexVersion.current().luceneVersion()
+ "] but the current lucene version is ["
+ org.apache.lucene.util.Version.LATEST
+ "]"

View File

@ -2272,7 +2272,7 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, Ch
final List<String> visibleClosedIndices = new ArrayList<>();
final ImmutableOpenMap<String, IndexMetadata> indicesMap = indices.build();
int oldestIndexVersionId = IndexVersion.CURRENT.id();
int oldestIndexVersionId = IndexVersion.current().id();
int totalNumberOfShards = 0;
int totalOpenIndexShards = 0;

View File

@ -1042,7 +1042,7 @@ public class MetadataCreateIndexService {
if (indexSettingsBuilder.get(IndexMetadata.SETTING_VERSION_CREATED) == null) {
DiscoveryNodes nodes = currentState.nodes();
IndexVersion createdVersion = IndexVersion.min(IndexVersion.CURRENT, nodes.getMaxDataNodeCompatibleIndexVersion());
IndexVersion createdVersion = IndexVersion.min(IndexVersion.current(), nodes.getMaxDataNodeCompatibleIndexVersion());
indexSettingsBuilder.put(IndexMetadata.SETTING_VERSION_CREATED, createdVersion.toVersion());
}
if (INDEX_NUMBER_OF_SHARDS_SETTING.exists(indexSettingsBuilder) == false) {

View File

@ -348,7 +348,7 @@ public class DiscoveryNode implements Writeable, ToXContentFragment {
IndexVersion.fromId(version.id)
);
} else {
return new VersionInformation(version, IndexVersion.MINIMUM_COMPATIBLE, IndexVersion.CURRENT);
return new VersionInformation(version, IndexVersion.MINIMUM_COMPATIBLE, IndexVersion.current());
}
}

View File

@ -874,7 +874,7 @@ public class DiscoveryNodes implements Iterable<DiscoveryNode>, SimpleDiffable<D
Objects.requireNonNullElse(minNonClientNodeVersion, Version.CURRENT),
Objects.requireNonNullElse(maxNodeVersion, Version.CURRENT),
Objects.requireNonNullElse(minNodeVersion, Version.CURRENT.minimumCompatibilityVersion()),
Objects.requireNonNullElse(maxDataNodeCompatibleIndexVersion, IndexVersion.CURRENT),
Objects.requireNonNullElse(maxDataNodeCompatibleIndexVersion, IndexVersion.current()),
Objects.requireNonNullElse(minSupportedIndexVersion, IndexVersion.MINIMUM_COMPATIBLE),
dataNodes.values()
.stream()

View File

@ -24,7 +24,7 @@ public record VersionInformation(Version nodeVersion, IndexVersion minIndexVersi
public static final VersionInformation CURRENT = new VersionInformation(
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT
IndexVersion.current()
);
public static VersionInformation inferVersions(Version nodeVersion) {

View File

@ -100,7 +100,7 @@ public interface DateFormatter {
DateMathParser toDateMathParser();
static DateFormatter forPattern(String input) {
return forPattern(input, IndexVersion.CURRENT);
return forPattern(input, IndexVersion.current());
}
static DateFormatter forPattern(String input, IndexVersion supportedVersion) {

View File

@ -90,7 +90,7 @@ public class HandshakingTransportAddressConnector implements TransportAddressCon
new VersionInformation(
Version.CURRENT.minimumCompatibilityVersion(),
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT
IndexVersion.current()
)
),
handshakeConnectionProfile,

View File

@ -624,7 +624,7 @@ public final class NodeEnvironment implements Closeable {
assert nodeIds.isEmpty() : nodeIds;
// If we couldn't find legacy metadata, we set the latest index version to this version. This happens
// when we are starting a new node and there are no indices to worry about.
metadata = new NodeMetadata(generateNodeId(settings), Version.CURRENT, IndexVersion.CURRENT);
metadata = new NodeMetadata(generateNodeId(settings), Version.CURRENT, IndexVersion.current());
} else {
assert nodeIds.equals(Collections.singleton(legacyMetadata.nodeId())) : nodeIds + " doesn't match " + legacyMetadata;
metadata = legacyMetadata;

View File

@ -13,6 +13,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.core.Assertions;
import org.elasticsearch.internal.VersionExtension;
import org.elasticsearch.xcontent.ToXContentFragment;
import org.elasticsearch.xcontent.XContentBuilder;
@ -53,7 +54,7 @@ import java.util.TreeMap;
* Each index version should only be used in a single merged commit (apart from BwC versions copied from {@link Version}).
* <p>
* To add a new index version, add a new constant at the bottom of the list that is one greater than the current highest version,
* ensure it has a unique id, and update the {@link #CURRENT} constant to point to the new version.
* ensure it has a unique id, and update the {@link #current()} constant to point to the new version.
* <h2>Reverting an index version</h2>
* If you revert a commit with an index version change, you <em>must</em> ensure there is a <em>new</em> index version
* representing the reverted change. <em>Do not</em> let the index version go backwards, it must <em>always</em> be incremented.
@ -178,16 +179,28 @@ public record IndexVersion(int id, Version luceneVersion) implements Comparable<
* Detached index versions added below here.
*/
/**
* Reference to the most recent index version.
* This should be the index version with the highest id.
*/
public static final IndexVersion CURRENT = V_8_10_0;
private static class CurrentHolder {
private static final IndexVersion CURRENT = findCurrent(V_8_10_0);
// finds the pluggable current version, or uses the given fallback
private static IndexVersion findCurrent(IndexVersion fallback) {
var versionExtension = VersionExtension.load();
if (versionExtension == null) {
return fallback;
}
var version = versionExtension.getCurrentIndexVersion();
assert version.onOrAfter(fallback);
assert version.luceneVersion.equals(Version.LATEST)
: "IndexVersion must be upgraded to ["
+ Version.LATEST
+ "] is still set to ["
+ version.luceneVersion
+ "]";
return version;
}
}
/**
* Reference to the earliest compatible index version to this version of the codebase.
* This should be the index version used by the first release of the previous major version.
*/
public static final IndexVersion MINIMUM_COMPATIBLE = V_7_0_0;
static {
@ -239,13 +252,6 @@ public record IndexVersion(int id, Version luceneVersion) implements Comparable<
static {
VERSION_IDS = getAllVersionIds(IndexVersion.class);
assert CURRENT.luceneVersion.equals(org.apache.lucene.util.Version.LATEST)
: "IndexVersion must be upgraded to ["
+ org.apache.lucene.util.Version.LATEST
+ "] is still set to ["
+ CURRENT.luceneVersion
+ "]";
}
static Collection<IndexVersion> getAllVersions() {
@ -298,6 +304,14 @@ public record IndexVersion(int id, Version luceneVersion) implements Comparable<
return version1.id > version2.id ? version1 : version2;
}
/**
* Returns the most recent index version.
* This should be the index version with the highest id.
*/
public static IndexVersion current() {
return CurrentHolder.CURRENT;
}
public boolean after(IndexVersion version) {
return version.id < id;
}

View File

@ -51,7 +51,7 @@ public class PreBuiltAnalyzerProviderFactory extends PreConfiguredAnalysisCompon
public AnalyzerProvider<?> get(IndexSettings indexSettings, Environment environment, String name, Settings settings)
throws IOException {
IndexVersion versionCreated = indexSettings.getIndexVersionCreated();
if (IndexVersion.CURRENT.equals(versionCreated) == false) {
if (IndexVersion.current().equals(versionCreated) == false) {
return super.get(indexSettings, environment, name, settings);
} else {
return current;

View File

@ -278,7 +278,7 @@ abstract class AbstractScriptFieldType<LeafFactory> extends MappedFieldType {
}
final RuntimeField createRuntimeField(Factory scriptFactory) {
return createRuntimeField(scriptFactory, IndexVersion.CURRENT);
return createRuntimeField(scriptFactory, IndexVersion.current());
}
final RuntimeField createRuntimeField(Factory scriptFactory, IndexVersion indexVersion) {

View File

@ -109,7 +109,7 @@ public class DateScriptFieldType extends AbstractScriptFieldType<DateFieldScript
Map<String, String> meta,
OnScriptError onScriptError
) {
return createFieldType(name, factory, script, meta, IndexVersion.CURRENT, onScriptError);
return createFieldType(name, factory, script, meta, IndexVersion.current(), onScriptError);
}
@Override

View File

@ -1047,7 +1047,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
Function<FieldMapper, NamedAnalyzer> initializer,
Supplier<NamedAnalyzer> defaultAnalyzer
) {
return analyzerParam(name, updateable, initializer, defaultAnalyzer, IndexVersion.CURRENT);
return analyzerParam(name, updateable, initializer, defaultAnalyzer, IndexVersion.current());
}
/**

View File

@ -277,7 +277,7 @@ public class TextFieldMapper extends FieldMapper {
final TextParams.Analyzers analyzers;
public Builder(String name, IndexAnalyzers indexAnalyzers) {
this(name, IndexVersion.CURRENT, indexAnalyzers);
this(name, IndexVersion.current(), indexAnalyzers);
}
public Builder(String name, IndexVersion indexCreatedVersion, IndexAnalyzers indexAnalyzers) {

View File

@ -9,6 +9,7 @@
package org.elasticsearch.internal;
import org.elasticsearch.TransportVersion;
import org.elasticsearch.index.IndexVersion;
import java.util.ServiceLoader;
@ -23,6 +24,13 @@ public interface VersionExtension {
*/
TransportVersion getCurrentTransportVersion();
/**
* Returns the {@link IndexVersion} that Elasticsearch should use.
* <p>
* This must be at least equal to the latest version found in {@link IndexVersion} V_* constants.
*/
IndexVersion getCurrentIndexVersion();
/**
* Loads a single VersionExtension, or returns {@code null} if none are found.
*/

View File

@ -302,7 +302,7 @@ public class ProxyConnectionStrategy extends RemoteConnectionStrategy {
new VersionInformation(
Version.CURRENT.minimumCompatibilityVersion(),
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT
IndexVersion.current()
)
);

View File

@ -495,7 +495,7 @@ public class SniffConnectionStrategy extends RemoteConnectionStrategy {
var seedVersion = new VersionInformation(
Version.CURRENT.minimumCompatibilityVersion(),
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT
IndexVersion.current()
);
if (proxyAddress == null || proxyAddress.isEmpty()) {
TransportAddress transportAddress = new TransportAddress(parseConfiguredAddress(address));

View File

@ -198,7 +198,7 @@ public class ClusterRerouteResponseTests extends ESTestCase {
clusterState.getNodes().get("node0").getEphemeralId(),
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT,
IndexVersion.current(),
Version.CURRENT.id
),
"""

View File

@ -363,10 +363,10 @@ public class ClusterStateTests extends ESTestCase {
ephemeralId,
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT,
IndexVersion.current(),
TransportVersion.current(),
IndexVersion.CURRENT,
IndexVersion.CURRENT,
IndexVersion.current(),
IndexVersion.current(),
allocationId,
allocationId
)
@ -614,10 +614,10 @@ public class ClusterStateTests extends ESTestCase {
ephemeralId,
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT,
IndexVersion.current(),
TransportVersion.current(),
IndexVersion.CURRENT,
IndexVersion.CURRENT,
IndexVersion.current(),
IndexVersion.current(),
allocationId,
allocationId
),
@ -871,10 +871,10 @@ public class ClusterStateTests extends ESTestCase {
ephemeralId,
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT,
IndexVersion.current(),
TransportVersion.current(),
IndexVersion.CURRENT,
IndexVersion.CURRENT,
IndexVersion.current(),
IndexVersion.current(),
allocationId,
allocationId
),

View File

@ -581,17 +581,17 @@ public class MetadataCreateIndexServiceTests extends ESTestCase {
}
public void testCalculateNumRoutingShards() {
assertEquals(1024, MetadataCreateIndexService.calculateNumRoutingShards(1, IndexVersion.CURRENT));
assertEquals(1024, MetadataCreateIndexService.calculateNumRoutingShards(2, IndexVersion.CURRENT));
assertEquals(768, MetadataCreateIndexService.calculateNumRoutingShards(3, IndexVersion.CURRENT));
assertEquals(576, MetadataCreateIndexService.calculateNumRoutingShards(9, IndexVersion.CURRENT));
assertEquals(1024, MetadataCreateIndexService.calculateNumRoutingShards(512, IndexVersion.CURRENT));
assertEquals(2048, MetadataCreateIndexService.calculateNumRoutingShards(1024, IndexVersion.CURRENT));
assertEquals(4096, MetadataCreateIndexService.calculateNumRoutingShards(2048, IndexVersion.CURRENT));
assertEquals(1024, MetadataCreateIndexService.calculateNumRoutingShards(1, IndexVersion.current()));
assertEquals(1024, MetadataCreateIndexService.calculateNumRoutingShards(2, IndexVersion.current()));
assertEquals(768, MetadataCreateIndexService.calculateNumRoutingShards(3, IndexVersion.current()));
assertEquals(576, MetadataCreateIndexService.calculateNumRoutingShards(9, IndexVersion.current()));
assertEquals(1024, MetadataCreateIndexService.calculateNumRoutingShards(512, IndexVersion.current()));
assertEquals(2048, MetadataCreateIndexService.calculateNumRoutingShards(1024, IndexVersion.current()));
assertEquals(4096, MetadataCreateIndexService.calculateNumRoutingShards(2048, IndexVersion.current()));
for (int i = 0; i < 1000; i++) {
int randomNumShards = randomIntBetween(1, 10000);
int numRoutingShards = MetadataCreateIndexService.calculateNumRoutingShards(randomNumShards, IndexVersion.CURRENT);
int numRoutingShards = MetadataCreateIndexService.calculateNumRoutingShards(randomNumShards, IndexVersion.current());
if (numRoutingShards <= 1024) {
assertTrue("numShards: " + randomNumShards, randomNumShards < 513);
assertTrue("numRoutingShards: " + numRoutingShards, numRoutingShards > 512);

View File

@ -767,18 +767,18 @@ public class MetadataTests extends ESTestCase {
public void testOldestIndexComputation() {
Metadata metadata = buildIndicesWithVersions(
IndexVersion.V_7_0_0,
IndexVersion.CURRENT,
IndexVersion.fromId(IndexVersion.CURRENT.id() + 1)
IndexVersion.current(),
IndexVersion.fromId(IndexVersion.current().id() + 1)
).build();
assertEquals(IndexVersion.V_7_0_0, metadata.oldestIndexVersion());
Metadata.Builder b = Metadata.builder();
assertEquals(IndexVersion.CURRENT, b.build().oldestIndexVersion());
assertEquals(IndexVersion.current(), b.build().oldestIndexVersion());
Throwable ex = expectThrows(
IllegalArgumentException.class,
() -> buildIndicesWithVersions(IndexVersion.V_7_0_0, IndexVersion.ZERO, IndexVersion.fromId(IndexVersion.CURRENT.id() + 1))
() -> buildIndicesWithVersions(IndexVersion.V_7_0_0, IndexVersion.ZERO, IndexVersion.fromId(IndexVersion.current().id() + 1))
.build()
);
@ -803,7 +803,7 @@ public class MetadataTests extends ESTestCase {
private static IndexMetadata.Builder buildIndexMetadata(String name, String alias, Boolean writeIndex) {
return IndexMetadata.builder(name)
.settings(settings(IndexVersion.CURRENT))
.settings(settings(IndexVersion.current()))
.creationDate(randomNonNegativeLong())
.putAlias(AliasMetadata.builder(alias).writeIndex(writeIndex))
.numberOfShards(1)

View File

@ -222,7 +222,7 @@ public class DiscoveryNodeTests extends ESTestCase {
withExternalId ? "test-external-id" : "test-name",
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT
IndexVersion.current()
)
)
);

View File

@ -22,12 +22,12 @@ public class QueriesTests extends ESTestCase {
public void testNonNestedQuery() {
// This is a custom query that extends AutomatonQuery and want to make sure the equals method works
assertEquals(Queries.newNonNestedFilter(IndexVersion.CURRENT), Queries.newNonNestedFilter(IndexVersion.CURRENT));
assertEquals(Queries.newNonNestedFilter(IndexVersion.current()), Queries.newNonNestedFilter(IndexVersion.current()));
assertEquals(
Queries.newNonNestedFilter(IndexVersion.CURRENT).hashCode(),
Queries.newNonNestedFilter(IndexVersion.CURRENT).hashCode()
Queries.newNonNestedFilter(IndexVersion.current()).hashCode(),
Queries.newNonNestedFilter(IndexVersion.current()).hashCode()
);
assertEquals(Queries.newNonNestedFilter(IndexVersion.CURRENT), new FieldExistsQuery(SeqNoFieldMapper.PRIMARY_TERM_NAME));
assertEquals(Queries.newNonNestedFilter(IndexVersion.current()), new FieldExistsQuery(SeqNoFieldMapper.PRIMARY_TERM_NAME));
}
public void testIsNegativeQuery() {

View File

@ -96,7 +96,7 @@ public class NodeMetadataTests extends ESTestCase {
v -> v.after(Version.CURRENT) || v.before(Version.CURRENT.minimumCompatibilityVersion()),
this::randomVersion
),
IndexVersion.CURRENT
IndexVersion.current()
).upgradeToCurrentVersion();
assertThat(nodeMetadata.nodeVersion(), equalTo(Version.CURRENT));
assertThat(nodeMetadata.nodeId(), equalTo(nodeId));
@ -107,7 +107,7 @@ public class NodeMetadataTests extends ESTestCase {
final IllegalStateException illegalStateException = expectThrows(
IllegalStateException.class,
() -> new NodeMetadata(nodeId, Version.V_EMPTY, IndexVersion.CURRENT).upgradeToCurrentVersion()
() -> new NodeMetadata(nodeId, Version.V_EMPTY, IndexVersion.current()).upgradeToCurrentVersion()
);
assertThat(
illegalStateException.getMessage(),
@ -118,7 +118,7 @@ public class NodeMetadataTests extends ESTestCase {
public void testDoesNotUpgradeFutureVersion() {
final IllegalStateException illegalStateException = expectThrows(
IllegalStateException.class,
() -> new NodeMetadata(randomAlphaOfLength(10), tooNewVersion(), IndexVersion.CURRENT).upgradeToCurrentVersion()
() -> new NodeMetadata(randomAlphaOfLength(10), tooNewVersion(), IndexVersion.current()).upgradeToCurrentVersion()
);
assertThat(
illegalStateException.getMessage(),
@ -129,7 +129,7 @@ public class NodeMetadataTests extends ESTestCase {
public void testDoesNotUpgradeAncientVersion() {
final IllegalStateException illegalStateException = expectThrows(
IllegalStateException.class,
() -> new NodeMetadata(randomAlphaOfLength(10), tooOldVersion(), IndexVersion.CURRENT).upgradeToCurrentVersion()
() -> new NodeMetadata(randomAlphaOfLength(10), tooOldVersion(), IndexVersion.current()).upgradeToCurrentVersion()
);
assertThat(
illegalStateException.getMessage(),
@ -150,7 +150,7 @@ public class NodeMetadataTests extends ESTestCase {
final String nodeId = randomAlphaOfLength(10);
final Version version = VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.V_8_0_0);
final NodeMetadata nodeMetadata = new NodeMetadata(nodeId, version, IndexVersion.CURRENT).upgradeToCurrentVersion();
final NodeMetadata nodeMetadata = new NodeMetadata(nodeId, version, IndexVersion.current()).upgradeToCurrentVersion();
assertThat(nodeMetadata.nodeVersion(), equalTo(Version.CURRENT));
assertThat(nodeMetadata.previousNodeVersion(), equalTo(version));
}
@ -160,7 +160,7 @@ public class NodeMetadataTests extends ESTestCase {
}
public static IndexVersion tooNewIndexVersion() {
return IndexVersion.fromId(between(IndexVersion.CURRENT.id() + 1, 99999999));
return IndexVersion.fromId(between(IndexVersion.current().id() + 1, 99999999));
}
public static Version tooOldVersion() {

View File

@ -1493,8 +1493,8 @@ public class PersistedClusterStateServiceTests extends ESTestCase {
final IndexVersion[] indexVersions = new IndexVersion[] {
oldVersion,
IndexVersion.CURRENT,
IndexVersion.fromId(IndexVersion.CURRENT.id() + 1) };
IndexVersion.current(),
IndexVersion.fromId(IndexVersion.current().id() + 1) };
int lastIndexNum = randomIntBetween(9, 50);
Metadata.Builder b = Metadata.builder();
for (IndexVersion indexVersion : indexVersions) {

View File

@ -167,7 +167,7 @@ public class IndexSettingsTests extends ESTestCase {
"index",
Settings.builder()
.put(IndexMetadata.SETTING_VERSION_CREATED, version.id())
.put(IndexMetadata.SETTING_VERSION_COMPATIBILITY, IndexVersion.CURRENT.id())
.put(IndexMetadata.SETTING_VERSION_COMPATIBILITY, IndexVersion.current().id())
.put("index.test.setting.int", 42)
.build()
)

View File

@ -130,9 +130,9 @@ public class IndexVersionTests extends ESTestCase {
public void testMin() {
assertEquals(
IndexVersionUtils.getPreviousVersion(),
IndexVersion.min(IndexVersion.CURRENT, IndexVersionUtils.getPreviousVersion())
IndexVersion.min(IndexVersion.current(), IndexVersionUtils.getPreviousVersion())
);
assertEquals(IndexVersion.fromId(1_01_01_99), IndexVersion.min(IndexVersion.fromId(1_01_01_99), IndexVersion.CURRENT));
assertEquals(IndexVersion.fromId(1_01_01_99), IndexVersion.min(IndexVersion.fromId(1_01_01_99), IndexVersion.current()));
IndexVersion version = IndexVersionUtils.randomVersion();
IndexVersion version1 = IndexVersionUtils.randomVersion();
if (version.id() <= version1.id()) {
@ -143,8 +143,8 @@ public class IndexVersionTests extends ESTestCase {
}
public void testMax() {
assertEquals(IndexVersion.CURRENT, IndexVersion.max(IndexVersion.CURRENT, IndexVersionUtils.getPreviousVersion()));
assertEquals(IndexVersion.CURRENT, IndexVersion.max(IndexVersion.fromId(1_01_01_99), IndexVersion.CURRENT));
assertEquals(IndexVersion.current(), IndexVersion.max(IndexVersion.current(), IndexVersionUtils.getPreviousVersion()));
assertEquals(IndexVersion.current(), IndexVersion.max(IndexVersion.fromId(1_01_01_99), IndexVersion.current()));
IndexVersion version = IndexVersionUtils.randomVersion();
IndexVersion version1 = IndexVersionUtils.randomVersion();
if (version.id() >= version1.id()) {
@ -155,8 +155,8 @@ public class IndexVersionTests extends ESTestCase {
}
public void testVersionConstantPresent() {
Set<IndexVersion> ignore = Set.of(IndexVersion.ZERO, IndexVersion.CURRENT, IndexVersion.MINIMUM_COMPATIBLE);
assertThat(IndexVersion.CURRENT, sameInstance(IndexVersion.fromId(IndexVersion.CURRENT.id())));
Set<IndexVersion> ignore = Set.of(IndexVersion.ZERO, IndexVersion.current(), IndexVersion.MINIMUM_COMPATIBLE);
assertThat(IndexVersion.current(), sameInstance(IndexVersion.fromId(IndexVersion.current().id())));
final int iters = scaledRandomIntBetween(20, 100);
for (int i = 0; i < iters; i++) {
IndexVersion version = IndexVersionUtils.randomVersion(ignore);
@ -166,7 +166,7 @@ public class IndexVersionTests extends ESTestCase {
}
public void testCURRENTIsLatest() {
assertThat(Collections.max(IndexVersion.getAllVersions()), is(IndexVersion.CURRENT));
assertThat(Collections.max(IndexVersion.getAllVersions()), is(IndexVersion.current()));
}
public void testToString() {

View File

@ -6734,7 +6734,7 @@ public class InternalEngineTests extends EngineTestCase {
Map<String, String> userData = new HashMap<>(store.readLastCommittedSegmentsInfo().userData);
userData.remove(Engine.MIN_RETAINED_SEQNO);
IndexWriterConfig indexWriterConfig = new IndexWriterConfig(null).setOpenMode(IndexWriterConfig.OpenMode.APPEND)
.setIndexCreatedVersionMajor(IndexVersion.CURRENT.luceneVersion().major)
.setIndexCreatedVersionMajor(IndexVersion.current().luceneVersion().major)
.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD)
.setCommitOnClose(false)
.setMergePolicy(NoMergePolicy.INSTANCE);

View File

@ -90,7 +90,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
false,
randomBoolean(),
randomBoolean(),
IndexVersion.CURRENT
IndexVersion.current()
)
) {
searcher = null;
@ -109,7 +109,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
true,
randomBoolean(),
randomBoolean(),
IndexVersion.CURRENT
IndexVersion.current()
)
) {
searcher = null;
@ -134,7 +134,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
false,
randomBoolean(),
randomBoolean(),
IndexVersion.CURRENT
IndexVersion.current()
)
) {
searcher = null;
@ -152,7 +152,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
true,
randomBoolean(),
randomBoolean(),
IndexVersion.CURRENT
IndexVersion.current()
)
) {
searcher = null;
@ -175,7 +175,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
true,
randomBoolean(),
randomBoolean(),
IndexVersion.CURRENT
IndexVersion.current()
)
) {
searcher = null;
@ -237,7 +237,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
false,
randomBoolean(),
accessStats,
IndexVersion.CURRENT
IndexVersion.current()
)
) {
if (accessStats) {

View File

@ -88,7 +88,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
final MapperBuilderContext context = MapperBuilderContext.root(false);
if (type.equals("string")) {
if (docValues) {
fieldType = new KeywordFieldMapper.Builder(fieldName, IndexVersion.CURRENT).build(context).fieldType();
fieldType = new KeywordFieldMapper.Builder(fieldName, IndexVersion.current()).build(context).fieldType();
} else {
fieldType = new TextFieldMapper.Builder(fieldName, createDefaultIndexAnalyzers()).fielddata(true)
.build(context)
@ -101,7 +101,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).docValues(docValues).build(context).fieldType();
} else if (type.equals("double")) {
@ -111,7 +111,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).docValues(docValues).build(context).fieldType();
} else if (type.equals("long")) {
@ -121,7 +121,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).docValues(docValues).build(context).fieldType();
} else if (type.equals("int")) {
@ -131,7 +131,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).docValues(docValues).build(context).fieldType();
} else if (type.equals("short")) {
@ -141,7 +141,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).docValues(docValues).build(context).fieldType();
} else if (type.equals("byte")) {
@ -151,11 +151,11 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).docValues(docValues).build(context).fieldType();
} else if (type.equals("geo_point")) {
fieldType = new GeoPointFieldMapper.Builder(fieldName, ScriptCompiler.NONE, false, IndexVersion.CURRENT, null).docValues(
fieldType = new GeoPointFieldMapper.Builder(fieldName, ScriptCompiler.NONE, false, IndexVersion.current(), null).docValues(
docValues
).build(context).fieldType();
} else if (type.equals("binary")) {

View File

@ -78,19 +78,19 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
indicesService.getCircuitBreakerService()
);
MapperBuilderContext context = MapperBuilderContext.root(false);
final MappedFieldType stringMapper = new KeywordFieldMapper.Builder("string", IndexVersion.CURRENT).build(context).fieldType();
final MappedFieldType stringMapper = new KeywordFieldMapper.Builder("string", IndexVersion.current()).build(context).fieldType();
ifdService.clear();
IndexFieldData<?> fd = ifdService.getForField(stringMapper, FieldDataContext.noRuntimeFields("test"));
assertTrue(fd instanceof SortedSetOrdinalsIndexFieldData);
for (MappedFieldType mapper : Arrays.asList(
new NumberFieldMapper.Builder("int", BYTE, ScriptCompiler.NONE, false, true, IndexVersion.CURRENT, null).build(context)
new NumberFieldMapper.Builder("int", BYTE, ScriptCompiler.NONE, false, true, IndexVersion.current(), null).build(context)
.fieldType(),
new NumberFieldMapper.Builder("int", SHORT, ScriptCompiler.NONE, false, true, IndexVersion.CURRENT, null).build(context)
new NumberFieldMapper.Builder("int", SHORT, ScriptCompiler.NONE, false, true, IndexVersion.current(), null).build(context)
.fieldType(),
new NumberFieldMapper.Builder("int", INTEGER, ScriptCompiler.NONE, false, true, IndexVersion.CURRENT, null).build(context)
new NumberFieldMapper.Builder("int", INTEGER, ScriptCompiler.NONE, false, true, IndexVersion.current(), null).build(context)
.fieldType(),
new NumberFieldMapper.Builder("long", LONG, ScriptCompiler.NONE, false, true, IndexVersion.CURRENT, null).build(context)
new NumberFieldMapper.Builder("long", LONG, ScriptCompiler.NONE, false, true, IndexVersion.current(), null).build(context)
.fieldType()
)) {
ifdService.clear();
@ -104,7 +104,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).build(context).fieldType();
ifdService.clear();
@ -117,7 +117,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).build(context).fieldType();
ifdService.clear();

View File

@ -137,7 +137,7 @@ public class BooleanScriptFieldTypeTests extends AbstractNonTextScriptFieldTypeT
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
assertThat(searcher.count(new ScriptScoreQuery(new MatchAllDocsQuery(), new Script("test"), new ScoreScript.LeafFactory() {
@Override
public boolean needs_score() {
@ -154,7 +154,7 @@ public class BooleanScriptFieldTypeTests extends AbstractNonTextScriptFieldTypeT
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}
@ -319,7 +319,7 @@ public class BooleanScriptFieldTypeTests extends AbstractNonTextScriptFieldTypeT
}
public void testDualingQueries() throws IOException {
BooleanFieldMapper ootb = new BooleanFieldMapper.Builder("foo", ScriptCompiler.NONE, false, IndexVersion.CURRENT).build(
BooleanFieldMapper ootb = new BooleanFieldMapper.Builder("foo", ScriptCompiler.NONE, false, IndexVersion.current()).build(
MapperBuilderContext.root(false)
);
try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) {

View File

@ -747,7 +747,7 @@ public class DateFieldMapperTests extends MapperTestCase {
null,
mock(ScriptService.class),
true,
IndexVersion.CURRENT
IndexVersion.current()
);
@SuppressWarnings("unchecked")

View File

@ -230,7 +230,7 @@ public class DateScriptFieldTypeTests extends AbstractNonTextScriptFieldTypeTest
}
};
}
}, searchContext.lookup(), 354.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 354.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}

View File

@ -284,7 +284,7 @@ public class DocumentMapperTests extends MapperServiceTestCase {
}
public void testEmptyDocumentMapper() {
MapperService mapperService = createMapperService(IndexVersion.CURRENT, Settings.EMPTY, () -> false);
MapperService mapperService = createMapperService(IndexVersion.current(), Settings.EMPTY, () -> false);
DocumentMapper documentMapper = DocumentMapper.createEmpty(mapperService);
assertEquals("{\"_doc\":{}}", Strings.toString(documentMapper.mapping()));
assertTrue(documentMapper.mappers().hasMappings());
@ -428,7 +428,7 @@ public class DocumentMapperTests extends MapperServiceTestCase {
builders[i].endObject().endObject().endObject();
}
final MapperService mapperService = createMapperService(IndexVersion.CURRENT, Settings.EMPTY, () -> false);
final MapperService mapperService = createMapperService(IndexVersion.current(), Settings.EMPTY, () -> false);
final CountDownLatch latch = new CountDownLatch(1);
final Thread[] threads = new Thread[numThreads];
for (int i = 0; i < threads.length; i++) {

View File

@ -138,7 +138,7 @@ public class DoubleScriptFieldTypeTests extends AbstractNonTextScriptFieldTypeTe
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}

View File

@ -331,7 +331,7 @@ public class DynamicTemplatesTests extends MapperServiceTestCase {
.endArray()
.endObject()
);
MapperService mapperService = createMapperService(IndexVersion.CURRENT, Settings.EMPTY, () -> true);
MapperService mapperService = createMapperService(IndexVersion.current(), Settings.EMPTY, () -> true);
mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), MapperService.MergeReason.INDEX_TEMPLATE);
// There should be no update if templates are not set.

View File

@ -158,7 +158,7 @@ public class FieldAliasMapperValidationTests extends ESTestCase {
}
private static FieldMapper createFieldMapper(String parent, String name) {
return new BooleanFieldMapper.Builder(name, ScriptCompiler.NONE, false, IndexVersion.CURRENT).build(
return new BooleanFieldMapper.Builder(name, ScriptCompiler.NONE, false, IndexVersion.current()).build(
new MapperBuilderContext(parent, false)
);
}
@ -168,7 +168,7 @@ public class FieldAliasMapperValidationTests extends ESTestCase {
}
private static NestedObjectMapper createNestedObjectMapper(String name) {
return new NestedObjectMapper.Builder(name, IndexVersion.CURRENT).build(MapperBuilderContext.root(false));
return new NestedObjectMapper.Builder(name, IndexVersion.current()).build(MapperBuilderContext.root(false));
}
private static MappingLookup createMappingLookup(

View File

@ -24,9 +24,13 @@ public class GeoPointFieldTypeTests extends FieldTypeTestCase {
public void testFetchSourceValue() throws IOException {
boolean ignoreMalformed = randomBoolean();
MappedFieldType mapper = new GeoPointFieldMapper.Builder("field", ScriptCompiler.NONE, ignoreMalformed, IndexVersion.CURRENT, null)
.build(MapperBuilderContext.root(false))
.fieldType();
MappedFieldType mapper = new GeoPointFieldMapper.Builder(
"field",
ScriptCompiler.NONE,
ignoreMalformed,
IndexVersion.current(),
null
).build(MapperBuilderContext.root(false)).fieldType();
Map<String, Object> jsonPoint = Map.of("type", "Point", "coordinates", List.of(42.0, 27.1));
Map<String, Object> otherJsonPoint = Map.of("type", "Point", "coordinates", List.of(30.0, 50.0));
@ -84,7 +88,7 @@ public class GeoPointFieldTypeTests extends FieldTypeTestCase {
}
public void testFetchVectorTile() throws IOException {
MappedFieldType mapper = new GeoPointFieldMapper.Builder("field", ScriptCompiler.NONE, false, IndexVersion.CURRENT, null).build(
MappedFieldType mapper = new GeoPointFieldMapper.Builder("field", ScriptCompiler.NONE, false, IndexVersion.current(), null).build(
MapperBuilderContext.root(false)
).fieldType();
final int z = randomIntBetween(1, 10);

View File

@ -148,7 +148,7 @@ public class GeoPointScriptFieldTypeTests extends AbstractNonTextScriptFieldType
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}

View File

@ -46,7 +46,7 @@ public class IdFieldTypeTests extends ESTestCase {
IndexMetadata indexMetadata = IndexMetadata.builder(IndexMetadata.INDEX_UUID_NA_VALUE).settings(indexSettings).build();
IndexSettings mockSettings = new IndexSettings(indexMetadata, Settings.EMPTY);
Mockito.when(context.getIndexSettings()).thenReturn(mockSettings);
Mockito.when(context.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
Mockito.when(context.indexVersionCreated()).thenReturn(IndexVersion.current());
MappedFieldType ft = new ProvidedIdFieldMapper.IdFieldType(() -> false);
Query query = ft.termQuery("id", context);
assertEquals(new TermInSetQuery("_id", Uid.encodeId("id")), query);

View File

@ -193,7 +193,7 @@ public class IpFieldMapperTests extends MapperTestCase {
MapperParsingException e = expectThrows(
MapperParsingException.class,
() -> createDocumentMapper(IndexVersion.CURRENT, fieldMapping(b -> {
() -> createDocumentMapper(IndexVersion.current(), fieldMapping(b -> {
b.field("type", "ip");
b.field("null_value", ":1");
}))

View File

@ -348,14 +348,14 @@ public class IpFieldTypeTests extends FieldTypeTestCase {
}
public void testFetchSourceValue() throws IOException {
MappedFieldType mapper = new IpFieldMapper.Builder("field", ScriptCompiler.NONE, true, IndexVersion.CURRENT).build(
MappedFieldType mapper = new IpFieldMapper.Builder("field", ScriptCompiler.NONE, true, IndexVersion.current()).build(
MapperBuilderContext.root(false)
).fieldType();
assertEquals(List.of("2001:db8::2:1"), fetchSourceValue(mapper, "2001:db8::2:1"));
assertEquals(List.of("2001:db8::2:1"), fetchSourceValue(mapper, "2001:db8:0:0:0:0:2:1"));
assertEquals(List.of("::1"), fetchSourceValue(mapper, "0:0:0:0:0:0:0:1"));
MappedFieldType nullValueMapper = new IpFieldMapper.Builder("field", ScriptCompiler.NONE, true, IndexVersion.CURRENT).nullValue(
MappedFieldType nullValueMapper = new IpFieldMapper.Builder("field", ScriptCompiler.NONE, true, IndexVersion.current()).nullValue(
"2001:db8:0:0:0:0:2:7"
).build(MapperBuilderContext.root(false)).fieldType();
assertEquals(List.of("2001:db8::2:7"), fetchSourceValue(nullValueMapper, null));

View File

@ -150,7 +150,7 @@ public class IpScriptFieldTypeTests extends AbstractScriptFieldTypeTestCase {
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}

View File

@ -602,7 +602,7 @@ public class KeywordFieldMapperTests extends MapperTestCase {
public void testDimensionInRoutingPath() throws IOException {
MapperService mapper = createMapperService(fieldMapping(b -> b.field("type", "keyword").field("time_series_dimension", true)));
IndexSettings settings = createIndexSettings(
IndexVersion.CURRENT,
IndexVersion.current(),
Settings.builder()
.put(IndexSettings.MODE.getKey(), "time_series")
.put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "field")

View File

@ -213,7 +213,7 @@ public class KeywordFieldTypeTests extends FieldTypeTestCase {
}
public void testFetchSourceValue() throws IOException {
MappedFieldType mapper = new KeywordFieldMapper.Builder("field", IndexVersion.CURRENT).build(MapperBuilderContext.root(false))
MappedFieldType mapper = new KeywordFieldMapper.Builder("field", IndexVersion.current()).build(MapperBuilderContext.root(false))
.fieldType();
assertEquals(List.of("value"), fetchSourceValue(mapper, "value"));
assertEquals(List.of("42"), fetchSourceValue(mapper, 42L));
@ -222,7 +222,7 @@ public class KeywordFieldTypeTests extends FieldTypeTestCase {
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> fetchSourceValue(mapper, "value", "format"));
assertEquals("Field [field] of type [keyword] doesn't support formats.", e.getMessage());
MappedFieldType ignoreAboveMapper = new KeywordFieldMapper.Builder("field", IndexVersion.CURRENT).ignoreAbove(4)
MappedFieldType ignoreAboveMapper = new KeywordFieldMapper.Builder("field", IndexVersion.current()).ignoreAbove(4)
.build(MapperBuilderContext.root(false))
.fieldType();
assertEquals(List.of(), fetchSourceValue(ignoreAboveMapper, "value"));
@ -233,13 +233,13 @@ public class KeywordFieldTypeTests extends FieldTypeTestCase {
"field",
createIndexAnalyzers(),
ScriptCompiler.NONE,
IndexVersion.CURRENT
IndexVersion.current()
).normalizer("lowercase").build(MapperBuilderContext.root(false)).fieldType();
assertEquals(List.of("value"), fetchSourceValue(normalizerMapper, "VALUE"));
assertEquals(List.of("42"), fetchSourceValue(normalizerMapper, 42L));
assertEquals(List.of("value"), fetchSourceValue(normalizerMapper, "value"));
MappedFieldType nullValueMapper = new KeywordFieldMapper.Builder("field", IndexVersion.CURRENT).nullValue("NULL")
MappedFieldType nullValueMapper = new KeywordFieldMapper.Builder("field", IndexVersion.current()).nullValue("NULL")
.build(MapperBuilderContext.root(false))
.fieldType();
assertEquals(List.of("NULL"), fetchSourceValue(nullValueMapper, null));

View File

@ -133,7 +133,7 @@ public class KeywordScriptFieldTypeTests extends AbstractScriptFieldTypeTestCase
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}

View File

@ -174,7 +174,7 @@ public class LongScriptFieldTypeTests extends AbstractNonTextScriptFieldTypeTest
}
};
}
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.CURRENT)), equalTo(1));
}, searchContext.lookup(), 2.5f, "test", 0, IndexVersion.current())), equalTo(1));
}
}
}

View File

@ -32,7 +32,7 @@ import java.util.function.Supplier;
public class MappingParserTests extends MapperServiceTestCase {
private static MappingParser createMappingParser(Settings settings) {
return createMappingParser(settings, IndexVersion.CURRENT, TransportVersion.current());
return createMappingParser(settings, IndexVersion.current(), TransportVersion.current());
}
private static MappingParser createMappingParser(Settings settings, IndexVersion version, TransportVersion transportVersion) {

View File

@ -37,10 +37,10 @@ public class MultiFieldsSerializationTests extends ESTestCase {
sortedNames.sort(Comparator.naturalOrder());
for (String name : names) {
builder.add(new BooleanFieldMapper.Builder(name, ScriptCompiler.NONE, false, IndexVersion.CURRENT));
builder.add(new BooleanFieldMapper.Builder(name, ScriptCompiler.NONE, false, IndexVersion.current()));
}
Mapper.Builder root = new BooleanFieldMapper.Builder("root", ScriptCompiler.NONE, false, IndexVersion.CURRENT);
Mapper.Builder root = new BooleanFieldMapper.Builder("root", ScriptCompiler.NONE, false, IndexVersion.current());
FieldMapper.MultiFields multiFields = builder.build(root, MapperBuilderContext.root(false));
String serialized = Strings.toString(multiFields);

View File

@ -46,7 +46,7 @@ public class NestedDocumentsTests extends MapperServiceTestCase {
}));
withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> {
NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new, IndexVersion.CURRENT);
NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new, IndexVersion.current());
LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0));
assertNotNull(leaf.advance(0));
@ -143,7 +143,7 @@ public class NestedDocumentsTests extends MapperServiceTestCase {
}));
withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> {
NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new, IndexVersion.CURRENT);
NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new, IndexVersion.current());
LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0));
assertNotNull(leaf.advance(0));
@ -258,7 +258,7 @@ public class NestedDocumentsTests extends MapperServiceTestCase {
}));
withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> {
NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new, IndexVersion.CURRENT);
NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new, IndexVersion.current());
LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0));
assertNotNull(leaf.advance(0));

View File

@ -64,7 +64,7 @@ public class NestedLookupTests extends MapperServiceTestCase {
}
private static NestedObjectMapper buildMapper(String name) {
return new NestedObjectMapper.Builder(name, IndexVersion.CURRENT).build(MapperBuilderContext.root(false));
return new NestedObjectMapper.Builder(name, IndexVersion.current()).build(MapperBuilderContext.root(false));
}
public void testAllParentFilters() {

View File

@ -1498,10 +1498,10 @@ public class NestedObjectMapperTests extends MapperServiceTestCase {
}
public void testMergeNested() {
NestedObjectMapper firstMapper = new NestedObjectMapper.Builder("nested1", IndexVersion.CURRENT).includeInParent(true)
NestedObjectMapper firstMapper = new NestedObjectMapper.Builder("nested1", IndexVersion.current()).includeInParent(true)
.includeInRoot(true)
.build(MapperBuilderContext.root(false));
NestedObjectMapper secondMapper = new NestedObjectMapper.Builder("nested1", IndexVersion.CURRENT).includeInParent(false)
NestedObjectMapper secondMapper = new NestedObjectMapper.Builder("nested1", IndexVersion.current()).includeInParent(false)
.includeInRoot(true)
.build(MapperBuilderContext.root(false));

View File

@ -812,7 +812,7 @@ public class NumberFieldTypeTests extends FieldTypeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).build(MapperBuilderContext.root(false)).fieldType();
assertEquals(List.of(3), fetchSourceValue(mapper, 3.14));
@ -825,7 +825,7 @@ public class NumberFieldTypeTests extends FieldTypeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).nullValue(2.71f).build(MapperBuilderContext.root(false)).fieldType();
assertEquals(List.of(2.71f), fetchSourceValue(nullValueMapper, ""));
@ -839,7 +839,7 @@ public class NumberFieldTypeTests extends FieldTypeTestCase {
ScriptCompiler.NONE,
false,
true,
IndexVersion.CURRENT,
IndexVersion.current(),
null
).build(MapperBuilderContext.root(false)).fieldType();
/*

View File

@ -180,7 +180,7 @@ public class ObjectMapperMergeTests extends ESTestCase {
}
private static RootObjectMapper createRootSubobjectFalseLeafWithDots() {
FieldMapper.Builder fieldBuilder = new KeywordFieldMapper.Builder("host.name", IndexVersion.CURRENT);
FieldMapper.Builder fieldBuilder = new KeywordFieldMapper.Builder("host.name", IndexVersion.current());
FieldMapper fieldMapper = fieldBuilder.build(MapperBuilderContext.root(false));
assertEquals("host.name", fieldMapper.simpleName());
assertEquals("host.name", fieldMapper.name());
@ -188,7 +188,7 @@ public class ObjectMapperMergeTests extends ESTestCase {
}
private static ObjectMapper.Builder createObjectSubobjectsFalseLeafWithDots() {
KeywordFieldMapper.Builder fieldBuilder = new KeywordFieldMapper.Builder("host.name", IndexVersion.CURRENT);
KeywordFieldMapper.Builder fieldBuilder = new KeywordFieldMapper.Builder("host.name", IndexVersion.current());
KeywordFieldMapper fieldMapper = fieldBuilder.build(new MapperBuilderContext("foo.metrics", false));
assertEquals("host.name", fieldMapper.simpleName());
assertEquals("foo.metrics.host.name", fieldMapper.name());
@ -212,7 +212,7 @@ public class ObjectMapperMergeTests extends ESTestCase {
private TextFieldMapper.Builder createTextKeywordMultiField(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name, createDefaultIndexAnalyzers());
builder.multiFieldsBuilder.add(new KeywordFieldMapper.Builder("keyword", IndexVersion.CURRENT));
builder.multiFieldsBuilder.add(new KeywordFieldMapper.Builder("keyword", IndexVersion.current()));
return builder;
}
}

View File

@ -290,7 +290,7 @@ public class ParametrizedMapperTests extends MapperServiceTestCase {
}
private static TestMapper fromMapping(String mapping) {
return fromMapping(mapping, IndexVersion.CURRENT, TransportVersion.current());
return fromMapping(mapping, IndexVersion.current(), TransportVersion.current());
}
private String toStringWithDefaults(ToXContent value) throws IOException {

View File

@ -67,7 +67,7 @@ public class TextFieldAnalyzerModeTests extends ESTestCase {
Map<String, Object> fieldNode = new HashMap<>();
fieldNode.put("analyzer", "my_analyzer");
MappingParserContext parserContext = mock(MappingParserContext.class);
when(parserContext.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(parserContext.indexVersionCreated()).thenReturn(IndexVersion.current());
// check AnalysisMode.ALL works
Map<String, NamedAnalyzer> analyzers = defaultAnalyzers();
@ -102,7 +102,7 @@ public class TextFieldAnalyzerModeTests extends ESTestCase {
fieldNode.put("search_analyzer", "standard");
}
MappingParserContext parserContext = mock(MappingParserContext.class);
when(parserContext.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(parserContext.indexVersionCreated()).thenReturn(IndexVersion.current());
// check AnalysisMode.ALL and AnalysisMode.SEARCH_TIME works
Map<String, NamedAnalyzer> analyzers = defaultAnalyzers();
@ -143,7 +143,7 @@ public class TextFieldAnalyzerModeTests extends ESTestCase {
Map<String, Object> fieldNode = new HashMap<>();
fieldNode.put("analyzer", "my_analyzer");
MappingParserContext parserContext = mock(MappingParserContext.class);
when(parserContext.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(parserContext.indexVersionCreated()).thenReturn(IndexVersion.current());
// check that "analyzer" set to AnalysisMode.INDEX_TIME is blocked if there is no search analyzer
AnalysisMode mode = AnalysisMode.INDEX_TIME;

View File

@ -112,7 +112,7 @@ public class TypeParsersTests extends ESTestCase {
// For indices created in 8.0 or later, we should throw an error.
Map<String, Object> fieldNodeCopy = XContentHelper.convertToMap(BytesReference.bytes(mapping), true, mapping.contentType()).v2();
IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersion.V_8_0_0, IndexVersion.CURRENT);
IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersion.V_8_0_0, IndexVersion.current());
TransportVersion transportVersion = TransportVersionUtils.randomVersionBetween(
random(),
TransportVersion.V_8_0_0,

View File

@ -418,7 +418,7 @@ public class FlattenedFieldMapperTests extends MapperTestCase {
fieldMapping(b -> b.field("type", "flattened").field("time_series_dimensions", List.of("key1", "subfield.key2")))
);
IndexSettings settings = createIndexSettings(
IndexVersion.CURRENT,
IndexVersion.current(),
Settings.builder()
.put(IndexSettings.MODE.getKey(), "time_series")
.putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), List.of("field.key1", "field.subfield.key2"))
@ -436,7 +436,7 @@ public class FlattenedFieldMapperTests extends MapperTestCase {
)
);
IndexSettings settings = createIndexSettings(
IndexVersion.CURRENT,
IndexVersion.current(),
Settings.builder()
.put(IndexSettings.MODE.getKey(), "time_series")
.putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), List.of("field.key1", "field.subfield.key2"))
@ -452,7 +452,7 @@ public class FlattenedFieldMapperTests extends MapperTestCase {
fieldMapping(b -> b.field("type", "flattened").field("time_series_dimensions", List.of("key1", "subfield.key2")))
);
IndexSettings settings = createIndexSettings(
IndexVersion.CURRENT,
IndexVersion.current(),
Settings.builder()
.put(IndexSettings.MODE.getKey(), "time_series")
.putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), List.of("field.key1", "field.subfield.key2", "field.key3"))
@ -483,7 +483,7 @@ public class FlattenedFieldMapperTests extends MapperTestCase {
b.endObject();
}));
IndexSettings settings = createIndexSettings(
IndexVersion.CURRENT,
IndexVersion.current(),
Settings.builder()
.put(IndexSettings.MODE.getKey(), "time_series")
.putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), List.of("flattened_field.key1", "keyword_field"))

View File

@ -31,7 +31,7 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
float[][] vectors = { { 1, 1, 1 }, { 1, 1, 2 }, { 1, 1, 3 } };
float[] expectedMagnitudes = { 1.7320f, 2.4495f, 3.3166f };
for (IndexVersion indexVersion : List.of(IndexVersion.V_7_4_0, IndexVersion.CURRENT)) {
for (IndexVersion indexVersion : List.of(IndexVersion.V_7_4_0, IndexVersion.current())) {
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, indexVersion);
DenseVectorDocValuesField field = new BinaryDenseVectorDocValuesField(docValues, "test", ElementType.FLOAT, dims, indexVersion);
DenseVectorScriptDocValues scriptDocValues = field.toScriptDocValues();
@ -50,7 +50,7 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
float[][] vectors = { { 1, 1, 1 }, { 1, 1, 2 }, { 1, 1, 3 } };
float[] expectedMagnitudes = { 1.7320f, 2.4495f, 3.3166f };
BinaryDocValues docValues = wrap(vectors, ElementType.BYTE, IndexVersion.CURRENT);
BinaryDocValues docValues = wrap(vectors, ElementType.BYTE, IndexVersion.current());
DenseVectorDocValuesField field = new ByteBinaryDenseVectorDocValuesField(docValues, "test", ElementType.BYTE, dims);
DenseVectorScriptDocValues scriptDocValues = field.toScriptDocValues();
for (int i = 0; i < vectors.length; i++) {
@ -64,7 +64,7 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
public void testFloatMetadataAndIterator() throws IOException {
int dims = 3;
IndexVersion indexVersion = IndexVersion.CURRENT;
IndexVersion indexVersion = IndexVersion.current();
float[][] vectors = fill(new float[randomIntBetween(1, 5)][dims], ElementType.FLOAT);
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, indexVersion);
DenseVectorDocValuesField field = new BinaryDenseVectorDocValuesField(docValues, "test", ElementType.FLOAT, dims, indexVersion);
@ -84,7 +84,7 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
public void testByteMetadataAndIterator() throws IOException {
int dims = 3;
IndexVersion indexVersion = IndexVersion.CURRENT;
IndexVersion indexVersion = IndexVersion.current();
float[][] vectors = fill(new float[randomIntBetween(1, 5)][dims], ElementType.BYTE);
BinaryDocValues docValues = wrap(vectors, ElementType.BYTE, indexVersion);
DenseVectorDocValuesField field = new ByteBinaryDenseVectorDocValuesField(docValues, "test", ElementType.BYTE, dims);
@ -114,13 +114,13 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
public void testFloatMissingValues() throws IOException {
int dims = 3;
float[][] vectors = { { 1, 1, 1 }, { 1, 1, 2 }, { 1, 1, 3 } };
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, IndexVersion.CURRENT);
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, IndexVersion.current());
DenseVectorDocValuesField field = new BinaryDenseVectorDocValuesField(
docValues,
"test",
ElementType.FLOAT,
dims,
IndexVersion.CURRENT
IndexVersion.current()
);
DenseVectorScriptDocValues scriptDocValues = field.toScriptDocValues();
@ -136,7 +136,7 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
public void testByteMissingValues() throws IOException {
int dims = 3;
float[][] vectors = { { 1, 1, 1 }, { 1, 1, 2 }, { 1, 1, 3 } };
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, IndexVersion.CURRENT);
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, IndexVersion.current());
DenseVectorDocValuesField field = new ByteBinaryDenseVectorDocValuesField(docValues, "test", ElementType.BYTE, dims);
DenseVectorScriptDocValues scriptDocValues = field.toScriptDocValues();
@ -152,13 +152,13 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
public void testFloatGetFunctionIsNotAccessible() throws IOException {
int dims = 3;
float[][] vectors = { { 1, 1, 1 }, { 1, 1, 2 }, { 1, 1, 3 } };
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, IndexVersion.CURRENT);
BinaryDocValues docValues = wrap(vectors, ElementType.FLOAT, IndexVersion.current());
DenseVectorDocValuesField field = new BinaryDenseVectorDocValuesField(
docValues,
"test",
ElementType.FLOAT,
dims,
IndexVersion.CURRENT
IndexVersion.current()
);
DenseVectorScriptDocValues scriptDocValues = field.toScriptDocValues();
@ -175,7 +175,7 @@ public class BinaryDenseVectorScriptDocValuesTests extends ESTestCase {
public void testByteGetFunctionIsNotAccessible() throws IOException {
int dims = 3;
float[][] vectors = { { 1, 1, 1 }, { 1, 1, 2 }, { 1, 1, 3 } };
BinaryDocValues docValues = wrap(vectors, ElementType.BYTE, IndexVersion.CURRENT);
BinaryDocValues docValues = wrap(vectors, ElementType.BYTE, IndexVersion.current());
DenseVectorDocValuesField field = new ByteBinaryDenseVectorDocValuesField(docValues, "test", ElementType.BYTE, dims);
DenseVectorScriptDocValues scriptDocValues = field.toScriptDocValues();

View File

@ -241,8 +241,8 @@ public class DenseVectorFieldMapperTests extends MapperTestCase {
assertThat(fields.get(0), instanceOf(BinaryDocValuesField.class));
// assert that after decoding the indexed value is equal to expected
BytesRef vectorBR = fields.get(0).binaryValue();
float[] decodedValues = decodeDenseVector(IndexVersion.CURRENT, vectorBR);
float decodedMagnitude = VectorEncoderDecoder.decodeMagnitude(IndexVersion.CURRENT, vectorBR);
float[] decodedValues = decodeDenseVector(IndexVersion.current(), vectorBR);
float decodedMagnitude = VectorEncoderDecoder.decodeMagnitude(IndexVersion.current(), vectorBR);
assertEquals(expectedMagnitude, decodedMagnitude, 0.001f);
assertArrayEquals("Decoded dense vector values is not equal to the indexed one.", validVector, decodedValues, 0.001f);
}

View File

@ -36,7 +36,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
private DenseVectorFieldType createFloatFieldType() {
return new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.FLOAT,
5,
indexed,
@ -48,7 +48,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
private DenseVectorFieldType createByteFieldType() {
return new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.BYTE,
5,
true,
@ -113,7 +113,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
public void testFloatCreateKnnQuery() {
DenseVectorFieldType unindexedField = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.FLOAT,
3,
false,
@ -128,7 +128,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
DenseVectorFieldType dotProductField = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.FLOAT,
3,
true,
@ -143,7 +143,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
DenseVectorFieldType cosineField = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.FLOAT,
3,
true,
@ -161,7 +161,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
{ // float type with 2048 dims
DenseVectorFieldType fieldWith2048dims = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.FLOAT,
2048,
true,
@ -179,7 +179,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
{ // byte type with 2048 dims
DenseVectorFieldType fieldWith2048dims = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.BYTE,
2048,
true,
@ -198,7 +198,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
public void testByteCreateKnnQuery() {
DenseVectorFieldType unindexedField = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.BYTE,
3,
false,
@ -213,7 +213,7 @@ public class DenseVectorFieldTypeTests extends FieldTypeTestCase {
DenseVectorFieldType cosineField = new DenseVectorFieldType(
"f",
IndexVersion.CURRENT,
IndexVersion.current(),
DenseVectorFieldMapper.ElementType.BYTE,
3,
true,

View File

@ -91,9 +91,9 @@ public class IndicesModuleTests extends ESTestCase {
public void testBuiltinMappers() {
IndicesModule module = new IndicesModule(Collections.emptyList());
{
IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersion.V_8_0_0, IndexVersion.CURRENT);
IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersion.V_8_0_0, IndexVersion.current());
assertThat(
module.getMapperRegistry().getMapperParser("object", IndexVersion.CURRENT),
module.getMapperRegistry().getMapperParser("object", IndexVersion.current()),
instanceOf(ObjectMapper.TypeParser.class)
);
assertFalse(module.getMapperRegistry().getMetadataMapperParsers(version).isEmpty());
@ -119,14 +119,14 @@ public class IndicesModuleTests extends ESTestCase {
IndicesModule noPluginsModule = new IndicesModule(Collections.emptyList());
IndicesModule module = new IndicesModule(fakePlugins);
MapperRegistry registry = module.getMapperRegistry();
assertThat(registry.getMapperParser("fake-mapper", IndexVersion.CURRENT), instanceOf(FakeMapperParser.class));
assertNull(noPluginsModule.getMapperRegistry().getMapperParser("fake-mapper", IndexVersion.CURRENT));
assertThat(registry.getMapperParser("fake-mapper", IndexVersion.current()), instanceOf(FakeMapperParser.class));
assertNull(noPluginsModule.getMapperRegistry().getMapperParser("fake-mapper", IndexVersion.current()));
assertThat(
registry.getMetadataMapperParsers(IndexVersion.CURRENT).size(),
greaterThan(noPluginsModule.getMapperRegistry().getMetadataMapperParsers(IndexVersion.CURRENT).size())
registry.getMetadataMapperParsers(IndexVersion.current()).size(),
greaterThan(noPluginsModule.getMapperRegistry().getMetadataMapperParsers(IndexVersion.current()).size())
);
Map<String, MetadataFieldMapper.TypeParser> metadataMapperParsers = module.getMapperRegistry()
.getMetadataMapperParsers(IndexVersion.CURRENT);
.getMetadataMapperParsers(IndexVersion.current());
Iterator<String> iterator = metadataMapperParsers.keySet().iterator();
assertEquals(IgnoredFieldMapper.NAME, iterator.next());
String last = null;

View File

@ -102,7 +102,7 @@ public class ShardSnapshotTaskRunnerTests extends ESTestCase {
public static BlobStoreIndexShardSnapshot.FileInfo dummyFileInfo() {
String filename = randomAlphaOfLength(10);
StoreFileMetadata metadata = new StoreFileMetadata(filename, 10, "CHECKSUM", IndexVersion.CURRENT.luceneVersion().toString());
StoreFileMetadata metadata = new StoreFileMetadata(filename, 10, "CHECKSUM", IndexVersion.current().luceneVersion().toString());
return new BlobStoreIndexShardSnapshot.FileInfo(filename, metadata, null);
}

View File

@ -49,11 +49,11 @@ public class VectorScoreScriptUtilsTests extends ESTestCase {
IndexVersion.V_7_4_0
),
new BinaryDenseVectorDocValuesField(
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.FLOAT, IndexVersion.CURRENT),
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.FLOAT, IndexVersion.current()),
"test",
ElementType.FLOAT,
dims,
IndexVersion.CURRENT
IndexVersion.current()
),
new KnnDenseVectorDocValuesField(KnnDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }), "test", dims)
);
@ -130,7 +130,7 @@ public class VectorScoreScriptUtilsTests extends ESTestCase {
List<DenseVectorDocValuesField> fields = List.of(
new ByteBinaryDenseVectorDocValuesField(
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.BYTE, IndexVersion.CURRENT),
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.BYTE, IndexVersion.current()),
"test",
ElementType.BYTE,
dims
@ -217,15 +217,15 @@ public class VectorScoreScriptUtilsTests extends ESTestCase {
IndexVersion.V_7_4_0
),
new BinaryDenseVectorDocValuesField(
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.FLOAT, IndexVersion.CURRENT),
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.FLOAT, IndexVersion.current()),
"field1",
ElementType.FLOAT,
dims,
IndexVersion.CURRENT
IndexVersion.current()
),
new KnnDenseVectorDocValuesField(KnnDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }), "field2", dims),
new ByteBinaryDenseVectorDocValuesField(
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.BYTE, IndexVersion.CURRENT),
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.BYTE, IndexVersion.current()),
"field3",
ElementType.BYTE,
dims
@ -363,7 +363,7 @@ public class VectorScoreScriptUtilsTests extends ESTestCase {
List<DenseVectorDocValuesField> fields = List.of(
new ByteBinaryDenseVectorDocValuesField(
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.BYTE, IndexVersion.CURRENT),
BinaryDenseVectorScriptDocValuesTests.wrap(new float[][] { docVector }, ElementType.BYTE, IndexVersion.current()),
"test",
ElementType.BYTE,
dims

View File

@ -67,7 +67,7 @@ public class DenseVectorTests extends ESTestCase {
assertEquals(knn.cosineSimilarity(arrayQV), knn.cosineSimilarity(listQV), 0.001f);
assertEquals(knn.cosineSimilarity((Object) listQV), knn.cosineSimilarity((Object) arrayQV), 0.001f);
for (IndexVersion indexVersion : List.of(IndexVersion.V_7_4_0, IndexVersion.CURRENT)) {
for (IndexVersion indexVersion : List.of(IndexVersion.V_7_4_0, IndexVersion.current())) {
BytesRef value = BinaryDenseVectorScriptDocValuesTests.mockEncodeDenseVector(docVector, ElementType.FLOAT, indexVersion);
BinaryDenseVector bdv = new BinaryDenseVector(docVector, value, dims, indexVersion);
@ -113,7 +113,7 @@ public class DenseVectorTests extends ESTestCase {
assertEquals(knn.cosineSimilarity(arrayQV), knn.cosineSimilarity(listQV), 0.001f);
assertEquals(knn.cosineSimilarity((Object) listQV), knn.cosineSimilarity((Object) arrayQV), 0.001f);
BytesRef value = BinaryDenseVectorScriptDocValuesTests.mockEncodeDenseVector(floatVector, ElementType.BYTE, IndexVersion.CURRENT);
BytesRef value = BinaryDenseVectorScriptDocValuesTests.mockEncodeDenseVector(floatVector, ElementType.BYTE, IndexVersion.current());
byte[] byteVectorValue = new byte[dims];
System.arraycopy(value.bytes, value.offset, byteVectorValue, 0, dims);
ByteBinaryDenseVector bdv = new ByteBinaryDenseVector(byteVectorValue, value, dims);
@ -220,7 +220,7 @@ public class DenseVectorTests extends ESTestCase {
e = expectThrows(UnsupportedOperationException.class, () -> knn.cosineSimilarity((Object) queryVector));
assertEquals(e.getMessage(), "use [double cosineSimilarity(float[] queryVector, boolean normalizeQueryVector)] instead");
BinaryDenseVector binary = new BinaryDenseVector(docVector, new BytesRef(docBuffer.array()), dims, IndexVersion.CURRENT);
BinaryDenseVector binary = new BinaryDenseVector(docVector, new BytesRef(docBuffer.array()), dims, IndexVersion.current());
e = expectThrows(UnsupportedOperationException.class, () -> binary.dotProduct(queryVector));
assertEquals(e.getMessage(), "use [double dotProduct(float[] queryVector)] instead");

View File

@ -500,7 +500,7 @@ public class FiltersAggregatorTests extends AggregatorTestCase {
assertThat(filters1.getBucketByKey("q1").getDocCount(), equalTo(1L));
},
new AggTestConfig(new FiltersAggregationBuilder("test", new KeyedFilter("q1", new TermQueryBuilder("author", "foo"))), ft)
.withQuery(Queries.newNonNestedFilter(IndexVersion.CURRENT))
.withQuery(Queries.newNonNestedFilter(IndexVersion.current()))
);
testCase(buildIndex, result -> {
InternalFilters filters = (InternalFilters) result;
@ -508,7 +508,7 @@ public class FiltersAggregatorTests extends AggregatorTestCase {
assertThat(filters.getBucketByKey("q1").getDocCount(), equalTo(1L));
},
new AggTestConfig(new FiltersAggregationBuilder("test", new KeyedFilter("q1", new MatchAllQueryBuilder())), ft).withQuery(
Queries.newNonNestedFilter(IndexVersion.CURRENT)
Queries.newNonNestedFilter(IndexVersion.current())
)
);
}

View File

@ -367,7 +367,7 @@ public class NestedAggregatorTests extends AggregatorTestCase {
MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(VALUE_FIELD_NAME, NumberFieldMapper.NumberType.LONG);
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(Queries.newNonNestedFilter(IndexVersion.CURRENT), BooleanClause.Occur.MUST);
bq.add(Queries.newNonNestedFilter(IndexVersion.current()), BooleanClause.Occur.MUST);
bq.add(new TermQuery(new Term(IdFieldMapper.NAME, Uid.encodeId("2"))), BooleanClause.Occur.MUST_NOT);
InternalNested nested = searchAndReduce(
@ -644,7 +644,7 @@ public class NestedAggregatorTests extends AggregatorTestCase {
Filter filter = searchAndReduce(
newIndexSearcher(indexReader),
new AggTestConfig(filterAggregationBuilder, fieldType1, fieldType2).withQuery(
Queries.newNonNestedFilter(IndexVersion.CURRENT)
Queries.newNonNestedFilter(IndexVersion.current())
)
);
@ -906,6 +906,6 @@ public class NestedAggregatorTests extends AggregatorTestCase {
);
public static NestedObjectMapper nestedObject(String path) {
return new NestedObjectMapper.Builder(path, IndexVersion.CURRENT).build(MapperBuilderContext.root(false));
return new NestedObjectMapper.Builder(path, IndexVersion.current()).build(MapperBuilderContext.root(false));
}
}

View File

@ -1438,7 +1438,7 @@ public class TermsAggregatorTests extends AggregatorTestCase {
StringTerms result = searchAndReduce(
newIndexSearcher(indexReader),
new AggTestConfig(terms, animalFieldType, nestedFieldType).withQuery(
Queries.newNonNestedFilter(IndexVersion.CURRENT)
Queries.newNonNestedFilter(IndexVersion.current())
)
);
assertThat(result.getBuckets().get(0).getKeyAsString(), equalTo("pig"));

View File

@ -266,7 +266,7 @@ public class FieldFetcherTests extends MapperServiceTestCase {
SeqNoFieldMapper.NAME,
SourceFieldMapper.NAME,
FieldNamesFieldMapper.NAME,
NestedPathFieldMapper.name(IndexVersion.CURRENT)
NestedPathFieldMapper.name(IndexVersion.current())
)) {
expectThrows(UnsupportedOperationException.class, () -> fetchFields(mapperService, source, fieldname));
}

View File

@ -88,7 +88,7 @@ public class ScriptScoreQueryTests extends ESTestCase {
null,
"index",
0,
IndexVersion.CURRENT
IndexVersion.current()
);
Weight weight = query.createWeight(searcher, ScoreMode.COMPLETE, 1.0f);
Explanation explanation = weight.explain(leafReaderContext, 0);
@ -109,7 +109,7 @@ public class ScriptScoreQueryTests extends ESTestCase {
null,
"index",
0,
IndexVersion.CURRENT
IndexVersion.current()
);
Weight weight = query.createWeight(searcher, ScoreMode.COMPLETE, 1.0f);
Explanation explanation = weight.explain(leafReaderContext, 0);
@ -134,7 +134,7 @@ public class ScriptScoreQueryTests extends ESTestCase {
null,
"index",
0,
IndexVersion.CURRENT
IndexVersion.current()
);
Weight weight = query.createWeight(searcher, ScoreMode.COMPLETE, 1.0f);
Explanation explanation = weight.explain(leafReaderContext, 0);
@ -157,7 +157,7 @@ public class ScriptScoreQueryTests extends ESTestCase {
null,
"index",
0,
IndexVersion.CURRENT
IndexVersion.current()
);
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> searcher.search(query, 1));
assertTrue(e.getMessage().contains("Must be a non-negative score!"));

View File

@ -187,7 +187,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
Index index = new Index(randomAlphaOfLengthBetween(1, 10), "_na_");
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(
index,
Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.CURRENT.id()).build()
Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current().id()).build()
);
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, mock(BitsetFilterCache.Listener.class));
BiFunction<MappedFieldType, FieldDataContext, IndexFieldData<?>> indexFieldDataLookup = (fieldType, fdc) -> {
@ -195,7 +195,7 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
return builder.build(new IndexFieldDataCache.None(), null);
};
NestedLookup nestedLookup = NestedLookup.build(
List.of(new NestedObjectMapper.Builder("path", IndexVersion.CURRENT).build(MapperBuilderContext.root(false)))
List.of(new NestedObjectMapper.Builder("path", IndexVersion.current()).build(MapperBuilderContext.root(false)))
);
return new SearchExecutionContext(
0,

View File

@ -477,7 +477,7 @@ public final class DataStreamTestHelper {
null,
ScriptCompiler.NONE,
false,
IndexVersion.CURRENT
IndexVersion.current()
).build(MapperBuilderContext.root(false));
ClusterService clusterService = ClusterServiceUtils.createClusterService(testThreadPool);
Environment env = mock(Environment.class);
@ -495,7 +495,7 @@ public final class DataStreamTestHelper {
DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER,
ScriptCompiler.NONE,
true,
IndexVersion.CURRENT
IndexVersion.current()
)
);
MetadataFieldMapper dtfm = getDataStreamTimestampFieldMapper();

View File

@ -42,7 +42,7 @@ public abstract class FieldTypeTestCase extends ESTestCase {
when(searchExecutionContext.isSourceEnabled()).thenReturn(true);
SearchLookup searchLookup = mock(SearchLookup.class);
when(searchExecutionContext.lookup()).thenReturn(searchLookup);
when(searchExecutionContext.indexVersionCreated()).thenReturn(IndexVersion.CURRENT);
when(searchExecutionContext.indexVersionCreated()).thenReturn(IndexVersion.current());
return searchExecutionContext;
}

View File

@ -155,7 +155,7 @@ public abstract class MapperServiceTestCase extends ESTestCase {
}
protected IndexVersion getVersion() {
return IndexVersion.CURRENT;
return IndexVersion.current();
}
protected final MapperService createMapperService(Settings settings, XContentBuilder mappings) throws IOException {
@ -173,7 +173,7 @@ public abstract class MapperServiceTestCase extends ESTestCase {
}
protected final MapperService createMapperService(Settings settings, String mappings) throws IOException {
MapperService mapperService = createMapperService(IndexVersion.CURRENT, settings, () -> true, mapping(b -> {}));
MapperService mapperService = createMapperService(IndexVersion.current(), settings, () -> true, mapping(b -> {}));
merge(mapperService, mappings);
return mapperService;
}

View File

@ -975,7 +975,7 @@ public abstract class MapperTestCase extends MapperServiceTestCase {
MapperService mapper = createMapperService(fieldMapping(this::minimalMapping));
try {
IndexSettings settings = createIndexSettings(
IndexVersion.CURRENT,
IndexVersion.current(),
Settings.builder()
.put(IndexSettings.MODE.getKey(), "time_series")
.put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "field")

View File

@ -166,7 +166,7 @@ public abstract class MetadataMapperTestCase extends MapperServiceTestCase {
public void testTypeAndFriendsAreDeprecatedFrom_8_6_0() throws IOException {
assumeTrue("Metadata field " + fieldName() + " isn't configurable", isConfigurable());
IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersion.V_8_6_0, IndexVersion.CURRENT);
IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersion.V_8_6_0, IndexVersion.current());
assumeTrue("Metadata field " + fieldName() + " is not supported on version " + version, isSupportedOn(version));
MapperService mapperService = createMapperService(version, mapping(b -> {}));
// these parameters were previously silently ignored, they are now deprecated in new indices

View File

@ -53,12 +53,12 @@ public class TestDocumentParserContext extends DocumentParserContext {
s -> null,
s -> null,
s -> null,
IndexVersion.CURRENT,
IndexVersion.current(),
() -> TransportVersion.current(),
() -> null,
null,
(type, name) -> Lucene.STANDARD_ANALYZER,
MapperTestCase.createIndexSettings(IndexVersion.CURRENT, Settings.EMPTY),
MapperTestCase.createIndexSettings(IndexVersion.current(), Settings.EMPTY),
null
),
source,

View File

@ -1221,7 +1221,7 @@ public abstract class AggregatorTestCase extends ESTestCase {
null,
null,
null,
IndexVersion.CURRENT,
IndexVersion.current(),
() -> TransportVersion.current(),
null,
ScriptCompiler.NONE,

View File

@ -1910,7 +1910,7 @@ public abstract class ESTestCase extends LuceneTestCase {
* @param message an additional message or link with information on the fix
*/
protected void skipTestWaitingForLuceneFix(org.apache.lucene.util.Version luceneVersionWithFix, String message) {
final boolean currentVersionHasFix = IndexVersion.CURRENT.luceneVersion().onOrAfter(luceneVersionWithFix);
final boolean currentVersionHasFix = IndexVersion.current().luceneVersion().onOrAfter(luceneVersionWithFix);
assumeTrue("Skipping test as it is waiting on a Lucene fix: " + message, currentVersionHasFix);
fail("Remove call of skipTestWaitingForLuceneFix in " + RandomizedTest.getContext().getTargetMethod());
}

View File

@ -74,8 +74,8 @@ public class IndexVersionUtils {
}
public static IndexVersion getPreviousVersion() {
IndexVersion version = getPreviousVersion(IndexVersion.CURRENT);
assert version.before(IndexVersion.CURRENT);
IndexVersion version = getPreviousVersion(IndexVersion.current());
assert version.before(IndexVersion.current());
return version;
}
@ -108,9 +108,9 @@ public class IndexVersionUtils {
return ALL_VERSIONS.get(place);
}
/** Returns a random {@code IndexVersion} that is compatible with {@link IndexVersion#CURRENT} */
/** Returns a random {@code IndexVersion} that is compatible with {@link IndexVersion#current()} */
public static IndexVersion randomCompatibleVersion(Random random) {
return randomVersionBetween(random, IndexVersion.MINIMUM_COMPATIBLE, IndexVersion.CURRENT);
return randomVersionBetween(random, IndexVersion.MINIMUM_COMPATIBLE, IndexVersion.current());
}
/** Returns a random {@code IndexVersion} that is compatible with the previous version to {@code version} */

View File

@ -147,7 +147,7 @@ public class AggregateDoubleMetricFieldTypeTests extends FieldTypeTestCase {
}
};
}
}, searchExecutionContext.lookup(), 7f, "test", 0, IndexVersion.CURRENT)), equalTo(2));
}, searchExecutionContext.lookup(), 7f, "test", 0, IndexVersion.current())), equalTo(2));
}
}
}

View File

@ -458,7 +458,7 @@ public class ClusterStatsMonitoringDocTests extends BaseMonitoringDocTestCase<Cl
Version.CURRENT,
Version.CURRENT,
IndexVersion.MINIMUM_COMPATIBLE,
IndexVersion.CURRENT,
IndexVersion.current(),
apmIndicesExist };
final String expectedJson = Strings.format("""
{

View File

@ -103,7 +103,7 @@ public class IndexRecoveryMonitoringDocTests extends BaseMonitoringDocTestCase<I
new TransportAddress(TransportAddress.META_ADDRESS, 9301),
singletonMap("attr", "value_1"),
singleton(DiscoveryNodeRole.DATA_ROLE),
new VersionInformation(Version.CURRENT.minimumCompatibilityVersion(), IndexVersion.MINIMUM_COMPATIBLE, IndexVersion.CURRENT)
new VersionInformation(Version.CURRENT.minimumCompatibilityVersion(), IndexVersion.MINIMUM_COMPATIBLE, IndexVersion.current())
);
final ShardId shardId = new ShardId("_index_a", "_uuid_a", 0);

Some files were not shown because too many files have changed in this diff Show More