Migrate IndexMetadata.getCreationVersion to IndexVersion (#97139)

This commit is contained in:
Simon Cooper 2023-06-29 08:38:50 +01:00 committed by GitHub
parent 94df6f2a74
commit 602ccd8d60
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
17 changed files with 89 additions and 100 deletions

View file

@ -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

View file

@ -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 {

View file

@ -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));

View file

@ -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);

View file

@ -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;

View file

@ -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);

View file

@ -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;
}
/**

View file

@ -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;

View file

@ -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();

View file

@ -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())

View file

@ -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(

View file

@ -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

View file

@ -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);

View file

@ -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) {

View file

@ -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
);
}

View file

@ -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();

View file

@ -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),