diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index 35e92c0d255fa..80d870e2dcc4a 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -218,12 +218,17 @@ public static void writeVersion(Version version, StreamOutput out) throws IOExce } /** - * Returns the smallest version between the 2. + * Returns the minimum version between the 2. */ - public static Version smallest(Version version1, Version version2) { + public static Version min(Version version1, Version version2) { return version1.id < version2.id ? version1 : version2; } + /** + * Returns the maximum version between the 2 + */ + public static Version max(Version version1, Version version2) { return version1.id > version2.id ? version1 : version2; } + /** * Returns the version given its string representation, current version if the argument is null or empty */ @@ -326,7 +331,22 @@ public Version minimumCompatibilityVersion() { bwcMajor = major; bwcMinor = 0; } - return Version.smallest(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); + return Version.min(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); + } + + /** + * Returns the minimum created index version that this version supports. Indices created with lower versions + * can't be used with this version. + */ + public Version minimumIndexCompatibilityVersion() { + final int bwcMajor; + if (major == 5) { + bwcMajor = 2; // we jumped from 2 to 5 + } else { + bwcMajor = major - 1; + } + final int bwcMinor = 0; + return Version.min(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); } /** @@ -414,5 +434,4 @@ public boolean isRC() { public boolean isRelease() { return build == 99; } - } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index c19dcdd0ecb41..b1ffccf6aeba4 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -316,7 +316,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { if (indexSettingsBuilder.get(SETTING_VERSION_CREATED) == null) { DiscoveryNodes nodes = currentState.nodes(); - final Version createdVersion = Version.smallest(Version.CURRENT, nodes.getSmallestNonClientNodeVersion()); + final Version createdVersion = Version.min(Version.CURRENT, nodes.getSmallestNonClientNodeVersion()); indexSettingsBuilder.put(SETTING_VERSION_CREATED, createdVersion); } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index 689eff0da61f4..2a2c6c65b967c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -20,6 +20,7 @@ package org.elasticsearch.cluster.metadata; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest; @@ -160,12 +161,14 @@ public ClusterState execute(ClusterState currentState) { MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder() .blocks(currentState.blocks()); + final Version minIndexCompatibilityVersion = currentState.getNodes().getMaxNodeVersion() + .minimumIndexCompatibilityVersion(); for (IndexMetaData closedMetaData : indicesToOpen) { final String indexName = closedMetaData.getIndex().getName(); IndexMetaData indexMetaData = IndexMetaData.builder(closedMetaData).state(IndexMetaData.State.OPEN).build(); // The index might be closed because we couldn't import it due to old incompatible version // We need to check that this index can be upgraded to the current version - indexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData); + indexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData, minIndexCompatibilityVersion); try { indicesService.verifyIndexMetadata(indexMetaData, indexMetaData); } catch (Exception e) { diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index 6e5b110563de1..2a8b80b9e689e 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -67,13 +67,13 @@ public MetaDataIndexUpgradeService(Settings settings, MapperRegistry mapperRegis * If the index does not need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index * cannot be updated the method throws an exception. */ - public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) { + public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { // Throws an exception if there are too-old segments: if (isUpgraded(indexMetaData)) { assert indexMetaData == archiveBrokenIndexSettings(indexMetaData) : "all settings must have been upgraded before"; return indexMetaData; } - checkSupportedVersion(indexMetaData); + checkSupportedVersion(indexMetaData, minimumIndexCompatibilityVersion); IndexMetaData newMetaData = indexMetaData; // we have to run this first otherwise in we try to create IndexSettings // with broken settings and fail in checkMappingsCompatibility @@ -92,21 +92,26 @@ boolean isUpgraded(IndexMetaData indexMetaData) { } /** - * Elasticsearch 5.0 no longer supports indices with pre Lucene v5.0 (Elasticsearch v2.0.0.beta1) segments. All indices - * that were created before Elasticsearch v2.0.0.beta1 should be reindexed in Elasticsearch 2.x - * before they can be opened by this version of elasticsearch. */ - private void checkSupportedVersion(IndexMetaData indexMetaData) { - if (indexMetaData.getState() == IndexMetaData.State.OPEN && isSupportedVersion(indexMetaData) == false) { - throw new IllegalStateException("The index [" + indexMetaData.getIndex() + "] was created before v2.0.0.beta1." - + " It should be reindexed in Elasticsearch 2.x before upgrading to " + Version.CURRENT + "."); + * Elasticsearch v6.0 no longer supports indices created pre v5.0. All indices + * that were created before Elasticsearch v5.0 should be re-indexed in Elasticsearch 5.x + * before they can be opened by this version of elasticsearch. + */ + private void checkSupportedVersion(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { + if (indexMetaData.getState() == IndexMetaData.State.OPEN && isSupportedVersion(indexMetaData, + minimumIndexCompatibilityVersion) == false) { + throw new IllegalStateException("The index [" + indexMetaData.getIndex() + "] was created with version [" + + indexMetaData.getCreationVersion() + "] but the minimum compatible version is [" + + + minimumIndexCompatibilityVersion + "]. It should be re-indexed in Elasticsearch " + minimumIndexCompatibilityVersion.major + + ".x before upgrading to " + Version.CURRENT + "."); } } /* * Returns true if this index can be supported by the current version of elasticsearch */ - private static boolean isSupportedVersion(IndexMetaData indexMetaData) { - return indexMetaData.getCreationVersion().onOrAfter(Version.V_5_0_0_beta1); + private static boolean isSupportedVersion(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { + return indexMetaData.getCreationVersion().onOrAfter(minimumIndexCompatibilityVersion); } /** @@ -173,4 +178,4 @@ IndexMetaData archiveBrokenIndexSettings(IndexMetaData indexMetaData) { return indexMetaData; } } -} \ No newline at end of file +} diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index e557b52c1d4e3..6d80a9573babd 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -56,10 +56,13 @@ public class DiscoveryNodes extends AbstractDiffable implements private final String masterNodeId; private final String localNodeId; private final Version minNonClientNodeVersion; + private final Version maxNodeVersion; + private final Version minNodeVersion; private DiscoveryNodes(ImmutableOpenMap nodes, ImmutableOpenMap dataNodes, ImmutableOpenMap masterNodes, ImmutableOpenMap ingestNodes, - String masterNodeId, String localNodeId, Version minNonClientNodeVersion) { + String masterNodeId, String localNodeId, Version minNonClientNodeVersion, Version maxNodeVersion, + Version minNodeVersion) { this.nodes = nodes; this.dataNodes = dataNodes; this.masterNodes = masterNodes; @@ -67,6 +70,8 @@ private DiscoveryNodes(ImmutableOpenMap nodes, ImmutableO this.masterNodeId = masterNodeId; this.localNodeId = localNodeId; this.minNonClientNodeVersion = minNonClientNodeVersion; + this.minNodeVersion = minNodeVersion; + this.maxNodeVersion = maxNodeVersion; } @Override @@ -235,6 +240,24 @@ public Version getSmallestNonClientNodeVersion() { return minNonClientNodeVersion; } + /** + * Returns the version of the node with the oldest version in the cluster. + * + * @return the oldest version in the cluster + */ + public Version getMinNodeVersion() { + return minNodeVersion; + } + + /** + * Returns the version of the node with the yougest version in the cluster + * + * @return the oldest version in the cluster + */ + public Version getMaxNodeVersion() { + return maxNodeVersion; + } + /** * Resolve a node with a given id * @@ -631,25 +654,27 @@ public DiscoveryNodes build() { ImmutableOpenMap.Builder masterNodesBuilder = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder ingestNodesBuilder = ImmutableOpenMap.builder(); Version minNodeVersion = Version.CURRENT; + Version maxNodeVersion = Version.CURRENT; Version minNonClientNodeVersion = Version.CURRENT; for (ObjectObjectCursor nodeEntry : nodes) { if (nodeEntry.value.isDataNode()) { dataNodesBuilder.put(nodeEntry.key, nodeEntry.value); - minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.getVersion()); + minNonClientNodeVersion = Version.min(minNonClientNodeVersion, nodeEntry.value.getVersion()); } if (nodeEntry.value.isMasterNode()) { masterNodesBuilder.put(nodeEntry.key, nodeEntry.value); - minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.getVersion()); + minNonClientNodeVersion = Version.min(minNonClientNodeVersion, nodeEntry.value.getVersion()); } if (nodeEntry.value.isIngestNode()) { ingestNodesBuilder.put(nodeEntry.key, nodeEntry.value); } - minNodeVersion = Version.smallest(minNodeVersion, nodeEntry.value.getVersion()); + minNodeVersion = Version.min(minNodeVersion, nodeEntry.value.getVersion()); + maxNodeVersion = Version.max(maxNodeVersion, nodeEntry.value.getVersion()); } return new DiscoveryNodes( nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), ingestNodesBuilder.build(), - masterNodeId, localNodeId, minNonClientNodeVersion + masterNodeId, localNodeId, minNonClientNodeVersion, maxNodeVersion, minNodeVersion ); } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java b/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java index 7ff8f935927a1..c852716819820 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java @@ -19,14 +19,16 @@ package org.elasticsearch.discovery.zen; +import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.EmptyTransportResponseHandler; import org.elasticsearch.transport.TransportChannel; @@ -37,6 +39,7 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; public class MembershipAction extends AbstractComponent { @@ -58,21 +61,20 @@ public interface MembershipListener { private final TransportService transportService; - private final DiscoveryNodesProvider nodesProvider; - private final MembershipListener listener; public MembershipAction(Settings settings, TransportService transportService, - DiscoveryNodesProvider nodesProvider, MembershipListener listener) { + Supplier localNodeSupplier, MembershipListener listener) { super(settings); this.transportService = transportService; - this.nodesProvider = nodesProvider; this.listener = listener; + transportService.registerRequestHandler(DISCOVERY_JOIN_ACTION_NAME, JoinRequest::new, ThreadPool.Names.GENERIC, new JoinRequestRequestHandler()); - transportService.registerRequestHandler(DISCOVERY_JOIN_VALIDATE_ACTION_NAME, ValidateJoinRequest::new, - ThreadPool.Names.GENERIC, new ValidateJoinRequestRequestHandler()); + transportService.registerRequestHandler(DISCOVERY_JOIN_VALIDATE_ACTION_NAME, + () -> new ValidateJoinRequest(localNodeSupplier), ThreadPool.Names.GENERIC, + new ValidateJoinRequestRequestHandler()); transportService.registerRequestHandler(DISCOVERY_LEAVE_ACTION_NAME, LeaveRequest::new, ThreadPool.Names.GENERIC, new LeaveRequestRequestHandler()); } @@ -152,20 +154,23 @@ public void onFailure(Exception e) { } } - class ValidateJoinRequest extends TransportRequest { + static class ValidateJoinRequest extends TransportRequest { + private final Supplier localNode; private ClusterState state; - ValidateJoinRequest() { + ValidateJoinRequest(Supplier localNode) { + this.localNode = localNode; } ValidateJoinRequest(ClusterState state) { this.state = state; + this.localNode = state.nodes()::getLocalNode; } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - this.state = ClusterState.Builder.readFrom(in, nodesProvider.nodes().getLocalNode()); + this.state = ClusterState.Builder.readFrom(in, localNode.get()); } @Override @@ -175,15 +180,31 @@ public void writeTo(StreamOutput out) throws IOException { } } - class ValidateJoinRequestRequestHandler implements TransportRequestHandler { + static class ValidateJoinRequestRequestHandler implements TransportRequestHandler { @Override public void messageReceived(ValidateJoinRequest request, TransportChannel channel) throws Exception { + ensureIndexCompatibility(Version.CURRENT.minimumIndexCompatibilityVersion(), request.state.getMetaData()); // for now, the mere fact that we can serialize the cluster state acts as validation.... channel.sendResponse(TransportResponse.Empty.INSTANCE); } } + /** + * Ensures that all indices are compatible with the supported index version. + * @throws IllegalStateException if any index is incompatible with the given version + */ + static void ensureIndexCompatibility(final Version supportedIndexVersion, MetaData metaData) { + // we ensure that all indices in the cluster we join are compatible with us no matter if they are + // closed or not we can't read mappings of these indices so we need to reject the join... + for (IndexMetaData idxMetaData : metaData) { + if (idxMetaData.getCreationVersion().before(supportedIndexVersion)) { + throw new IllegalStateException("index " + idxMetaData.getIndex() + " version not supported: " + + idxMetaData.getCreationVersion() + " minimum compatible index version is: " + supportedIndexVersion); + } + } + } + public static class LeaveRequest extends TransportRequest { private DiscoveryNode node; diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 6d77e2f48fe33..4dce87aa86e70 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -410,7 +410,6 @@ class JoinTaskExecutor implements ClusterStateTaskExecutor { @Override public BatchResult execute(ClusterState currentState, List joiningNodes) throws Exception { final BatchResult.Builder results = BatchResult.builder(); - final DiscoveryNodes currentNodes = currentState.nodes(); boolean nodesChanged = false; ClusterState.Builder newState; @@ -435,8 +434,10 @@ public BatchResult execute(ClusterState currentState, List execute(ClusterState currentState, List) () -> new ParameterizedMessage("found dangled index [{}] on node [{}]. This index cannot be upgraded to the latest version, adding as closed", indexMetaData.getIndex(), request.fromNode), ex); diff --git a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java index ed6317fcb3079..6ce6ee61938de 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -83,6 +83,7 @@ import java.util.Set; import java.util.stream.Collectors; +import static java.util.Collections.min; import static java.util.Collections.unmodifiableSet; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_DATE; @@ -225,6 +226,8 @@ public ClusterState execute(ClusterState currentState) { if (!renamedIndices.isEmpty()) { // We have some indices to restore ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); + final Version minIndexCompatibilityVersion = currentState.getNodes().getMaxNodeVersion() + .minimumIndexCompatibilityVersion(); for (Map.Entry indexEntry : renamedIndices.entrySet()) { String index = indexEntry.getValue(); boolean partial = checkPartial(index); @@ -233,7 +236,8 @@ public ClusterState execute(ClusterState currentState) { IndexMetaData snapshotIndexMetaData = metaData.index(index); snapshotIndexMetaData = updateIndexSettings(snapshotIndexMetaData, request.indexSettings, request.ignoreIndexSettings); try { - snapshotIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(snapshotIndexMetaData); + snapshotIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(snapshotIndexMetaData, + minIndexCompatibilityVersion); } catch (Exception ex) { throw new SnapshotRestoreException(snapshot, "cannot restore index [" + index + "] because it cannot be upgraded", ex); } diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index 11e8de3c3cc2f..d9d3030c787c5 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -909,7 +909,7 @@ public void sendRequest(final DiscoveryNode node, final long requestId, final St // we pick the smallest of the 2, to support both backward and forward compatibility // note, this is the only place we need to do this, since from here on, we use the serialized version // as the version to use also when the node receiving this request will send the response with - Version version = Version.smallest(getCurrentVersion(), node.getVersion()); + Version version = Version.min(getCurrentVersion(), node.getVersion()); stream.setVersion(version); threadPool.getThreadContext().writeTo(stream); diff --git a/core/src/test/java/org/elasticsearch/VersionTests.java b/core/src/test/java/org/elasticsearch/VersionTests.java index 3bccee941b335..51f7dd8f85769 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -64,7 +64,37 @@ public void testVersionComparison() throws Exception { assertTrue(Version.fromString("5.0.0").onOrAfter(Version.fromString("5.0.0-beta2"))); assertTrue(Version.fromString("5.0.0-rc1").onOrAfter(Version.fromString("5.0.0-beta24"))); assertTrue(Version.fromString("5.0.0-alpha24").before(Version.fromString("5.0.0-beta0"))); + } + + public void testMin() { + assertEquals(VersionUtils.getPreviousVersion(), Version.min(Version.CURRENT, VersionUtils.getPreviousVersion())); + assertEquals(Version.fromString("1.0.1"), Version.min(Version.fromString("1.0.1"), Version.CURRENT)); + Version version = VersionUtils.randomVersion(random()); + Version version1 = VersionUtils.randomVersion(random()); + if (version.id <= version1.id) { + assertEquals(version, Version.min(version1, version)); + } else { + assertEquals(version1, Version.min(version1, version)); + } + } + + public void testMax() { + assertEquals(Version.CURRENT, Version.max(Version.CURRENT, VersionUtils.getPreviousVersion())); + assertEquals(Version.CURRENT, Version.max(Version.fromString("1.0.1"), Version.CURRENT)); + Version version = VersionUtils.randomVersion(random()); + Version version1 = VersionUtils.randomVersion(random()); + if (version.id >= version1.id) { + assertEquals(version, Version.max(version1, version)); + } else { + assertEquals(version1, Version.max(version1, version)); + } + } + public void testMinimumIndexCompatibilityVersion() { + assertEquals(Version.V_5_0_0, Version.V_6_0_0_alpha1_UNRELEASED.minimumIndexCompatibilityVersion()); + assertEquals(Version.V_2_0_0, Version.V_5_0_0.minimumIndexCompatibilityVersion()); + assertEquals(Version.V_2_0_0, Version.V_5_1_1_UNRELEASED.minimumIndexCompatibilityVersion()); + assertEquals(Version.V_2_0_0, Version.V_5_0_0_alpha1.minimumIndexCompatibilityVersion()); } public void testVersionConstantPresent() { diff --git a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java index a7e3583a5b395..1fd71c7ae5119 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java @@ -189,7 +189,7 @@ public void testAllVersionsTested() throws Exception { for (Version v : VersionUtils.allReleasedVersions()) { if (VersionUtils.isSnapshot(v)) continue; // snapshots are unreleased, so there is no backcompat yet if (v.isRelease() == false) continue; // no guarantees for prereleases - if (v.before(Version.V_5_0_0)) continue; // we can only support one major version backward + if (v.before(Version.CURRENT.minimumIndexCompatibilityVersion())) continue; // we can only support one major version backward if (v.equals(Version.CURRENT)) continue; // the current version is always compatible with itself expectedVersions.add("index-" + v.toString() + ".zip"); } diff --git a/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java b/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java index 8bae0cbe635e5..5a46f50382d0c 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java @@ -97,7 +97,7 @@ public void testRestoreOldSnapshots() throws Exception { for (Version v : VersionUtils.allReleasedVersions()) { if (VersionUtils.isSnapshot(v)) continue; // snapshots are unreleased, so there is no backcompat yet if (v.isRelease() == false) continue; // no guarantees for prereleases - if (v.before(Version.V_5_0_0)) continue; // we only support versions N and N-1 + if (v.before(Version.CURRENT.minimumIndexCompatibilityVersion())) continue; // we only support versions N and N-1 if (v.equals(Version.CURRENT)) continue; // the current version is always compatible with itself expectedVersions.add(v.toString()); } @@ -206,7 +206,7 @@ private void assertUnsupportedIndexFailsToRestore(String repo, String snapshot) logger.info("--> restoring unsupported snapshot"); try { client().admin().cluster().prepareRestoreSnapshot(repo, snapshot).setRestoreGlobalState(true).setWaitForCompletion(true).get(); - fail("should have failed to restore"); + fail("should have failed to restore - " + repo); } catch (SnapshotRestoreException ex) { assertThat(ex.getMessage(), containsString("cannot restore index")); assertThat(ex.getMessage(), containsString("because it cannot be upgraded")); diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java index 508c93284fe37..a1dfdbc74f9ac 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java @@ -60,11 +60,11 @@ public void testUpgrade() { Collections.emptyMap()), IndexScopedSettings.DEFAULT_SCOPED_SETTINGS); IndexMetaData src = newIndexMeta("foo", Settings.builder().put("index.refresh_interval", "-200").build()); assertFalse(service.isUpgraded(src)); - src = service.upgradeIndexMetaData(src); + src = service.upgradeIndexMetaData(src, Version.CURRENT.minimumIndexCompatibilityVersion()); assertTrue(service.isUpgraded(src)); assertEquals("-200", src.getSettings().get("archived.index.refresh_interval")); assertNull(src.getSettings().get("index.refresh_interval")); - assertSame(src, service.upgradeIndexMetaData(src)); // no double upgrade + assertSame(src, service.upgradeIndexMetaData(src, Version.CURRENT.minimumIndexCompatibilityVersion())); // no double upgrade } public void testIsUpgraded() { @@ -87,16 +87,17 @@ public void testFailUpgrade() { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.fromString("2.4.0")) .put(IndexMetaData.SETTING_VERSION_MINIMUM_COMPATIBLE, Version.CURRENT.luceneVersion.toString()).build()); - String message = expectThrows(IllegalStateException.class, () -> service.upgradeIndexMetaData(metaData)).getMessage(); - assertEquals(message, "The index [[foo/BOOM]] was created before v2.0.0.beta1. It should be reindexed in Elasticsearch 2.x " + - "before upgrading to " + Version.CURRENT.toString() + "."); + String message = expectThrows(IllegalStateException.class, () -> service.upgradeIndexMetaData(metaData, + Version.CURRENT.minimumIndexCompatibilityVersion())).getMessage(); + assertEquals(message, "The index [[foo/BOOM]] was created with version [2.4.0] but the minimum compatible version is [5.0.0]." + + " It should be re-indexed in Elasticsearch 5.x before upgrading to " + Version.CURRENT.toString() + "."); IndexMetaData goodMeta = newIndexMeta("foo", Settings.builder() .put(IndexMetaData.SETTING_VERSION_UPGRADED, Version.V_5_0_0_beta1) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.fromString("5.1.0")) .put(IndexMetaData.SETTING_VERSION_MINIMUM_COMPATIBLE, Version.CURRENT.luceneVersion.toString()).build()); - service.upgradeIndexMetaData(goodMeta); + service.upgradeIndexMetaData(goodMeta, Version.V_5_0_0.minimumIndexCompatibilityVersion()); } public static IndexMetaData newIndexMeta(String name, Settings indexSettings) { diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 342919fb8812c..4ad1c5fdd08b7 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -22,6 +22,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.elasticsearch.Version; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; import java.util.ArrayList; import java.util.Arrays; @@ -250,4 +251,22 @@ Set matchingNodeIds(DiscoveryNodes nodes) { abstract Set matchingNodeIds(DiscoveryNodes nodes); } + + public void testMaxMinNodeVersion() { + DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); + discoBuilder.add(new DiscoveryNode("name_" + 1, "node_" + 1, buildNewFakeTransportAddress(), Collections.emptyMap(), + new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), + Version.fromString("5.1.0"))); + discoBuilder.add(new DiscoveryNode("name_" + 2, "node_" + 2, buildNewFakeTransportAddress(), Collections.emptyMap(), + new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), + Version.fromString("6.3.0"))); + discoBuilder.add(new DiscoveryNode("name_" + 3, "node_" + 3, buildNewFakeTransportAddress(), Collections.emptyMap(), + new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), + Version.fromString("1.1.0"))); + discoBuilder.localNodeId("name_1"); + discoBuilder.masterNodeId("name_2"); + DiscoveryNodes build = discoBuilder.build(); + assertEquals( Version.fromString("6.3.0"), build.getMaxNodeVersion()); + assertEquals( Version.fromString("1.1.0"), build.getMinNodeVersion()); + } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index acc5d4e801868..b46df47afbcd5 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -20,13 +20,16 @@ package org.elasticsearch.discovery.zen; import java.io.Closeable; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import org.apache.lucene.util.IOUtils; @@ -35,22 +38,40 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode.Role; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseOptions; import org.elasticsearch.transport.TransportService; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_DATE; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED; +import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations; import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; @@ -283,4 +304,82 @@ private Set fdNodesForState(ClusterState clusterState, DiscoveryN }); return discoveryNodes; } + + public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { + final int iters = randomIntBetween(3, 10); + for (int i = 0; i < iters; i++) { + ClusterState.Builder stateBuilder = ClusterState.builder(ClusterName.DEFAULT); + final DiscoveryNode otherNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), + EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); + MembershipAction.ValidateJoinRequestRequestHandler request = new MembershipAction.ValidateJoinRequestRequestHandler(); + final boolean incompatible = randomBoolean(); + IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder() + .put(SETTING_VERSION_CREATED, incompatible ? VersionUtils.getPreviousVersion(Version.CURRENT.minimumIndexCompatibilityVersion()) + : VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumIndexCompatibilityVersion(), Version.CURRENT)) + .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0) + .put(SETTING_CREATION_DATE, System.currentTimeMillis())) + .state(IndexMetaData.State.OPEN) + .build(); + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(indexMetaData.getIndex()); + RoutingTable.Builder routing = new RoutingTable.Builder(); + routing.addAsNew(indexMetaData); + final ShardId shardId = new ShardId("test", "_na_", 0); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + + final DiscoveryNode primaryNode = otherNode; + indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting("test", 0, primaryNode.getId(), null, true, + ShardRoutingState.INITIALIZING, new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "getting there"))); + indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build()); + IndexRoutingTable indexRoutingTable = indexRoutingTableBuilder.build(); + IndexMetaData updatedIndexMetaData = updateActiveAllocations(indexRoutingTable, indexMetaData); + stateBuilder.metaData(MetaData.builder().put(updatedIndexMetaData, false).generateClusterUuidIfNeeded()) + .routingTable(RoutingTable.builder().add(indexRoutingTable).build()); + if (incompatible) { + IllegalStateException ex = expectThrows(IllegalStateException.class, () -> + request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), null)); + assertEquals("index [test] version not supported: " + + VersionUtils.getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion()) + + " minimum compatible index version is: " + Version.CURRENT.minimumCompatibilityVersion(), ex.getMessage()); + } else { + AtomicBoolean sendResponse = new AtomicBoolean(false); + request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), new TransportChannel() { + @Override + public String action() { + return null; + } + + @Override + public String getProfileName() { + return null; + } + + @Override + public long getRequestId() { + return 0; + } + + @Override + public String getChannelType() { + return null; + } + + @Override + public void sendResponse(TransportResponse response) throws IOException { + sendResponse.set(true); + } + + @Override + public void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException { + + } + + @Override + public void sendResponse(Exception exception) throws IOException { + + } + }); + assertTrue(sendResponse.get()); + } + } + } } diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index de510eb293f01..bed21193ac69b 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -414,7 +414,7 @@ public void testRecoverBrokenIndexMetadata() throws Exception { IndexMetaData metaData = state.getMetaData().index("test"); for (NodeEnvironment services : internalCluster().getInstances(NodeEnvironment.class)) { IndexMetaData brokenMeta = IndexMetaData.builder(metaData).settings(Settings.builder().put(metaData.getSettings()) - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_5_0_0_beta1.id) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT.minimumIndexCompatibilityVersion().id) // this is invalid but should be archived .put("index.similarity.BM25.type", "classic") // this one is not validated ahead of time and breaks allocation diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java b/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java index 2a37a7f0a6004..b106ca64227cc 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java @@ -396,7 +396,7 @@ public MockMetaDataIndexUpgradeService(boolean upgrade) { this.upgrade = upgrade; } @Override - public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) { + public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { return upgrade ? IndexMetaData.builder(indexMetaData).build() : indexMetaData; } } diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index fca09e743322d..d1d266a42a466 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices.cluster; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteRequest; import org.elasticsearch.action.admin.cluster.reroute.TransportClusterRerouteAction; @@ -158,7 +159,7 @@ public ClusterStateChanges() { MetaDataIndexUpgradeService metaDataIndexUpgradeService = new MetaDataIndexUpgradeService(settings, null, null) { // metaData upgrader should do nothing @Override - public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) { + public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { return indexMetaData; } }; diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESBackcompatTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESBackcompatTestCase.java index 1a7aac925f30e..bc5aa63bf82f6 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESBackcompatTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESBackcompatTestCase.java @@ -142,7 +142,7 @@ private Version compatibilityVersion(Class clazz) { } CompatibilityVersion annotation = clazz.getAnnotation(CompatibilityVersion.class); if (annotation != null) { - return Version.smallest(Version.fromId(annotation.version()), compatibilityVersion(clazz.getSuperclass())); + return Version.min(Version.fromId(annotation.version()), compatibilityVersion(clazz.getSuperclass())); } return compatibilityVersion(clazz.getSuperclass()); }