Преглед на файлове

Migrate IndexMetadata.getCreationVersion to IndexVersion (#97139)

Simon Cooper преди 2 години
родител
ревизия
602ccd8d60
променени са 17 файла, в които са добавени 89 реда и са изтрити 100 реда
  1. 15 15
      docs/reference/migration/apis/feature-migration.asciidoc
  2. 3 2
      qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/FeatureUpgradeIT.java
  3. 2 2
      qa/system-indices/src/javaRestTest/java/org/elasticsearch/system/indices/FeatureUpgradeApiIT.java
  4. 12 12
      server/src/main/java/org/elasticsearch/action/admin/cluster/migration/GetFeatureUpgradeStatusResponse.java
  5. 6 4
      server/src/main/java/org/elasticsearch/action/admin/cluster/migration/TransportGetFeatureUpgradeStatusAction.java
  6. 18 36
      server/src/main/java/org/elasticsearch/action/admin/cluster/stats/VersionStats.java
  7. 3 3
      server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java
  8. 2 0
      server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java
  9. 1 1
      server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java
  10. 1 1
      server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java
  11. 2 2
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java
  12. 1 1
      server/src/main/java/org/elasticsearch/index/shard/ShardSplittingQuery.java
  13. 2 2
      server/src/main/java/org/elasticsearch/snapshots/RestoreService.java
  14. 2 2
      server/src/main/java/org/elasticsearch/upgrades/SystemIndexMigrator.java
  15. 3 3
      server/src/test/java/org/elasticsearch/action/admin/cluster/migration/GetFeatureUpgradeStatusResponseTests.java
  16. 8 6
      server/src/test/java/org/elasticsearch/action/admin/cluster/migration/TransportGetFeatureUpgradeStatusActionTests.java
  17. 8 8
      server/src/test/java/org/elasticsearch/action/admin/cluster/stats/VersionStatsTests.java

+ 15 - 15
docs/reference/migration/apis/feature-migration.asciidoc

@@ -56,85 +56,85 @@ Example response:
   "features" : [
     {
       "feature_name" : "async_search",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "enrich",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "ent_search",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "fleet",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "geoip",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "kibana",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "logstash_management",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "machine_learning",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "searchable_snapshots",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "security",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "synonyms",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "tasks",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "transform",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     },
     {
       "feature_name" : "watcher",
-      "minimum_index_version" : "{version}",
+      "minimum_index_version" : "8100099",
       "migration_status" : "NO_MIGRATION_NEEDED",
       "indices" : [ ]
     }
@@ -142,7 +142,7 @@ Example response:
   "migration_status" : "NO_MIGRATION_NEEDED"
 }
 --------------------------------------------------
-
+// TESTRESPONSE[s/"minimum_index_version" : "8100099"/"minimum_index_version" : $body.$_path/]
 
 When you submit a POST request to the `_migration/system_features` endpoint to
 start the migration process, the response indicates what features will be

+ 3 - 2
qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/FeatureUpgradeIT.java

@@ -17,6 +17,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import static org.hamcrest.Matchers.aMapWithSize;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
 
@@ -92,8 +93,8 @@ public class FeatureUpgradeIT extends AbstractRollingTestCase {
                     .findFirst()
                     .orElse(Collections.emptyMap());
 
-                assertThat(feature.size(), equalTo(4));
-                assertThat(feature.get("minimum_index_version"), equalTo(UPGRADE_FROM_VERSION.toString()));
+                assertThat(feature, aMapWithSize(4));
+                assertThat(feature.get("minimum_index_version"), equalTo(Integer.toString(UPGRADE_FROM_VERSION.id)));
                 if (UPGRADE_FROM_VERSION.before(TransportGetFeatureUpgradeStatusAction.NO_UPGRADE_REQUIRED_VERSION)) {
                     assertThat(feature.get("migration_status"), equalTo("MIGRATION_NEEDED"));
                 } else {

+ 2 - 2
qa/system-indices/src/javaRestTest/java/org/elasticsearch/system/indices/FeatureUpgradeApiIT.java

@@ -8,12 +8,12 @@
 
 package org.elasticsearch.system.indices;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.client.Request;
 import org.elasticsearch.client.Response;
 import org.elasticsearch.common.settings.SecureString;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.test.XContentTestUtils;
 import org.elasticsearch.test.rest.ESRestTestCase;
 import org.junit.After;
@@ -61,7 +61,7 @@ public class FeatureUpgradeApiIT extends ESRestTestCase {
             .orElse(Collections.emptyMap());
 
         assertThat(testFeature.size(), equalTo(4));
-        assertThat(testFeature.get("minimum_index_version"), equalTo(Version.CURRENT.toString()));
+        assertThat(testFeature.get("minimum_index_version"), equalTo(IndexVersion.current().toString()));
         assertThat(testFeature.get("migration_status"), equalTo("NO_MIGRATION_NEEDED"));
         assertThat(testFeature.get("indices"), instanceOf(List.class));
 

+ 12 - 12
server/src/main/java/org/elasticsearch/action/admin/cluster/migration/GetFeatureUpgradeStatusResponse.java

@@ -9,12 +9,12 @@
 package org.elasticsearch.action.admin.cluster.migration;
 
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.core.Nullable;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.xcontent.ToXContentObject;
 import org.elasticsearch.xcontent.XContentBuilder;
 
@@ -124,7 +124,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
      */
     public static class FeatureUpgradeStatus implements Writeable, ToXContentObject {
         private final String featureName;
-        private final Version minimumIndexVersion;
+        private final IndexVersion minimumIndexVersion;
         private final UpgradeStatus upgradeStatus;
         private final List<IndexInfo> indexInfos;
 
@@ -136,7 +136,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
          */
         public FeatureUpgradeStatus(
             String featureName,
-            Version minimumIndexVersion,
+            IndexVersion minimumIndexVersion,
             UpgradeStatus upgradeStatus,
             List<IndexInfo> indexInfos
         ) {
@@ -152,7 +152,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
          */
         public FeatureUpgradeStatus(StreamInput in) throws IOException {
             this.featureName = in.readString();
-            this.minimumIndexVersion = Version.readVersion(in);
+            this.minimumIndexVersion = IndexVersion.readVersion(in);
             this.upgradeStatus = in.readEnum(UpgradeStatus.class);
             this.indexInfos = in.readImmutableList(IndexInfo::new);
         }
@@ -161,7 +161,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
             return this.featureName;
         }
 
-        public Version getMinimumIndexVersion() {
+        public IndexVersion getMinimumIndexVersion() {
             return this.minimumIndexVersion;
         }
 
@@ -176,7 +176,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
         @Override
         public void writeTo(StreamOutput out) throws IOException {
             out.writeString(this.featureName);
-            Version.writeVersion(this.minimumIndexVersion, out);
+            IndexVersion.writeVersion(this.minimumIndexVersion, out);
             out.writeEnum(this.upgradeStatus);
             out.writeList(this.indexInfos);
         }
@@ -240,16 +240,16 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
         );
 
         private final String indexName;
-        private final Version version;
+        private final IndexVersion version;
         @Nullable
         private final Exception exception; // Present if this index failed
 
         /**
          * @param indexName Name of the index
-         * @param version Version of Elasticsearch that created the index
+         * @param version   Index version
          * @param exception The exception that this index's migration failed with, if applicable
          */
-        public IndexInfo(String indexName, Version version, Exception exception) {
+        public IndexInfo(String indexName, IndexVersion version, Exception exception) {
             this.indexName = indexName;
             this.version = version;
             this.exception = exception;
@@ -261,7 +261,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
          */
         public IndexInfo(StreamInput in) throws IOException {
             this.indexName = in.readString();
-            this.version = Version.readVersion(in);
+            this.version = IndexVersion.readVersion(in);
             boolean hasException = in.readBoolean();
             if (hasException) {
                 this.exception = in.readException();
@@ -274,7 +274,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
             return this.indexName;
         }
 
-        public Version getVersion() {
+        public IndexVersion getVersion() {
             return this.version;
         }
 
@@ -285,7 +285,7 @@ public class GetFeatureUpgradeStatusResponse extends ActionResponse implements T
         @Override
         public void writeTo(StreamOutput out) throws IOException {
             out.writeString(this.indexName);
-            Version.writeVersion(this.version, out);
+            IndexVersion.writeVersion(this.version, out);
             if (exception != null) {
                 out.writeBoolean(true);
                 out.writeException(this.exception);

+ 6 - 4
server/src/main/java/org/elasticsearch/action/admin/cluster/migration/TransportGetFeatureUpgradeStatusAction.java

@@ -18,6 +18,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.inject.Inject;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.indices.SystemIndices;
 import org.elasticsearch.persistent.PersistentTasksCustomMetadata;
 import org.elasticsearch.persistent.PersistentTasksService;
@@ -52,6 +53,7 @@ public class TransportGetFeatureUpgradeStatusAction extends TransportMasterNodeA
      * Once all feature migrations for 8.x -> 9.x have been tested, we can bump this to Version.V_8_0_0
      */
     public static final Version NO_UPGRADE_REQUIRED_VERSION = Version.V_7_0_0;
+    public static final IndexVersion NO_UPGRADE_REQUIRED_INDEX_VERSION = IndexVersion.V_7_0_0;
 
     private final SystemIndices systemIndices;
     PersistentTasksService persistentTasksService;
@@ -124,14 +126,14 @@ public class TransportGetFeatureUpgradeStatusAction extends TransportMasterNodeA
 
         List<GetFeatureUpgradeStatusResponse.IndexInfo> indexInfos = getIndexInfos(state, feature);
 
-        Version minimumVersion = indexInfos.stream()
+        IndexVersion minimumVersion = indexInfos.stream()
             .map(GetFeatureUpgradeStatusResponse.IndexInfo::getVersion)
-            .min(Version::compareTo)
-            .orElse(Version.CURRENT);
+            .min(IndexVersion::compareTo)
+            .orElse(IndexVersion.current());
         GetFeatureUpgradeStatusResponse.UpgradeStatus initialStatus;
         if (featureName.equals(currentFeature)) {
             initialStatus = IN_PROGRESS;
-        } else if (minimumVersion.before(NO_UPGRADE_REQUIRED_VERSION)) {
+        } else if (minimumVersion.before(NO_UPGRADE_REQUIRED_INDEX_VERSION)) {
             initialStatus = MIGRATION_NEEDED;
         } else {
             initialStatus = NO_MIGRATION_NEEDED;

+ 18 - 36
server/src/main/java/org/elasticsearch/action/admin/cluster/stats/VersionStats.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.action.admin.cluster.stats;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.action.admin.indices.stats.ShardStats;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
@@ -17,6 +16,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.xcontent.ToXContentFragment;
 import org.elasticsearch.xcontent.ToXContentObject;
 import org.elasticsearch.xcontent.XContentBuilder;
@@ -42,9 +42,9 @@ public final class VersionStats implements ToXContentFragment, Writeable {
     private final Set<SingleVersionStats> versionStats;
 
     public static VersionStats of(Metadata metadata, List<ClusterStatsNodeResponse> nodeResponses) {
-        final Map<Version, Integer> indexCounts = new HashMap<>();
-        final Map<Version, Integer> primaryShardCounts = new HashMap<>();
-        final Map<Version, Long> primaryByteCounts = new HashMap<>();
+        final Map<IndexVersion, Integer> indexCounts = new HashMap<>();
+        final Map<IndexVersion, Integer> primaryShardCounts = new HashMap<>();
+        final Map<IndexVersion, Long> primaryByteCounts = new HashMap<>();
         final Map<String, List<ShardStats>> indexPrimaryShardStats = new HashMap<>();
 
         // Build a map from index name to primary shard stats
@@ -69,38 +69,20 @@ public final class VersionStats implements ToXContentFragment, Writeable {
         for (Map.Entry<String, IndexMetadata> cursor : metadata.indices().entrySet()) {
             IndexMetadata indexMetadata = cursor.getValue();
             // Increment version-specific index counts
-            indexCounts.compute(indexMetadata.getCreationVersion(), (v, i) -> {
-                if (i == null) {
-                    return 1;
-                } else {
-                    return i + 1;
-                }
-            });
+            indexCounts.merge(indexMetadata.getCreationVersion(), 1, Integer::sum);
             // Increment version-specific primary shard counts
-            primaryShardCounts.compute(indexMetadata.getCreationVersion(), (v, i) -> {
-                if (i == null) {
-                    return indexMetadata.getNumberOfShards();
-                } else {
-                    return i + indexMetadata.getNumberOfShards();
-                }
-            });
+            primaryShardCounts.merge(indexMetadata.getCreationVersion(), indexMetadata.getNumberOfShards(), Integer::sum);
             // Increment version-specific primary shard sizes
-            primaryByteCounts.compute(indexMetadata.getCreationVersion(), (v, i) -> {
-                String indexName = indexMetadata.getIndex().getName();
-                long indexPrimarySize = indexPrimaryShardStats.getOrDefault(indexName, Collections.emptyList())
-                    .stream()
-                    .mapToLong(stats -> stats.getStats().getStore().sizeInBytes())
-                    .sum();
-                if (i == null) {
-                    return indexPrimarySize;
-                } else {
-                    return i + indexPrimarySize;
-                }
-            });
+            String indexName = indexMetadata.getIndex().getName();
+            long indexPrimarySize = indexPrimaryShardStats.getOrDefault(indexName, Collections.emptyList())
+                .stream()
+                .mapToLong(stats -> stats.getStats().getStore().sizeInBytes())
+                .sum();
+            primaryByteCounts.merge(indexMetadata.getCreationVersion(), indexPrimarySize, Long::sum);
         }
         List<SingleVersionStats> calculatedStats = new ArrayList<>(indexCounts.size());
-        for (Map.Entry<Version, Integer> indexVersionCount : indexCounts.entrySet()) {
-            Version v = indexVersionCount.getKey();
+        for (Map.Entry<IndexVersion, Integer> indexVersionCount : indexCounts.entrySet()) {
+            IndexVersion v = indexVersionCount.getKey();
             SingleVersionStats singleStats = new SingleVersionStats(
                 v,
                 indexVersionCount.getValue(),
@@ -164,12 +146,12 @@ public final class VersionStats implements ToXContentFragment, Writeable {
 
     static class SingleVersionStats implements ToXContentObject, Writeable, Comparable<SingleVersionStats> {
 
-        public final Version version;
+        public final IndexVersion version;
         public final int indexCount;
         public final int primaryShardCount;
         public final long totalPrimaryByteCount;
 
-        SingleVersionStats(Version version, int indexCount, int primaryShardCount, long totalPrimaryByteCount) {
+        SingleVersionStats(IndexVersion version, int indexCount, int primaryShardCount, long totalPrimaryByteCount) {
             this.version = version;
             this.indexCount = indexCount;
             this.primaryShardCount = primaryShardCount;
@@ -177,7 +159,7 @@ public final class VersionStats implements ToXContentFragment, Writeable {
         }
 
         SingleVersionStats(StreamInput in) throws IOException {
-            this.version = Version.readVersion(in);
+            this.version = IndexVersion.readVersion(in);
             this.indexCount = in.readVInt();
             this.primaryShardCount = in.readVInt();
             this.totalPrimaryByteCount = in.readVLong();
@@ -196,7 +178,7 @@ public final class VersionStats implements ToXContentFragment, Writeable {
 
         @Override
         public void writeTo(StreamOutput out) throws IOException {
-            Version.writeVersion(this.version, out);
+            IndexVersion.writeVersion(this.version, out);
             out.writeVInt(this.indexCount);
             out.writeVInt(this.primaryShardCount);
             out.writeVLong(this.totalPrimaryByteCount);

+ 3 - 3
server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java

@@ -1035,12 +1035,12 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
     }
 
     /**
-     * Return the {@link Version} on which this index has been created. This
+     * Return the {@link IndexVersion} on which this index has been created. This
      * information is typically useful for backward compatibility.
      * To check index compatibility (e.g. N-1 checks), use {@link #getCompatibilityVersion()} instead.
      */
-    public Version getCreationVersion() {
-        return indexCreatedVersion.toVersion();
+    public IndexVersion getCreationVersion() {
+        return indexCreatedVersion;
     }
 
     /**

+ 2 - 0
server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java

@@ -29,6 +29,7 @@ import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexNotFoundException;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.indices.IndexClosedException;
 import org.elasticsearch.indices.InvalidIndexNameException;
 import org.elasticsearch.indices.SystemIndices;
@@ -60,6 +61,7 @@ public class IndexNameExpressionResolver {
 
     public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds";
     public static final Version SYSTEM_INDEX_ENFORCEMENT_VERSION = Version.V_8_0_0;
+    public static final IndexVersion SYSTEM_INDEX_ENFORCEMENT_INDEX_VERSION = IndexVersion.V_8_0_0;
 
     private final ThreadContext threadContext;
     private final SystemIndices systemIndices;

+ 1 - 1
server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java

@@ -2567,7 +2567,7 @@ public class Metadata implements Iterable<IndexMetadata>, Diffable<Metadata>, Ch
             if (isNonEmpty(groupedBySystemStatus.get(false)) && isNonEmpty(groupedBySystemStatus.get(true))) {
                 final List<String> newVersionSystemIndices = groupedBySystemStatus.get(true)
                     .stream()
-                    .filter(i -> i.getCreationVersion().onOrAfter(IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_VERSION))
+                    .filter(i -> i.getCreationVersion().onOrAfter(IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_INDEX_VERSION))
                     .map(i -> i.getIndex().getName())
                     .sorted() // reliable error message for testing
                     .toList();

+ 1 - 1
server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java

@@ -1574,7 +1574,7 @@ public class MetadataCreateIndexService {
             builder.put(sourceMetadata.getSettings().filter(sourceSettingsPredicate));
         }
 
-        indexSettingsBuilder.put(IndexMetadata.SETTING_VERSION_CREATED, sourceMetadata.getCreationVersion())
+        indexSettingsBuilder.put(IndexMetadata.SETTING_VERSION_CREATED, sourceMetadata.getCreationVersion().id())
             .put(builder.build())
             .put(IndexMetadata.SETTING_ROUTING_PARTITION_SIZE, sourceMetadata.getRoutingPartitionSize())
             .put(IndexMetadata.INDEX_RESIZE_SOURCE_NAME.getKey(), resizeSourceIndex.getName())

+ 2 - 2
server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java

@@ -11,7 +11,6 @@ package org.elasticsearch.cluster.routing.allocation.allocator;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.util.ArrayUtil;
-import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.MetadataIndexStateService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -27,6 +26,7 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.gateway.PriorityComparator;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.threadpool.ThreadPool;
 
@@ -145,7 +145,7 @@ public class DesiredBalanceReconciler {
 
             final var shardCounts = allocation.metadata().stream().filter(indexMetadata ->
             // skip any pre-7.2 closed indices which have no routing table entries at all
-            indexMetadata.getCreationVersion().onOrAfter(Version.V_7_2_0)
+            indexMetadata.getCreationVersion().onOrAfter(IndexVersion.V_7_2_0)
                 || indexMetadata.getState() == IndexMetadata.State.OPEN
                 || MetadataIndexStateService.isIndexVerifiedBeforeClosed(indexMetadata))
                 .flatMap(

+ 1 - 1
server/src/main/java/org/elasticsearch/index/shard/ShardSplittingQuery.java

@@ -59,7 +59,7 @@ final class ShardSplittingQuery extends Query {
         this.indexMetadata = indexMetadata;
         this.indexRouting = IndexRouting.fromIndexMetadata(indexMetadata);
         this.shardId = shardId;
-        this.nestedParentBitSetProducer = hasNested ? newParentDocBitSetProducer(indexMetadata.getCreationVersion().indexVersion) : null;
+        this.nestedParentBitSetProducer = hasNested ? newParentDocBitSetProducer(indexMetadata.getCreationVersion()) : null;
     }
 
     @Override

+ 2 - 2
server/src/main/java/org/elasticsearch/snapshots/RestoreService.java

@@ -1283,7 +1283,7 @@ public class RestoreService implements ClusterStateApplier {
                 final IndexId index = indexEntry.getValue();
                 final IndexMetadata originalIndexMetadata = metadata.index(index.getName());
                 repositoriesService.getPreRestoreVersionChecks()
-                    .forEach(check -> check.accept(snapshot, originalIndexMetadata.getCreationVersion().indexVersion));
+                    .forEach(check -> check.accept(snapshot, originalIndexMetadata.getCreationVersion()));
                 IndexMetadata snapshotIndexMetadata = updateIndexSettings(
                     snapshot,
                     originalIndexMetadata,
@@ -1594,7 +1594,7 @@ public class RestoreService implements ClusterStateApplier {
         ClusterState clusterState,
         IndicesService indicesService
     ) {
-        if (snapshotIndexMetadata.getCreationVersion().before(Version.fromString("5.0.0"))) {
+        if (snapshotIndexMetadata.getCreationVersion().before(IndexVersion.fromId(5000099))) {
             throw new IllegalArgumentException("can't restore an index created before version 5.0.0");
         }
         IndexMetadata.Builder convertedIndexMetadataBuilder = IndexMetadata.builder(snapshotIndexMetadata);

+ 2 - 2
server/src/main/java/org/elasticsearch/upgrades/SystemIndexMigrator.java

@@ -55,7 +55,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
-import static org.elasticsearch.action.admin.cluster.migration.TransportGetFeatureUpgradeStatusAction.NO_UPGRADE_REQUIRED_VERSION;
+import static org.elasticsearch.action.admin.cluster.migration.TransportGetFeatureUpgradeStatusAction.NO_UPGRADE_REQUIRED_INDEX_VERSION;
 import static org.elasticsearch.cluster.metadata.IndexMetadata.State.CLOSE;
 import static org.elasticsearch.core.Strings.format;
 
@@ -359,7 +359,7 @@ public class SystemIndexMigrator extends AllocatedPersistentTask {
         if (indexMetadata == null) {
             return false;
         }
-        return indexMetadata.isSystem() && indexMetadata.getCreationVersion().before(NO_UPGRADE_REQUIRED_VERSION);
+        return indexMetadata.isSystem() && indexMetadata.getCreationVersion().before(NO_UPGRADE_REQUIRED_INDEX_VERSION);
     }
 
     private void migrateSingleIndex(ClusterState clusterState, Consumer<BulkByScrollResponse> listener) {

+ 3 - 3
server/src/test/java/org/elasticsearch/action/admin/cluster/migration/GetFeatureUpgradeStatusResponseTests.java

@@ -8,8 +8,8 @@
 
 package org.elasticsearch.action.admin.cluster.migration;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.test.AbstractWireSerializingTestCase;
 
 import java.util.Collections;
@@ -89,7 +89,7 @@ public class GetFeatureUpgradeStatusResponseTests extends AbstractWireSerializin
     private static GetFeatureUpgradeStatusResponse.FeatureUpgradeStatus createFeatureStatus() {
         return new GetFeatureUpgradeStatusResponse.FeatureUpgradeStatus(
             randomAlphaOfLengthBetween(3, 20),
-            randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()),
+            randomFrom(IndexVersion.current(), IndexVersion.MINIMUM_COMPATIBLE),
             randomFrom(org.elasticsearch.action.admin.cluster.migration.GetFeatureUpgradeStatusResponse.UpgradeStatus.values()),
             randomList(4, GetFeatureUpgradeStatusResponseTests::getIndexInfo)
         );
@@ -98,7 +98,7 @@ public class GetFeatureUpgradeStatusResponseTests extends AbstractWireSerializin
     private static GetFeatureUpgradeStatusResponse.IndexInfo getIndexInfo() {
         return new GetFeatureUpgradeStatusResponse.IndexInfo(
             randomAlphaOfLengthBetween(3, 20),
-            randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()),
+            randomFrom(IndexVersion.current(), IndexVersion.MINIMUM_COMPATIBLE),
             null
         );
     }

+ 8 - 6
server/src/test/java/org/elasticsearch/action/admin/cluster/migration/TransportGetFeatureUpgradeStatusActionTests.java

@@ -13,6 +13,7 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.SystemIndexDescriptorUtils;
 import org.elasticsearch.indices.SystemIndices;
@@ -24,13 +25,14 @@ import java.util.Map;
 
 import static org.elasticsearch.action.admin.cluster.migration.GetFeatureUpgradeStatusResponse.UpgradeStatus.MIGRATION_NEEDED;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
 
 public class TransportGetFeatureUpgradeStatusActionTests extends ESTestCase {
 
     public static String TEST_SYSTEM_INDEX_PATTERN = ".test*";
+    private static final IndexVersion TEST_OLD_VERSION = IndexVersion.fromId(6000099);
     private static final ClusterState CLUSTER_STATE = getClusterState();
     private static final SystemIndices.Feature FEATURE = getFeature();
-    private static final Version TEST_OLD_VERSION = Version.fromString("6.0.0");
 
     public void testGetFeatureStatus() {
         GetFeatureUpgradeStatusResponse.FeatureUpgradeStatus status = TransportGetFeatureUpgradeStatusAction.getFeatureUpgradeStatus(
@@ -41,7 +43,7 @@ public class TransportGetFeatureUpgradeStatusActionTests extends ESTestCase {
         assertThat(status.getUpgradeStatus(), equalTo(MIGRATION_NEEDED));
         assertThat(status.getFeatureName(), equalTo("test-feature"));
         assertThat(status.getMinimumIndexVersion(), equalTo(TEST_OLD_VERSION));
-        assertThat(status.getIndexVersions().size(), equalTo(2)); // additional testing below
+        assertThat(status.getIndexVersions(), hasSize(2)); // additional testing below
     }
 
     public void testGetIndexInfos() {
@@ -50,11 +52,11 @@ public class TransportGetFeatureUpgradeStatusActionTests extends ESTestCase {
             FEATURE
         );
 
-        assertThat(versions.size(), equalTo(2));
+        assertThat(versions, hasSize(2));
 
         {
             GetFeatureUpgradeStatusResponse.IndexInfo version = versions.get(0);
-            assertThat(version.getVersion(), equalTo(Version.CURRENT));
+            assertThat(version.getVersion(), equalTo(IndexVersion.current()));
             assertThat(version.getIndexName(), equalTo(".test-index-1"));
         }
         {
@@ -77,7 +79,7 @@ public class TransportGetFeatureUpgradeStatusActionTests extends ESTestCase {
 
     private static ClusterState getClusterState() {
         IndexMetadata indexMetadata1 = IndexMetadata.builder(".test-index-1")
-            .settings(Settings.builder().put("index.version.created", Version.CURRENT).build())
+            .settings(Settings.builder().put("index.version.created", IndexVersion.current().id()).build())
             .numberOfShards(1)
             .numberOfReplicas(0)
             .build();
@@ -86,7 +88,7 @@ public class TransportGetFeatureUpgradeStatusActionTests extends ESTestCase {
         assert Version.CURRENT.major < 9;
 
         IndexMetadata indexMetadata2 = IndexMetadata.builder(".test-index-2")
-            .settings(Settings.builder().put("index.version.created", Version.fromString("6.0.0")).build())
+            .settings(Settings.builder().put("index.version.created", TEST_OLD_VERSION.id()).build())
             .numberOfShards(1)
             .numberOfReplicas(0)
             .build();

+ 8 - 8
server/src/test/java/org/elasticsearch/action/admin/cluster/stats/VersionStatsTests.java

@@ -20,6 +20,7 @@ import org.elasticsearch.cluster.routing.RecoverySource;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.UnassignedInfo;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.shard.IndexShard;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardPath;
@@ -28,7 +29,6 @@ import org.elasticsearch.test.AbstractWireSerializingTestCase;
 
 import java.nio.file.Path;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -54,7 +54,7 @@ public class VersionStatsTests extends AbstractWireSerializingTestCase<VersionSt
         return new VersionStats(instance.versionStats().stream().map(svs -> {
             return switch (randomIntBetween(1, 4)) {
                 case 1 -> new VersionStats.SingleVersionStats(
-                    Version.V_7_3_0,
+                    IndexVersion.V_7_3_0,
                     svs.indexCount,
                     svs.primaryShardCount,
                     svs.totalPrimaryByteCount
@@ -93,8 +93,8 @@ public class VersionStatsTests extends AbstractWireSerializingTestCase<VersionSt
             .build();
         stats = VersionStats.of(metadata, Collections.emptyList());
         assertThat(stats.versionStats().size(), equalTo(2));
-        VersionStats.SingleVersionStats s1 = new VersionStats.SingleVersionStats(Version.CURRENT, 2, 7, 0);
-        VersionStats.SingleVersionStats s2 = new VersionStats.SingleVersionStats(Version.V_7_0_0, 1, 2, 0);
+        VersionStats.SingleVersionStats s1 = new VersionStats.SingleVersionStats(IndexVersion.current(), 2, 7, 0);
+        VersionStats.SingleVersionStats s2 = new VersionStats.SingleVersionStats(IndexVersion.V_7_0_0, 1, 2, 0);
         assertThat(stats.versionStats(), containsInAnyOrder(s1, s2));
 
         ShardId shardId = new ShardId("bar", "uuid", 0);
@@ -132,8 +132,8 @@ public class VersionStatsTests extends AbstractWireSerializingTestCase<VersionSt
 
         stats = VersionStats.of(metadata, Collections.singletonList(nodeResponse));
         assertThat(stats.versionStats().size(), equalTo(2));
-        s1 = new VersionStats.SingleVersionStats(Version.CURRENT, 2, 7, 100);
-        s2 = new VersionStats.SingleVersionStats(Version.V_7_0_0, 1, 2, 0);
+        s1 = new VersionStats.SingleVersionStats(IndexVersion.current(), 2, 7, 100);
+        s2 = new VersionStats.SingleVersionStats(IndexVersion.V_7_0_0, 1, 2, 0);
         assertThat(stats.versionStats(), containsInAnyOrder(s1, s2));
     }
 
@@ -142,9 +142,9 @@ public class VersionStatsTests extends AbstractWireSerializingTestCase<VersionSt
     }
 
     public static VersionStats randomInstance() {
-        List<Version> versions = Arrays.asList(Version.CURRENT, Version.V_7_0_0, Version.V_7_1_0, Version.V_7_2_0);
+        List<IndexVersion> versions = List.of(IndexVersion.current(), IndexVersion.V_7_0_0, IndexVersion.V_7_1_0, IndexVersion.V_7_2_0);
         List<VersionStats.SingleVersionStats> stats = new ArrayList<>();
-        for (Version v : versions) {
+        for (IndexVersion v : versions) {
             VersionStats.SingleVersionStats s = new VersionStats.SingleVersionStats(
                 v,
                 randomIntBetween(10, 20),