diff --git a/build.gradle b/build.gradle index c950db7893039..777f83797fb3f 100644 --- a/build.gradle +++ b/build.gradle @@ -125,9 +125,9 @@ tasks.register("verifyVersions") { * after the backport of the backcompat code is complete. */ -boolean bwc_tests_enabled = false +boolean bwc_tests_enabled = true // place a PR link here when committing bwc changes: -String bwc_tests_disabled_issue = "https://github.com/elastic/elasticsearch/pull/82689" +String bwc_tests_disabled_issue = "" /* * FIPS 140-2 behavior was fixed in 7.11.0. Before that there is no way to run elasticsearch in a * JVM that is properly configured to be in fips mode with BCFIPS. For now we need to disable diff --git a/docs/changelog/82689.yaml b/docs/changelog/82689.yaml new file mode 100644 index 0000000000000..9ce92281675d1 --- /dev/null +++ b/docs/changelog/82689.yaml @@ -0,0 +1,6 @@ +pr: 82689 +summary: Enhancement/prevent 6.8 indices upgrade +area: Infra/Core +type: enhancement +issues: + - 81326 diff --git a/server/src/internalClusterTest/java/org/elasticsearch/bwcompat/RecoveryWithUnsupportedIndicesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/bwcompat/RecoveryWithUnsupportedIndicesIT.java index 9854c494be472..6ed240d935c0d 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/bwcompat/RecoveryWithUnsupportedIndicesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/bwcompat/RecoveryWithUnsupportedIndicesIT.java @@ -32,8 +32,7 @@ public class RecoveryWithUnsupportedIndicesIT extends ESIntegTestCase { /** * Return settings that could be used to start a node that has the given zipped home directory. */ - private Settings prepareBackwardsDataDir(Path backwardsIndex) throws IOException { - Path indexDir = createTempDir(); + private Settings prepareBackwardsDataDir(Path indexDir, Path backwardsIndex) throws IOException { Path dataDir = indexDir.resolve("data"); try (InputStream stream = Files.newInputStream(backwardsIndex)) { TestUtil.unzip(stream, indexDir); @@ -73,8 +72,9 @@ private Settings prepareBackwardsDataDir(Path backwardsIndex) throws IOException public void testUpgradeStartClusterOn_2_4_5() throws Exception { String indexName = "unsupported-2.4.5"; + Path indexDir = createTempDir(); logger.info("Checking static index {}", indexName); - Settings nodeSettings = prepareBackwardsDataDir(getDataPath("/indices/bwc").resolve(indexName + ".zip")); + Settings nodeSettings = prepareBackwardsDataDir(indexDir, getDataPath("/indices/bwc").resolve(indexName + ".zip")); assertThat( ExceptionsHelper.unwrap( expectThrows(Exception.class, () -> internalCluster().startNode(nodeSettings)), diff --git a/server/src/internalClusterTest/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/server/src/internalClusterTest/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index 955dac12e7514..d1ed3de951197 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -608,7 +608,7 @@ public void testHalfDeletedIndexImport() throws Exception { .putCustom(IndexGraveyard.TYPE, IndexGraveyard.builder().addTombstone(metadata.index("test").getIndex()).build()) .build() ); - NodeMetadata.FORMAT.writeAndCleanup(new NodeMetadata(nodeId, Version.CURRENT), paths); + NodeMetadata.FORMAT.writeAndCleanup(new NodeMetadata(nodeId, Version.CURRENT, metadata.oldestIndexVersion()), paths); }); ensureGreen(); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java index e2abffd4380c3..fa1ba64c2fe9f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/Metadata.java @@ -215,6 +215,8 @@ default boolean isRestorable() { private SortedMap indicesLookup; private final Map mappingsByHash; + private final Version oldestIndexVersion; + private Metadata( String clusterUUID, boolean clusterUUIDCommitted, @@ -236,7 +238,8 @@ private Metadata( String[] allClosedIndices, String[] visibleClosedIndices, SortedMap indicesLookup, - Map mappingsByHash + Map mappingsByHash, + Version oldestIndexVersion ) { this.clusterUUID = clusterUUID; this.clusterUUIDCommitted = clusterUUIDCommitted; @@ -259,6 +262,7 @@ private Metadata( this.visibleClosedIndices = visibleClosedIndices; this.indicesLookup = indicesLookup; this.mappingsByHash = mappingsByHash; + this.oldestIndexVersion = oldestIndexVersion; } public Metadata withIncrementedVersion() { @@ -283,7 +287,8 @@ public Metadata withIncrementedVersion() { allClosedIndices, visibleClosedIndices, indicesLookup, - mappingsByHash + mappingsByHash, + oldestIndexVersion ); } @@ -326,6 +331,10 @@ public CoordinationMetadata coordinationMetadata() { return this.coordinationMetadata; } + public Version oldestIndexVersion() { + return this.oldestIndexVersion; + } + public boolean hasAlias(String alias) { IndexAbstraction indexAbstraction = getIndicesLookup().get(alias); if (indexAbstraction != null) { @@ -1062,6 +1071,7 @@ public static Metadata readFrom(StreamInput in) throws IOException { Custom customIndexMetadata = in.readNamedWriteable(Custom.class); builder.putCustom(customIndexMetadata.getWriteableName(), customIndexMetadata); } + return builder.build(); } @@ -1596,6 +1606,9 @@ public Metadata build(boolean builtIndicesLookupEagerly) { final List visibleClosedIndices = new ArrayList<>(); final Set allAliases = new HashSet<>(); final ImmutableOpenMap indicesMap = indices.build(); + + int oldestIndexVersionId = Version.CURRENT.id; + for (IndexMetadata indexMetadata : indicesMap.values()) { final String name = indexMetadata.getIndex().getName(); boolean added = allIndices.add(name); @@ -1616,6 +1629,7 @@ public Metadata build(boolean builtIndicesLookupEagerly) { } } indexMetadata.getAliases().keysIt().forEachRemaining(allAliases::add); + oldestIndexVersionId = Math.min(oldestIndexVersionId, indexMetadata.getCreationVersion().id); } final ArrayList duplicates = new ArrayList<>(); @@ -1738,7 +1752,8 @@ public Metadata build(boolean builtIndicesLookupEagerly) { allClosedIndicesArray, visibleClosedIndicesArray, indicesLookup, - Collections.unmodifiableMap(mappingsByHash) + Collections.unmodifiableMap(mappingsByHash), + Version.fromId(oldestIndexVersionId) ); } @@ -1888,6 +1903,7 @@ public static void toXContent(Metadata metadata, XContentBuilder builder, ToXCon builder.endObject(); } } + builder.endObject(); } diff --git a/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 3cd696aefba13..cfc3dd0707199 100644 --- a/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -40,6 +40,7 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.gateway.CorruptStateException; import org.elasticsearch.gateway.MetadataStateFormat; import org.elasticsearch.gateway.PersistedClusterStateService; import org.elasticsearch.index.Index; @@ -248,6 +249,7 @@ public void close() { /** * Setup the environment. * @param settings settings from elasticsearch.yml + * @param environment global environment */ public NodeEnvironment(Settings settings, Environment environment) throws IOException { boolean success = false; @@ -391,6 +393,9 @@ private static boolean upgradeLegacyNodeFolders(Logger logger, Settings settings // move contents from legacy path to new path assert nodeLock.getNodePaths().length == legacyNodeLock.getNodePaths().length; try { + // first check if we are upgrading from an index compatible version + checkForIndexCompatibility(logger, legacyNodeLock.getNodePaths()); + final List> upgradeActions = new ArrayList<>(); for (int i = 0; i < legacyNodeLock.getNodePaths().length; i++) { final NodePath legacyNodePath = legacyNodeLock.getNodePaths()[i]; @@ -470,6 +475,46 @@ private static boolean upgradeLegacyNodeFolders(Logger logger, Settings settings return true; } + /** + * Checks to see if we can upgrade to this version based on the existing index state. Upgrading + * from older versions can cause irreversible changes if allowed. + * @param logger + * @param nodePaths + * @throws IOException + */ + static void checkForIndexCompatibility(Logger logger, NodePath... nodePaths) throws IOException { + final Path[] paths = Arrays.stream(nodePaths).map(np -> np.path).toArray(Path[]::new); + NodeMetadata metadata = PersistedClusterStateService.nodeMetadata(paths); + + // We are upgrading the cluster, but we didn't find any previous metadata. Corrupted state or incompatible version. + if (metadata == null) { + throw new CorruptStateException( + "Format version is not supported. Upgrading to [" + + Version.CURRENT + + "] is only supported from version [" + + Version.CURRENT.minimumCompatibilityVersion() + + "]." + ); + } + + metadata.verifyUpgradeToCurrentVersion(); + + logger.info("oldest index version recorded in NodeMetadata {}", metadata.oldestIndexVersion()); + + if (metadata.oldestIndexVersion().before(Version.CURRENT.minimumIndexCompatibilityVersion())) { + throw new IllegalStateException( + "cannot upgrade node because incompatible indices created with version [" + + metadata.oldestIndexVersion() + + "] exist, while the minimum compatible index version is [" + + Version.CURRENT.minimumIndexCompatibilityVersion() + + "]. " + + "Upgrade your older indices by reindexing them in version [" + + Version.CURRENT.minimumCompatibilityVersion() + + "] first." + ); + } + } + private void maybeLogPathDetails() throws IOException { // We do some I/O in here, so skip this if DEBUG/INFO are not enabled: @@ -551,12 +596,15 @@ private static NodeMetadata loadNodeMetadata(Settings settings, Logger logger, N final NodeMetadata legacyMetadata = NodeMetadata.FORMAT.loadLatestState(logger, NamedXContentRegistry.EMPTY, paths); if (legacyMetadata == null) { assert nodeIds.isEmpty() : nodeIds; - metadata = new NodeMetadata(generateNodeId(settings), Version.CURRENT); + // 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, Version.CURRENT); } else { assert nodeIds.equals(Collections.singleton(legacyMetadata.nodeId())) : nodeIds + " doesn't match " + legacyMetadata; metadata = legacyMetadata; } } + metadata = metadata.upgradeToCurrentVersion(); assert metadata.nodeVersion().equals(Version.CURRENT) : metadata.nodeVersion() + " != " + Version.CURRENT; diff --git a/server/src/main/java/org/elasticsearch/env/NodeMetadata.java b/server/src/main/java/org/elasticsearch/env/NodeMetadata.java index 0817985a547cb..e77e9b0709d7e 100644 --- a/server/src/main/java/org/elasticsearch/env/NodeMetadata.java +++ b/server/src/main/java/org/elasticsearch/env/NodeMetadata.java @@ -28,6 +28,7 @@ public final class NodeMetadata { static final String NODE_ID_KEY = "node_id"; static final String NODE_VERSION_KEY = "node_version"; + static final String OLDEST_INDEX_VERSION_KEY = "oldest_index_version"; private final String nodeId; @@ -35,14 +36,22 @@ public final class NodeMetadata { private final Version previousNodeVersion; - private NodeMetadata(final String nodeId, final Version nodeVersion, final Version previousNodeVersion) { + private final Version oldestIndexVersion; + + private NodeMetadata( + final String nodeId, + final Version nodeVersion, + final Version previousNodeVersion, + final Version oldestIndexVersion + ) { this.nodeId = Objects.requireNonNull(nodeId); this.nodeVersion = Objects.requireNonNull(nodeVersion); this.previousNodeVersion = Objects.requireNonNull(previousNodeVersion); + this.oldestIndexVersion = Objects.requireNonNull(oldestIndexVersion); } - public NodeMetadata(final String nodeId, final Version nodeVersion) { - this(nodeId, nodeVersion, nodeVersion); + public NodeMetadata(final String nodeId, final Version nodeVersion, final Version oldestIndexVersion) { + this(nodeId, nodeVersion, nodeVersion, oldestIndexVersion); } @Override @@ -52,12 +61,13 @@ public boolean equals(Object o) { NodeMetadata that = (NodeMetadata) o; return nodeId.equals(that.nodeId) && nodeVersion.equals(that.nodeVersion) + && oldestIndexVersion.equals(that.oldestIndexVersion) && Objects.equals(previousNodeVersion, that.previousNodeVersion); } @Override public int hashCode() { - return Objects.hash(nodeId, nodeVersion, previousNodeVersion); + return Objects.hash(nodeId, nodeVersion, previousNodeVersion, oldestIndexVersion); } @Override @@ -70,6 +80,8 @@ public String toString() { + nodeVersion + ", previousNodeVersion=" + previousNodeVersion + + ", oldestIndexVersion=" + + oldestIndexVersion + '}'; } @@ -91,7 +103,11 @@ public Version previousNodeVersion() { return previousNodeVersion; } - public NodeMetadata upgradeToCurrentVersion() { + public Version oldestIndexVersion() { + return oldestIndexVersion; + } + + public void verifyUpgradeToCurrentVersion() { assert (nodeVersion.equals(Version.V_EMPTY) == false) || (Version.CURRENT.major <= Version.V_7_0_0.major + 1) : "version is required in the node metadata from v9 onwards"; @@ -113,14 +129,19 @@ public NodeMetadata upgradeToCurrentVersion() { "cannot downgrade a node from version [" + nodeVersion + "] to version [" + Version.CURRENT + "]" ); } + } - return nodeVersion.equals(Version.CURRENT) ? this : new NodeMetadata(nodeId, Version.CURRENT, nodeVersion); + public NodeMetadata upgradeToCurrentVersion() { + verifyUpgradeToCurrentVersion(); + + return nodeVersion.equals(Version.CURRENT) ? this : new NodeMetadata(nodeId, Version.CURRENT, nodeVersion, oldestIndexVersion); } private static class Builder { String nodeId; Version nodeVersion; Version previousNodeVersion; + Version oldestIndexVersion; public void setNodeId(String nodeId) { this.nodeId = nodeId; @@ -134,8 +155,13 @@ public void setPreviousNodeVersionId(int previousNodeVersionId) { this.previousNodeVersion = Version.fromId(previousNodeVersionId); } + public void setOldestIndexVersion(int oldestIndexVersion) { + this.oldestIndexVersion = Version.fromId(oldestIndexVersion); + } + public NodeMetadata build() { final Version nodeVersion; + final Version oldestIndexVersion; if (this.nodeVersion == null) { assert Version.CURRENT.major <= Version.V_7_0_0.major + 1 : "version is required in the node metadata from v9 onwards"; nodeVersion = Version.V_EMPTY; @@ -145,8 +171,13 @@ public NodeMetadata build() { if (this.previousNodeVersion == null) { previousNodeVersion = nodeVersion; } + if (this.oldestIndexVersion == null) { + oldestIndexVersion = Version.V_EMPTY; + } else { + oldestIndexVersion = this.oldestIndexVersion; + } - return new NodeMetadata(nodeId, nodeVersion, previousNodeVersion); + return new NodeMetadata(nodeId, nodeVersion, previousNodeVersion, oldestIndexVersion); } } @@ -163,6 +194,7 @@ static class NodeMetadataStateFormat extends MetadataStateFormat { objectParser = new ObjectParser<>("node_meta_data", ignoreUnknownFields, Builder::new); objectParser.declareString(Builder::setNodeId, new ParseField(NODE_ID_KEY)); objectParser.declareInt(Builder::setNodeVersionId, new ParseField(NODE_VERSION_KEY)); + objectParser.declareInt(Builder::setOldestIndexVersion, new ParseField(OLDEST_INDEX_VERSION_KEY)); } @Override @@ -176,6 +208,7 @@ protected XContentBuilder newXContentBuilder(XContentType type, OutputStream str public void toXContent(XContentBuilder builder, NodeMetadata nodeMetadata) throws IOException { builder.field(NODE_ID_KEY, nodeMetadata.nodeId); builder.field(NODE_VERSION_KEY, nodeMetadata.nodeVersion.id); + builder.field(OLDEST_INDEX_VERSION_KEY, nodeMetadata.oldestIndexVersion.id); } @Override diff --git a/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index 5a4d81e3321df..98b00c92288d3 100644 --- a/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -143,7 +143,11 @@ public void start( } // write legacy node metadata to prevent accidental downgrades from spawning empty cluster state NodeMetadata.FORMAT.writeAndCleanup( - new NodeMetadata(persistedClusterStateService.getNodeId(), Version.CURRENT), + new NodeMetadata( + persistedClusterStateService.getNodeId(), + Version.CURRENT, + clusterState.metadata().oldestIndexVersion() + ), persistedClusterStateService.getDataPaths() ); success = true; @@ -177,7 +181,11 @@ public void start( metaStateService.deleteAll(); // write legacy node metadata to prevent downgrades from spawning empty cluster state NodeMetadata.FORMAT.writeAndCleanup( - new NodeMetadata(persistedClusterStateService.getNodeId(), Version.CURRENT), + new NodeMetadata( + persistedClusterStateService.getNodeId(), + Version.CURRENT, + clusterState.metadata().oldestIndexVersion() + ), persistedClusterStateService.getDataPaths() ); } catch (IOException e) { @@ -468,7 +476,11 @@ public void setCurrentTerm(long currentTerm) { getWriterSafe().writeFullStateAndCommit(currentTerm, lastAcceptedState); } else { writeNextStateFully = true; // in case of failure; this flag is cleared on success - getWriterSafe().writeIncrementalTermUpdateAndCommit(currentTerm, lastAcceptedState.version()); + getWriterSafe().writeIncrementalTermUpdateAndCommit( + currentTerm, + lastAcceptedState.version(), + lastAcceptedState.metadata().oldestIndexVersion() + ); } } catch (IOException e) { throw new ElasticsearchException(e); diff --git a/server/src/main/java/org/elasticsearch/gateway/PersistedClusterStateService.java b/server/src/main/java/org/elasticsearch/gateway/PersistedClusterStateService.java index 5bda581a46132..d6b7d705c9b97 100644 --- a/server/src/main/java/org/elasticsearch/gateway/PersistedClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/gateway/PersistedClusterStateService.java @@ -122,12 +122,13 @@ public class PersistedClusterStateService { private static final String LAST_ACCEPTED_VERSION_KEY = "last_accepted_version"; private static final String NODE_ID_KEY = "node_id"; private static final String NODE_VERSION_KEY = "node_version"; + private static final String OLDEST_INDEX_VERSION_KEY = "oldest_index_version"; private static final String TYPE_FIELD_NAME = "type"; private static final String DATA_FIELD_NAME = "data"; private static final String GLOBAL_TYPE_NAME = "global"; private static final String INDEX_TYPE_NAME = "index"; private static final String INDEX_UUID_FIELD_NAME = "index_uuid"; - private static final int COMMIT_DATA_SIZE = 4; + private static final int COMMIT_DATA_SIZE = 5; private static final MergePolicy NO_MERGE_POLICY = noMergePolicy(); private static final MergePolicy DEFAULT_MERGE_POLICY = defaultMergePolicy(); @@ -285,6 +286,7 @@ public boolean empty() { public static NodeMetadata nodeMetadata(Path... dataPaths) throws IOException { String nodeId = null; Version version = null; + Version oldestIndexVersion = Version.V_EMPTY; for (final Path dataPath : dataPaths) { final Path indexPath = dataPath.resolve(METADATA_DIRECTORY_NAME); if (Files.exists(indexPath)) { @@ -301,6 +303,11 @@ public static NodeMetadata nodeMetadata(Path... dataPaths) throws IOException { } else if (nodeId == null) { nodeId = thisNodeId; version = Version.fromId(Integer.parseInt(userData.get(NODE_VERSION_KEY))); + if (userData.containsKey(OLDEST_INDEX_VERSION_KEY)) { + oldestIndexVersion = Version.fromId(Integer.parseInt(userData.get(OLDEST_INDEX_VERSION_KEY))); + } else { + oldestIndexVersion = Version.V_EMPTY; + } } } catch (IndexNotFoundException e) { logger.debug(new ParameterizedMessage("no on-disk state at {}", indexPath), e); @@ -310,7 +317,7 @@ public static NodeMetadata nodeMetadata(Path... dataPaths) throws IOException { if (nodeId == null) { return null; } - return new NodeMetadata(nodeId, version); + return new NodeMetadata(nodeId, version, oldestIndexVersion); } /** @@ -615,7 +622,7 @@ void startWrite() { indexWriter.getConfig().setMergePolicy(NO_MERGE_POLICY); } - void prepareCommit(String nodeId, long currentTerm, long lastAcceptedVersion) throws IOException { + void prepareCommit(String nodeId, long currentTerm, long lastAcceptedVersion, Version oldestIndexVersion) throws IOException { indexWriter.getConfig().setMergePolicy(DEFAULT_MERGE_POLICY); indexWriter.maybeMerge(); @@ -623,6 +630,7 @@ void prepareCommit(String nodeId, long currentTerm, long lastAcceptedVersion) th commitData.put(CURRENT_TERM_KEY, Long.toString(currentTerm)); commitData.put(LAST_ACCEPTED_VERSION_KEY, Long.toString(lastAcceptedVersion)); commitData.put(NODE_VERSION_KEY, Integer.toString(Version.CURRENT.id)); + commitData.put(OLDEST_INDEX_VERSION_KEY, Integer.toString(oldestIndexVersion.id)); commitData.put(NODE_ID_KEY, nodeId); indexWriter.setLiveCommitData(commitData.entrySet()); indexWriter.prepareCommit(); @@ -702,7 +710,7 @@ public void writeFullStateAndCommit(long currentTerm, ClusterState clusterState) } final WriterStats stats = overwriteMetadata(clusterState.metadata()); - commit(currentTerm, clusterState.version()); + commit(currentTerm, clusterState.version(), clusterState.metadata().oldestIndexVersion()); fullStateWritten = true; final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; final TimeValue finalSlowWriteLoggingThreshold = slowWriteLoggingThresholdSupplier.get(); @@ -742,7 +750,7 @@ void writeIncrementalStateAndCommit(long currentTerm, ClusterState previousClust } final WriterStats stats = updateMetadata(previousClusterState.metadata(), clusterState.metadata()); - commit(currentTerm, clusterState.version()); + commit(currentTerm, clusterState.version(), clusterState.metadata().oldestIndexVersion()); final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; final TimeValue finalSlowWriteLoggingThreshold = slowWriteLoggingThresholdSupplier.get(); if (durationMillis >= finalSlowWriteLoggingThreshold.getMillis()) { @@ -897,17 +905,18 @@ private DocumentBuffer allocateBuffer() { return new DocumentBuffer(documentBufferUsed + extraSpace, bigArrays); } - public void writeIncrementalTermUpdateAndCommit(long currentTerm, long lastAcceptedVersion) throws IOException { + public void writeIncrementalTermUpdateAndCommit(long currentTerm, long lastAcceptedVersion, Version oldestIndexVersion) + throws IOException { ensureOpen(); ensureFullStateWritten(); - commit(currentTerm, lastAcceptedVersion); + commit(currentTerm, lastAcceptedVersion, oldestIndexVersion); } - void commit(long currentTerm, long lastAcceptedVersion) throws IOException { + void commit(long currentTerm, long lastAcceptedVersion, Version oldestIndexVersion) throws IOException { ensureOpen(); try { for (MetadataIndexWriter metadataIndexWriter : metadataIndexWriters) { - metadataIndexWriter.prepareCommit(nodeId, currentTerm, lastAcceptedVersion); + metadataIndexWriter.prepareCommit(nodeId, currentTerm, lastAcceptedVersion, oldestIndexVersion); } } catch (Exception e) { try { diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataTests.java index 60cf1bf9a15fd..e2a9043307432 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataTests.java @@ -883,6 +883,42 @@ public void testFindMappingsWithFilters() throws IOException { } } + public void testOldestIndexComputation() { + Metadata metadata = buildIndicesWithVersions( + new Version[] { Version.V_7_0_0, Version.CURRENT, Version.fromId(Version.CURRENT.id + 1) } + ).build(); + + assertEquals(Version.V_7_0_0, metadata.oldestIndexVersion()); + + Metadata.Builder b = Metadata.builder(); + assertEquals(Version.CURRENT, b.build().oldestIndexVersion()); + + Throwable ex = expectThrows( + IllegalArgumentException.class, + () -> buildIndicesWithVersions(new Version[] { Version.V_7_0_0, Version.V_EMPTY, Version.fromId(Version.CURRENT.id + 1) }) + .build() + ); + + assertEquals("[index.version.created] is not present in the index settings for index with UUID [null]", ex.getMessage()); + } + + private Metadata.Builder buildIndicesWithVersions(Version... indexVersions) { + + int lastIndexNum = randomIntBetween(9, 50); + Metadata.Builder b = Metadata.builder(); + for (Version indexVersion : indexVersions) { + IndexMetadata im = IndexMetadata.builder(DataStream.getDefaultBackingIndexName("index", lastIndexNum)) + .settings(settings(indexVersion)) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + b.put(im, false); + lastIndexNum = randomIntBetween(lastIndexNum + 1, lastIndexNum + 50); + } + + return b; + } + private static IndexMetadata.Builder buildIndexMetadata(String name, String alias, Boolean writeIndex) { return IndexMetadata.builder(name) .settings(settings(Version.CURRENT)) diff --git a/server/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/server/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index 4be0d21f8bcec..11a77b75e0a26 100644 --- a/server/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/server/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -7,17 +7,29 @@ */ package org.elasticsearch.env; +import org.apache.lucene.analysis.core.KeywordAnalyzer; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.NIOFSDirectory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.node.DiscoveryNodeRole; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.PathUtils; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.gateway.MetadataStateFormat; +import org.elasticsearch.gateway.PersistedClusterStateService; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; @@ -39,6 +51,10 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import static org.elasticsearch.env.NodeEnvironment.checkForIndexCompatibility; +import static org.elasticsearch.env.NodeMetadata.NODE_VERSION_KEY; +import static org.elasticsearch.env.NodeMetadata.OLDEST_INDEX_VERSION_KEY; +import static org.elasticsearch.gateway.PersistedClusterStateService.METADATA_DIRECTORY_NAME; import static org.elasticsearch.test.NodeRoles.nonDataNode; import static org.elasticsearch.test.NodeRoles.nonMasterNode; import static org.hamcrest.CoreMatchers.equalTo; @@ -523,6 +539,66 @@ public void testBlocksDowngradeToVersionWithMultipleNodesInDataPath() throws IOE } } + public void testIndexCompatibilityChecks() throws IOException { + final Settings settings = buildEnvSettings(Settings.EMPTY); + + try (NodeEnvironment env = newNodeEnvironment(settings)) { + try ( + PersistedClusterStateService.Writer writer = new PersistedClusterStateService( + env.nodeDataPaths(), + env.nodeId(), + xContentRegistry(), + BigArrays.NON_RECYCLING_INSTANCE, + new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + () -> 0L + ).createWriter() + ) { + writer.writeFullStateAndCommit( + 1L, + ClusterState.builder(ClusterName.DEFAULT) + .metadata( + Metadata.builder() + .persistentSettings(Settings.builder().put(Metadata.SETTING_READ_ONLY_SETTING.getKey(), true).build()) + .build() + ) + .build() + ); + } + + Version oldIndexVersion = Version.fromId(between(1, Version.CURRENT.minimumIndexCompatibilityVersion().id - 1)); + overrideOldestIndexVersion(oldIndexVersion, env.nodeDataPaths()); + + IllegalStateException ex = expectThrows( + IllegalStateException.class, + "Must fail the check on index that's too old", + () -> checkForIndexCompatibility(logger, env.nodePaths()) + ); + + assertThat(ex.getMessage(), containsString("[" + oldIndexVersion + "] exist")); + assertThat(ex.getMessage(), startsWith("cannot upgrade node because incompatible indices created with version")); + + // This should work + overrideOldestIndexVersion(Version.CURRENT.minimumIndexCompatibilityVersion(), env.nodeDataPaths()); + checkForIndexCompatibility(logger, env.nodePaths()); + + // Trying to boot with newer version should pass this check + overrideOldestIndexVersion(NodeMetadataTests.tooNewVersion(), env.nodeDataPaths()); + checkForIndexCompatibility(logger, env.nodePaths()); + + // Simulate empty old index version, attempting to upgrade before 7.17 + removeOldestIndexVersion(oldIndexVersion, env.nodeDataPaths()); + + ex = expectThrows( + IllegalStateException.class, + "Must fail the check on index that's too old", + () -> checkForIndexCompatibility(logger, env.nodePaths()) + ); + + assertThat(ex.getMessage(), startsWith("cannot upgrade a node from version [" + oldIndexVersion + "] directly")); + assertThat(ex.getMessage(), containsString("upgrade to version [" + Version.CURRENT.minimumCompatibilityVersion())); + } + } + private void verifyFailsOnShardData(Settings settings, Path indexPath, String shardDataDirName) { IllegalStateException ex = expectThrows( IllegalStateException.class, @@ -603,4 +679,54 @@ public NodeEnvironment newNodeEnvironment(String[] dataPaths, String sharedDataP .build(); return new NodeEnvironment(build, TestEnvironment.newEnvironment(build)); } + + private static void overrideOldestIndexVersion(Version oldVersion, Path... dataPaths) throws IOException { + for (final Path dataPath : dataPaths) { + final Path indexPath = dataPath.resolve(METADATA_DIRECTORY_NAME); + if (Files.exists(indexPath)) { + try (DirectoryReader reader = DirectoryReader.open(new NIOFSDirectory(dataPath.resolve(METADATA_DIRECTORY_NAME)))) { + final Map userData = reader.getIndexCommit().getUserData(); + final IndexWriterConfig indexWriterConfig = new IndexWriterConfig(new KeywordAnalyzer()); + + try ( + IndexWriter indexWriter = new IndexWriter( + new NIOFSDirectory(dataPath.resolve(METADATA_DIRECTORY_NAME)), + indexWriterConfig + ) + ) { + final Map commitData = new HashMap<>(userData); + commitData.put(NODE_VERSION_KEY, Integer.toString(Version.CURRENT.minimumCompatibilityVersion().id)); + commitData.put(OLDEST_INDEX_VERSION_KEY, Integer.toString(oldVersion.id)); + indexWriter.setLiveCommitData(commitData.entrySet()); + indexWriter.commit(); + } + } + } + } + } + + private static void removeOldestIndexVersion(Version oldVersion, Path... dataPaths) throws IOException { + for (final Path dataPath : dataPaths) { + final Path indexPath = dataPath.resolve(METADATA_DIRECTORY_NAME); + if (Files.exists(indexPath)) { + try (DirectoryReader reader = DirectoryReader.open(new NIOFSDirectory(dataPath.resolve(METADATA_DIRECTORY_NAME)))) { + final Map userData = reader.getIndexCommit().getUserData(); + final IndexWriterConfig indexWriterConfig = new IndexWriterConfig(new KeywordAnalyzer()); + + try ( + IndexWriter indexWriter = new IndexWriter( + new NIOFSDirectory(dataPath.resolve(METADATA_DIRECTORY_NAME)), + indexWriterConfig + ) + ) { + final Map commitData = new HashMap<>(userData); + commitData.put(NODE_VERSION_KEY, Integer.toString(oldVersion.id)); + commitData.remove(OLDEST_INDEX_VERSION_KEY); + indexWriter.setLiveCommitData(commitData.entrySet()); + indexWriter.commit(); + } + } + } + } + } } diff --git a/server/src/test/java/org/elasticsearch/env/NodeMetadataTests.java b/server/src/test/java/org/elasticsearch/env/NodeMetadataTests.java index 84f1926ef9b8b..0f10ea2b6548c 100644 --- a/server/src/test/java/org/elasticsearch/env/NodeMetadataTests.java +++ b/server/src/test/java/org/elasticsearch/env/NodeMetadataTests.java @@ -34,22 +34,41 @@ private Version randomVersion() { public void testEqualsHashcodeSerialization() { final Path tempDir = createTempDir(); - EqualsHashCodeTestUtils.checkEqualsAndHashCode(new NodeMetadata(randomAlphaOfLength(10), randomVersion()), nodeMetadata -> { - final long generation = NodeMetadata.FORMAT.writeAndCleanup(nodeMetadata, tempDir); - final Tuple nodeMetadataLongTuple = NodeMetadata.FORMAT.loadLatestStateWithGeneration( - logger, - xContentRegistry(), - tempDir - ); - assertThat(nodeMetadataLongTuple.v2(), equalTo(generation)); - return nodeMetadataLongTuple.v1(); - }, nodeMetadata -> { - if (randomBoolean()) { - return new NodeMetadata(randomAlphaOfLength(21 - nodeMetadata.nodeId().length()), nodeMetadata.nodeVersion()); - } else { - return new NodeMetadata(nodeMetadata.nodeId(), randomValueOtherThan(nodeMetadata.nodeVersion(), this::randomVersion)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new NodeMetadata(randomAlphaOfLength(10), randomVersion(), randomVersion()), + nodeMetadata -> { + final long generation = NodeMetadata.FORMAT.writeAndCleanup(nodeMetadata, tempDir); + final Tuple nodeMetadataLongTuple = NodeMetadata.FORMAT.loadLatestStateWithGeneration( + logger, + xContentRegistry(), + tempDir + ); + assertThat(nodeMetadataLongTuple.v2(), equalTo(generation)); + return nodeMetadataLongTuple.v1(); + }, + nodeMetadata -> { + switch (randomInt(3)) { + case 0: + return new NodeMetadata( + randomAlphaOfLength(21 - nodeMetadata.nodeId().length()), + nodeMetadata.nodeVersion(), + Version.CURRENT + ); + case 1: + return new NodeMetadata( + nodeMetadata.nodeId(), + randomValueOtherThan(nodeMetadata.nodeVersion(), this::randomVersion), + Version.CURRENT + ); + default: + return new NodeMetadata( + nodeMetadata.nodeId(), + nodeMetadata.nodeVersion(), + randomValueOtherThan(Version.CURRENT, this::randomVersion) + ); + } } - }); + ); } public void testReadsFormatWithoutVersion() throws IOException { @@ -75,7 +94,8 @@ public void testUpgradesLegitimateVersions() { randomValueOtherThanMany( v -> v.after(Version.CURRENT) || v.before(Version.CURRENT.minimumCompatibilityVersion()), this::randomVersion - ) + ), + Version.CURRENT ).upgradeToCurrentVersion(); assertThat(nodeMetadata.nodeVersion(), equalTo(Version.CURRENT)); assertThat(nodeMetadata.nodeId(), equalTo(nodeId)); @@ -86,7 +106,7 @@ public void testUpgradesMissingVersion() { final IllegalStateException illegalStateException = expectThrows( IllegalStateException.class, - () -> new NodeMetadata(nodeId, Version.V_EMPTY).upgradeToCurrentVersion() + () -> new NodeMetadata(nodeId, Version.V_EMPTY, Version.CURRENT).upgradeToCurrentVersion() ); assertThat( illegalStateException.getMessage(), @@ -97,7 +117,7 @@ public void testUpgradesMissingVersion() { public void testDoesNotUpgradeFutureVersion() { final IllegalStateException illegalStateException = expectThrows( IllegalStateException.class, - () -> new NodeMetadata(randomAlphaOfLength(10), tooNewVersion()).upgradeToCurrentVersion() + () -> new NodeMetadata(randomAlphaOfLength(10), tooNewVersion(), Version.CURRENT).upgradeToCurrentVersion() ); assertThat( illegalStateException.getMessage(), @@ -108,7 +128,7 @@ public void testDoesNotUpgradeFutureVersion() { public void testDoesNotUpgradeAncientVersion() { final IllegalStateException illegalStateException = expectThrows( IllegalStateException.class, - () -> new NodeMetadata(randomAlphaOfLength(10), tooOldVersion()).upgradeToCurrentVersion() + () -> new NodeMetadata(randomAlphaOfLength(10), tooOldVersion(), Version.CURRENT).upgradeToCurrentVersion() ); assertThat( illegalStateException.getMessage(), @@ -129,7 +149,7 @@ public void testUpgradeMarksPreviousVersion() { 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).upgradeToCurrentVersion(); + final NodeMetadata nodeMetadata = new NodeMetadata(nodeId, version, Version.CURRENT).upgradeToCurrentVersion(); assertThat(nodeMetadata.nodeVersion(), equalTo(Version.CURRENT)); assertThat(nodeMetadata.previousNodeVersion(), equalTo(version)); } diff --git a/server/src/test/java/org/elasticsearch/gateway/PersistedClusterStateServiceTests.java b/server/src/test/java/org/elasticsearch/gateway/PersistedClusterStateServiceTests.java index cc03be151d0c3..c64f3de2903df 100644 --- a/server/src/test/java/org/elasticsearch/gateway/PersistedClusterStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/PersistedClusterStateServiceTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.CoordinationMetadata; +import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -516,7 +517,7 @@ public void sync(Collection names) { if (randomBoolean()) { writeState(writer, newTerm, newState, clusterState); } else { - writer.commit(newTerm, newState.version()); + writer.commit(newTerm, newState.version(), newState.metadata().oldestIndexVersion()); } }).getMessage(), containsString("simulated")); assertFalse(writer.isOpen()); @@ -568,7 +569,7 @@ public void rename(String source, String dest) throws IOException { if (randomBoolean()) { writeState(writer, newTerm, newState, clusterState); } else { - writer.commit(newTerm, newState.version()); + writer.commit(newTerm, newState.version(), newState.metadata().oldestIndexVersion()); } }).getMessage(), containsString("simulated")); assertFalse(writer.isOpen()); @@ -1251,6 +1252,48 @@ public void testLimitsFileCount() throws IOException { } } + public void testOldestIndexVersionIsCorrectlySerialized() throws IOException { + final Path[] dataPaths1 = createDataPaths(); + final Path[] dataPaths2 = createDataPaths(); + final Path[] combinedPaths = Stream.concat(Arrays.stream(dataPaths1), Arrays.stream(dataPaths2)).toArray(Path[]::new); + + Version oldVersion = Version.fromId(Version.CURRENT.minimumIndexCompatibilityVersion().id - 1); + + final Version[] indexVersions = new Version[] { oldVersion, Version.CURRENT, Version.fromId(Version.CURRENT.id + 1) }; + int lastIndexNum = randomIntBetween(9, 50); + Metadata.Builder b = Metadata.builder(); + for (Version indexVersion : indexVersions) { + String indexUUID = UUIDs.randomBase64UUID(random()); + IndexMetadata im = IndexMetadata.builder(DataStream.getDefaultBackingIndexName("index", lastIndexNum)) + .settings(settings(indexVersion).put(IndexMetadata.SETTING_INDEX_UUID, indexUUID)) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + b.put(im, false); + lastIndexNum = randomIntBetween(lastIndexNum + 1, lastIndexNum + 50); + } + + Metadata metadata = b.build(); + + try (NodeEnvironment nodeEnvironment = newNodeEnvironment(combinedPaths)) { + try (Writer writer = newPersistedClusterStateService(nodeEnvironment).createWriter()) { + final ClusterState clusterState = loadPersistedClusterState(newPersistedClusterStateService(nodeEnvironment)); + writeState( + writer, + 0L, + ClusterState.builder(clusterState).metadata(metadata).version(randomLongBetween(1L, Long.MAX_VALUE)).build(), + clusterState + ); + } + + PersistedClusterStateService.OnDiskState fromDisk = newPersistedClusterStateService(nodeEnvironment).loadBestOnDiskState(); + NodeMetadata nodeMetadata = PersistedClusterStateService.nodeMetadata(nodeEnvironment.nodeDataPaths()); + + assertEquals(oldVersion, nodeMetadata.oldestIndexVersion()); + assertEquals(oldVersion, fromDisk.metadata.oldestIndexVersion()); + } + } + private void assertExpectedLogs( long currentTerm, ClusterState previousState, diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityImplicitBehaviorBootstrapCheckTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityImplicitBehaviorBootstrapCheckTests.java index 3380a70ae9265..139fdff01b85b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityImplicitBehaviorBootstrapCheckTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityImplicitBehaviorBootstrapCheckTests.java @@ -30,7 +30,7 @@ public void testFailureUpgradeFrom7xWithImplicitSecuritySettings() throws Except Version.V_8_0_0, () -> VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.V_8_0_0) ); - NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion); + NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion, Version.CURRENT); nodeMetadata = nodeMetadata.upgradeToCurrentVersion(); BootstrapCheck.BootstrapCheckResult result = new SecurityImplicitBehaviorBootstrapCheck(nodeMetadata).check( createTestContext(Settings.EMPTY, createLicensesMetadata(previousVersion, randomFrom("basic", "trial"))) @@ -59,7 +59,7 @@ public void testUpgradeFrom7xWithImplicitSecuritySettingsOnGoldPlus() throws Exc Version.V_8_0_0, () -> VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.V_8_0_0) ); - NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion); + NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion, Version.CURRENT); nodeMetadata = nodeMetadata.upgradeToCurrentVersion(); BootstrapCheck.BootstrapCheckResult result = new SecurityImplicitBehaviorBootstrapCheck(nodeMetadata).check( createTestContext(Settings.EMPTY, createLicensesMetadata(previousVersion, randomFrom("gold", "platinum"))) @@ -72,7 +72,7 @@ public void testUpgradeFrom7xWithExplicitSecuritySettings() throws Exception { Version.V_8_0_0, () -> VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.V_8_0_0) ); - NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion); + NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion, Version.CURRENT); nodeMetadata = nodeMetadata.upgradeToCurrentVersion(); BootstrapCheck.BootstrapCheckResult result = new SecurityImplicitBehaviorBootstrapCheck(nodeMetadata).check( createTestContext( @@ -85,7 +85,7 @@ public void testUpgradeFrom7xWithExplicitSecuritySettings() throws Exception { public void testUpgradeFrom8xWithImplicitSecuritySettings() throws Exception { final Version previousVersion = VersionUtils.randomVersionBetween(random(), Version.V_8_0_0, null); - NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion); + NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion, Version.CURRENT); nodeMetadata = nodeMetadata.upgradeToCurrentVersion(); BootstrapCheck.BootstrapCheckResult result = new SecurityImplicitBehaviorBootstrapCheck(nodeMetadata).check( createTestContext(Settings.EMPTY, createLicensesMetadata(previousVersion, randomFrom("basic", "trial"))) @@ -95,7 +95,7 @@ public void testUpgradeFrom8xWithImplicitSecuritySettings() throws Exception { public void testUpgradeFrom8xWithExplicitSecuritySettings() throws Exception { final Version previousVersion = VersionUtils.randomVersionBetween(random(), Version.V_8_0_0, null); - NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion); + NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(10), previousVersion, Version.CURRENT); nodeMetadata = nodeMetadata.upgradeToCurrentVersion(); BootstrapCheck.BootstrapCheckResult result = new SecurityImplicitBehaviorBootstrapCheck(nodeMetadata).check( createTestContext( diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index d1b9da81b6373..cdcf19cd61d70 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -129,7 +129,7 @@ public Map getRealms(SecurityComponents components) { private Collection createComponentsUtil(Settings settings, SecurityExtension... extensions) throws Exception { Environment env = TestEnvironment.newEnvironment(settings); - NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(8), Version.CURRENT); + NodeMetadata nodeMetadata = new NodeMetadata(randomAlphaOfLength(8), Version.CURRENT, Version.CURRENT); licenseState = new TestUtils.UpdatableLicenseState(settings); SSLService sslService = new SSLService(env); security = new Security(settings, null, Arrays.asList(extensions)) {