From 5965ab346bd284023b8aaccbf84ec365efb74ce2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 28 Nov 2016 17:22:07 +0100 Subject: [PATCH 01/10] Add validation for supported index version on node join Today we can easily join a cluster that holds an index we don't support since we currently allow rolling upgrades from 5.x to 6.x. This commit adds additional safety that fails cluster state validation if there is an open index with an incompatible index version created in the cluster. Realtes to #21670 --- .../main/java/org/elasticsearch/Version.java | 15 +++ .../discovery/zen/MembershipAction.java | 38 ++++++-- .../discovery/zen/ZenDiscovery.java | 2 +- .../java/org/elasticsearch/VersionTests.java | 6 ++ .../discovery/zen/ZenDiscoveryUnitTests.java | 94 +++++++++++++++++++ 5 files changed, 144 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index 981cca72d4437..4f63366022b9c 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -324,6 +324,21 @@ public Version minimumCompatibilityVersion() { return Version.smallest(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.smallest(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); + } + /** * Returns true iff both version are compatible. Otherwise false */ 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..8e668a35b3636 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,21 @@ 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 +155,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,13 +181,25 @@ 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 { + MetaData metaData = request.state.getMetaData(); + ensureAllIndicesAreCompatible(metaData); // for now, the mere fact that we can serialize the cluster state acts as validation.... channel.sendResponse(TransportResponse.Empty.INSTANCE); } + + void ensureAllIndicesAreCompatible(MetaData metaData) { + for (IndexMetaData idxMetaData : metaData) { + if(idxMetaData.getState() == IndexMetaData.State.OPEN && + idxMetaData.getCreationVersion().before(Version.CURRENT.minimumIndexCompatibilityVersion())) { + throw new IllegalStateException("index " + idxMetaData.getIndex() + " version not supported: " + + idxMetaData.getCreationVersion()); + } + } + } } public static class LeaveRequest extends TransportRequest { diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index a15c59027057a..41053705ec953 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -183,7 +183,7 @@ public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService t new NewPendingClusterStateListener(), discoverySettings, clusterService.getClusterName()); - this.membership = new MembershipAction(settings, transportService, this, new MembershipListener()); + this.membership = new MembershipAction(settings, transportService, this::localNode, new MembershipListener()); this.joinThreadControl = new JoinThreadControl(threadPool); transportService.registerRequestHandler( diff --git a/core/src/test/java/org/elasticsearch/VersionTests.java b/core/src/test/java/org/elasticsearch/VersionTests.java index 3bccee941b335..18fddaab80635 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -64,7 +64,13 @@ 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 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_0_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/discovery/zen/ZenDiscoveryUnitTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index acc5d4e801868..ac5764e30f2db 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,77 @@ private Set fdNodesForState(ClusterState clusterState, DiscoveryN }); return discoveryNodes; } + + public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { + 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 closed = randomBoolean(); + IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder() + .put(SETTING_VERSION_CREATED, VersionUtils.getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion())) + .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0) + .put(SETTING_CREATION_DATE, System.currentTimeMillis())) + .state(closed ? IndexMetaData.State.CLOSE : 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 (closed == false) { + IllegalStateException ex = expectThrows(IllegalStateException.class, () -> + request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), null)); + assertEquals("index [test] version not supported: " + + VersionUtils.getPreviousVersion(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()); + } + } } From d9aed75b6cd882e8fc744a3cbf1455016cd240aa Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 29 Nov 2016 14:01:03 +0100 Subject: [PATCH 02/10] don't allow closed indices either --- .../discovery/zen/MembershipAction.java | 4 +- .../discovery/zen/ZenDiscoveryUnitTests.java | 142 +++++++++--------- 2 files changed, 74 insertions(+), 72 deletions(-) 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 8e668a35b3636..0cd6fbfa50273 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java @@ -61,7 +61,6 @@ public interface MembershipListener { private final TransportService transportService; - private final MembershipListener listener; public MembershipAction(Settings settings, TransportService transportService, @@ -193,8 +192,7 @@ public void messageReceived(ValidateJoinRequest request, TransportChannel channe void ensureAllIndicesAreCompatible(MetaData metaData) { for (IndexMetaData idxMetaData : metaData) { - if(idxMetaData.getState() == IndexMetaData.State.OPEN && - idxMetaData.getCreationVersion().before(Version.CURRENT.minimumIndexCompatibilityVersion())) { + if(idxMetaData.getCreationVersion().before(Version.CURRENT.minimumIndexCompatibilityVersion())) { throw new IllegalStateException("index " + idxMetaData.getIndex() + " version not supported: " + idxMetaData.getCreationVersion()); } 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 ac5764e30f2db..cb832830b91af 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -306,75 +306,79 @@ private Set fdNodesForState(ClusterState clusterState, DiscoveryN } public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { - 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 closed = randomBoolean(); - IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder() - .put(SETTING_VERSION_CREATED, VersionUtils.getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion())) - .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0) - .put(SETTING_CREATION_DATE, System.currentTimeMillis())) - .state(closed ? IndexMetaData.State.CLOSE : 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 (closed == false) { - IllegalStateException ex = expectThrows(IllegalStateException.class, () -> - request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), null)); - assertEquals("index [test] version not supported: " - + VersionUtils.getPreviousVersion(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()); + 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()), 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()); + } } } } From 5c9a61fad7b08a9b80a31ee3ab5821fdfd887a76 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 29 Nov 2016 16:34:52 +0100 Subject: [PATCH 03/10] add more comments, inline method and add a more informative error message --- .../discovery/zen/MembershipAction.java | 18 ++++++++---------- .../discovery/zen/ZenDiscoveryUnitTests.java | 3 ++- 2 files changed, 10 insertions(+), 11 deletions(-) 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 0cd6fbfa50273..e403e6c10ec4c 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java @@ -184,19 +184,17 @@ static class ValidateJoinRequestRequestHandler implements TransportRequestHandle @Override public void messageReceived(ValidateJoinRequest request, TransportChannel channel) throws Exception { - MetaData metaData = request.state.getMetaData(); - ensureAllIndicesAreCompatible(metaData); - // for now, the mere fact that we can serialize the cluster state acts as validation.... - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } - - void ensureAllIndicesAreCompatible(MetaData metaData) { - for (IndexMetaData idxMetaData : metaData) { - if(idxMetaData.getCreationVersion().before(Version.CURRENT.minimumIndexCompatibilityVersion())) { + // 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... + final Version supportedIndexVersion = Version.CURRENT.minimumIndexCompatibilityVersion(); + for (IndexMetaData idxMetaData : request.state.getMetaData()) { + if(idxMetaData.getCreationVersion().before(supportedIndexVersion)) { throw new IllegalStateException("index " + idxMetaData.getIndex() + " version not supported: " - + idxMetaData.getCreationVersion()); + + idxMetaData.getCreationVersion() + " minimum compatible index version is: " + supportedIndexVersion); } } + // for now, the mere fact that we can serialize the cluster state acts as validation.... + channel.sendResponse(TransportResponse.Empty.INSTANCE); } } 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 cb832830b91af..b46df47afbcd5 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -338,7 +338,8 @@ public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { IllegalStateException ex = expectThrows(IllegalStateException.class, () -> request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), null)); assertEquals("index [test] version not supported: " - + VersionUtils.getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion()), ex.getMessage()); + + 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() { From f63bb988bd06d4cfdd4f0a467a1236e966f1cf06 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 29 Nov 2016 17:28:07 +0100 Subject: [PATCH 04/10] Add more validation into several places that execute on the cluster state thread to ensure we don't have a race --- .../discovery/zen/MembershipAction.java | 25 ++++++++++++------- .../discovery/zen/NodeJoinController.java | 7 ++++-- .../discovery/zen/ZenDiscovery.java | 4 ++- .../gateway/LocalAllocateDangledIndices.java | 12 ++++++++- 4 files changed, 35 insertions(+), 13 deletions(-) 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 e403e6c10ec4c..c852716819820 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java @@ -184,20 +184,27 @@ static class ValidateJoinRequestRequestHandler implements TransportRequestHandle @Override public void messageReceived(ValidateJoinRequest request, TransportChannel channel) throws Exception { - // 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... - final Version supportedIndexVersion = Version.CURRENT.minimumIndexCompatibilityVersion(); - for (IndexMetaData idxMetaData : request.state.getMetaData()) { - if(idxMetaData.getCreationVersion().before(supportedIndexVersion)) { - throw new IllegalStateException("index " + idxMetaData.getIndex() + " version not supported: " - + idxMetaData.getCreationVersion() + " minimum compatible index version is: " + supportedIndexVersion); - } - } + 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..2b5ec3c940d06 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 Date: Wed, 30 Nov 2016 09:45:23 +0100 Subject: [PATCH 05/10] move old 2.x version to unsupported --- .../bwcompat/OldIndexBackwardsCompatibilityIT.java | 2 +- ...-beta1.zip => unsupported-index-2.0.0-beta1.zip} | Bin ...-beta2.zip => unsupported-index-2.0.0-beta2.zip} | Bin ....0.0-rc1.zip => unsupported-index-2.0.0-rc1.zip} | Bin ...{index-2.0.0.zip => unsupported-index-2.0.0.zip} | Bin ...{index-2.0.1.zip => unsupported-index-2.0.1.zip} | Bin ...{index-2.0.2.zip => unsupported-index-2.0.2.zip} | Bin ...{index-2.1.0.zip => unsupported-index-2.1.0.zip} | Bin ...{index-2.1.1.zip => unsupported-index-2.1.1.zip} | Bin ...{index-2.1.2.zip => unsupported-index-2.1.2.zip} | Bin ...{index-2.2.0.zip => unsupported-index-2.2.0.zip} | Bin ...{index-2.2.1.zip => unsupported-index-2.2.1.zip} | Bin ...{index-2.2.2.zip => unsupported-index-2.2.2.zip} | Bin ...{index-2.3.0.zip => unsupported-index-2.3.0.zip} | Bin ...{index-2.3.1.zip => unsupported-index-2.3.1.zip} | Bin ...{index-2.3.2.zip => unsupported-index-2.3.2.zip} | Bin ...{index-2.3.3.zip => unsupported-index-2.3.3.zip} | Bin ...{index-2.3.4.zip => unsupported-index-2.3.4.zip} | Bin ...{index-2.3.5.zip => unsupported-index-2.3.5.zip} | Bin ...{index-2.4.0.zip => unsupported-index-2.4.0.zip} | Bin ...{index-2.4.1.zip => unsupported-index-2.4.1.zip} | Bin ...{index-2.4.2.zip => unsupported-index-2.4.2.zip} | Bin ...p => unsupportedmissing-checksum-repo-2.3.4.zip} | Bin ....0-beta1.zip => unsupportedrepo-2.0.0-beta1.zip} | Bin ....0-beta2.zip => unsupportedrepo-2.0.0-beta2.zip} | Bin ...-2.0.0-rc1.zip => unsupportedrepo-2.0.0-rc1.zip} | Bin .../{repo-2.0.0.zip => unsupportedrepo-2.0.0.zip} | Bin .../{repo-2.0.1.zip => unsupportedrepo-2.0.1.zip} | Bin .../{repo-2.0.2.zip => unsupportedrepo-2.0.2.zip} | Bin .../{repo-2.1.0.zip => unsupportedrepo-2.1.0.zip} | Bin .../{repo-2.1.1.zip => unsupportedrepo-2.1.1.zip} | Bin .../{repo-2.1.2.zip => unsupportedrepo-2.1.2.zip} | Bin .../{repo-2.2.0.zip => unsupportedrepo-2.2.0.zip} | Bin .../{repo-2.2.1.zip => unsupportedrepo-2.2.1.zip} | Bin .../{repo-2.2.2.zip => unsupportedrepo-2.2.2.zip} | Bin .../{repo-2.3.0.zip => unsupportedrepo-2.3.0.zip} | Bin .../{repo-2.3.1.zip => unsupportedrepo-2.3.1.zip} | Bin .../{repo-2.3.2.zip => unsupportedrepo-2.3.2.zip} | Bin .../{repo-2.3.3.zip => unsupportedrepo-2.3.3.zip} | Bin .../{repo-2.3.4.zip => unsupportedrepo-2.3.4.zip} | Bin .../{repo-2.3.5.zip => unsupportedrepo-2.3.5.zip} | Bin .../{repo-2.4.0.zip => unsupportedrepo-2.4.0.zip} | Bin .../{repo-2.4.1.zip => unsupportedrepo-2.4.1.zip} | Bin .../{repo-2.4.2.zip => unsupportedrepo-2.4.2.zip} | Bin 44 files changed, 1 insertion(+), 1 deletion(-) rename core/src/test/resources/indices/bwc/{index-2.0.0-beta1.zip => unsupported-index-2.0.0-beta1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.0.0-beta2.zip => unsupported-index-2.0.0-beta2.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.0.0-rc1.zip => unsupported-index-2.0.0-rc1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.0.0.zip => unsupported-index-2.0.0.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.0.1.zip => unsupported-index-2.0.1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.0.2.zip => unsupported-index-2.0.2.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.1.0.zip => unsupported-index-2.1.0.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.1.1.zip => unsupported-index-2.1.1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.1.2.zip => unsupported-index-2.1.2.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.2.0.zip => unsupported-index-2.2.0.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.2.1.zip => unsupported-index-2.2.1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.2.2.zip => unsupported-index-2.2.2.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.3.0.zip => unsupported-index-2.3.0.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.3.1.zip => unsupported-index-2.3.1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.3.2.zip => unsupported-index-2.3.2.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.3.3.zip => unsupported-index-2.3.3.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.3.4.zip => unsupported-index-2.3.4.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.3.5.zip => unsupported-index-2.3.5.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.4.0.zip => unsupported-index-2.4.0.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.4.1.zip => unsupported-index-2.4.1.zip} (100%) rename core/src/test/resources/indices/bwc/{index-2.4.2.zip => unsupported-index-2.4.2.zip} (100%) rename core/src/test/resources/indices/bwc/{missing-checksum-repo-2.3.4.zip => unsupportedmissing-checksum-repo-2.3.4.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.0.0-beta1.zip => unsupportedrepo-2.0.0-beta1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.0.0-beta2.zip => unsupportedrepo-2.0.0-beta2.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.0.0-rc1.zip => unsupportedrepo-2.0.0-rc1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.0.0.zip => unsupportedrepo-2.0.0.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.0.1.zip => unsupportedrepo-2.0.1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.0.2.zip => unsupportedrepo-2.0.2.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.1.0.zip => unsupportedrepo-2.1.0.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.1.1.zip => unsupportedrepo-2.1.1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.1.2.zip => unsupportedrepo-2.1.2.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.2.0.zip => unsupportedrepo-2.2.0.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.2.1.zip => unsupportedrepo-2.2.1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.2.2.zip => unsupportedrepo-2.2.2.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.3.0.zip => unsupportedrepo-2.3.0.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.3.1.zip => unsupportedrepo-2.3.1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.3.2.zip => unsupportedrepo-2.3.2.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.3.3.zip => unsupportedrepo-2.3.3.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.3.4.zip => unsupportedrepo-2.3.4.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.3.5.zip => unsupportedrepo-2.3.5.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.4.0.zip => unsupportedrepo-2.4.0.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.4.1.zip => unsupportedrepo-2.4.1.zip} (100%) rename core/src/test/resources/indices/bwc/{repo-2.4.2.zip => unsupportedrepo-2.4.2.zip} (100%) diff --git a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java index cea041d77772d..a86b92d0bab33 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityIT.java @@ -188,7 +188,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.onOrBefore(Version.V_2_0_0_beta1)) continue; // we can only test back one major lucene version + if (v.onOrBefore(Version.V_5_0_0_beta1)) continue; // we can only test back one major lucene version 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/resources/indices/bwc/index-2.0.0-beta1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.0.0-beta1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.0.0-beta1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.0.0-beta1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.0.0-beta2.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.0.0-beta2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.0.0-beta2.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.0.0-beta2.zip diff --git a/core/src/test/resources/indices/bwc/index-2.0.0-rc1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.0.0-rc1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.0.0-rc1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.0.0-rc1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.0.0.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.0.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.0.0.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.0.0.zip diff --git a/core/src/test/resources/indices/bwc/index-2.0.1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.0.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.0.1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.0.1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.0.2.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.0.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.0.2.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.0.2.zip diff --git a/core/src/test/resources/indices/bwc/index-2.1.0.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.1.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.1.0.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.1.0.zip diff --git a/core/src/test/resources/indices/bwc/index-2.1.1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.1.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.1.1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.1.1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.1.2.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.1.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.1.2.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.1.2.zip diff --git a/core/src/test/resources/indices/bwc/index-2.2.0.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.2.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.2.0.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.2.0.zip diff --git a/core/src/test/resources/indices/bwc/index-2.2.1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.2.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.2.1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.2.1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.2.2.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.2.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.2.2.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.2.2.zip diff --git a/core/src/test/resources/indices/bwc/index-2.3.0.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.3.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.3.0.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.3.0.zip diff --git a/core/src/test/resources/indices/bwc/index-2.3.1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.3.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.3.1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.3.1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.3.2.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.3.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.3.2.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.3.2.zip diff --git a/core/src/test/resources/indices/bwc/index-2.3.3.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.3.3.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.3.3.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.3.3.zip diff --git a/core/src/test/resources/indices/bwc/index-2.3.4.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.3.4.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.3.4.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.3.4.zip diff --git a/core/src/test/resources/indices/bwc/index-2.3.5.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.3.5.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.3.5.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.3.5.zip diff --git a/core/src/test/resources/indices/bwc/index-2.4.0.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.4.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.4.0.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.4.0.zip diff --git a/core/src/test/resources/indices/bwc/index-2.4.1.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.4.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.4.1.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.4.1.zip diff --git a/core/src/test/resources/indices/bwc/index-2.4.2.zip b/core/src/test/resources/indices/bwc/unsupported-index-2.4.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/index-2.4.2.zip rename to core/src/test/resources/indices/bwc/unsupported-index-2.4.2.zip diff --git a/core/src/test/resources/indices/bwc/missing-checksum-repo-2.3.4.zip b/core/src/test/resources/indices/bwc/unsupportedmissing-checksum-repo-2.3.4.zip similarity index 100% rename from core/src/test/resources/indices/bwc/missing-checksum-repo-2.3.4.zip rename to core/src/test/resources/indices/bwc/unsupportedmissing-checksum-repo-2.3.4.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.0.0-beta1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0-beta1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.0.0-beta1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0-beta1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.0.0-beta2.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0-beta2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.0.0-beta2.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0-beta2.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.0.0-rc1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0-rc1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.0.0-rc1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0-rc1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.0.0.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.0.0.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.0.0.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.0.1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.0.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.0.1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.0.1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.0.2.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.0.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.0.2.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.0.2.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.1.0.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.1.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.1.0.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.1.0.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.1.1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.1.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.1.1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.1.1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.1.2.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.1.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.1.2.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.1.2.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.2.0.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.2.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.2.0.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.2.0.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.2.1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.2.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.2.1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.2.1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.2.2.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.2.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.2.2.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.2.2.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.3.0.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.3.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.3.0.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.3.0.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.3.1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.3.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.3.1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.3.1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.3.2.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.3.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.3.2.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.3.2.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.3.3.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.3.3.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.3.3.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.3.3.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.3.4.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.3.4.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.3.4.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.3.4.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.3.5.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.3.5.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.3.5.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.3.5.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.4.0.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.4.0.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.4.0.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.4.0.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.4.1.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.4.1.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.4.1.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.4.1.zip diff --git a/core/src/test/resources/indices/bwc/repo-2.4.2.zip b/core/src/test/resources/indices/bwc/unsupportedrepo-2.4.2.zip similarity index 100% rename from core/src/test/resources/indices/bwc/repo-2.4.2.zip rename to core/src/test/resources/indices/bwc/unsupportedrepo-2.4.2.zip From c709e34cc4c58b78861be0a9a70436f2a57a787e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 30 Nov 2016 12:14:06 +0100 Subject: [PATCH 06/10] Ensure we can't restore or open any indices that are not supported --- .../metadata/MetaDataIndexStateService.java | 5 ++- .../metadata/MetaDataIndexUpgradeService.java | 23 +++++----- .../gateway/GatewayMetaState.java | 3 +- .../gateway/LocalAllocateDangledIndices.java | 3 +- .../snapshots/RestoreService.java | 6 ++- .../bwcompat/RestoreBackwardsCompatIT.java | 42 +----------------- .../MetaDataIndexUpgradeServiceTests.java | 9 ++-- .../gateway/GatewayMetaStateTests.java | 2 +- .../indices/cluster/ClusterStateChanges.java | 3 +- ...unsupportedmissing-checksum-repo-2.3.4.zip | Bin 11371 -> 0 bytes 10 files changed, 35 insertions(+), 61 deletions(-) delete mode 100644 core/src/test/resources/indices/bwc/unsupportedmissing-checksum-repo-2.3.4.zip 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..b998c3542a21c 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()); + Version minIndexCompatibilityVersion = currentState.getNodes().getSmallestNonClientNodeVersion() + .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 b261b8850c6f6..d0e882c7791e4 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,22 @@ 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 + * Elasticsearch 6.0 no longer supports indices with pre Lucene v5.0 (Elasticsearch v5.0.0.beta1) segments. All indices + * that were created before Elasticsearch v5.0.0.beta1 should be reindexed in Elasticsearch 5.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 + "."); + 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 before v5.0.0.beta1." + + " It should be reindexed in Elasticsearch 5.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_2_0_0_beta1); + private static boolean isSupportedVersion(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { + return indexMetaData.getCreationVersion().onOrAfter(minimumIndexCompatibilityVersion); } /** @@ -173,4 +174,4 @@ IndexMetaData archiveBrokenIndexSettings(IndexMetaData indexMetaData) { return indexMetaData; } } -} \ No newline at end of file +} diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index b609d0bacaed9..98b39cd2c8ae3 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -245,7 +245,8 @@ static MetaData upgradeMetaData(MetaData metaData, boolean changed = false; final MetaData.Builder upgradedMetaData = MetaData.builder(metaData); for (IndexMetaData indexMetaData : metaData) { - IndexMetaData newMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData); + IndexMetaData newMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData, + Version.CURRENT.minimumIndexCompatibilityVersion()); changed |= indexMetaData != newMetaData; upgradedMetaData.put(newMetaData, false); } diff --git a/core/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java b/core/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java index cb4413e7d7e61..b0328447c02a5 100644 --- a/core/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java +++ b/core/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java @@ -154,7 +154,8 @@ public ClusterState execute(ClusterState currentState) { try { // The dangled index might be from an older version, we need to make sure it's compatible // with the current version and upgrade it if needed. - upgradedIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData); + upgradedIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData, + minIndexCompatibilityVersion); } catch (Exception ex) { // upgrade failed - adding index as closed logger.warn((Supplier) () -> 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..ffaf20af03c41 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(); + Version minIndexCompatibilityVersion = currentState.getNodes().getSmallestNonClientNodeVersion() + .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/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java b/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java index 16321dbd9fec7..875ce03c58d74 100644 --- a/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java +++ b/core/src/test/java/org/elasticsearch/bwcompat/RestoreBackwardsCompatIT.java @@ -98,7 +98,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.onOrBefore(Version.V_2_0_0_beta1)) continue; // we can only test back one major lucene version + if (v.onOrBefore(Version.V_5_0_0_beta1)) continue; // we can only test back one major lucene version if (v.equals(Version.CURRENT)) continue; // the current version is always compatible with itself expectedVersions.add(v.toString()); } @@ -128,44 +128,6 @@ public void testRestoreUnsupportedSnapshots() throws Exception { } } - public void testRestoreSnapshotWithMissingChecksum() throws Exception { - final String repo = "test_repo"; - final String snapshot = "test_1"; - final String indexName = "index-2.3.4"; - final String repoFileId = "missing-checksum-repo-2.3.4"; - Path repoFile = getBwcIndicesPath().resolve(repoFileId + ".zip"); - URI repoFileUri = repoFile.toUri(); - URI repoJarUri = new URI("jar:" + repoFileUri.toString() + "!/repo/"); - logger.info("--> creating repository [{}] for repo file [{}]", repo, repoFileId); - assertAcked(client().admin().cluster().preparePutRepository(repo) - .setType("url") - .setSettings(Settings.builder().put("url", repoJarUri.toString()))); - - logger.info("--> get snapshot and check its indices"); - GetSnapshotsResponse getSnapshotsResponse = client().admin().cluster().prepareGetSnapshots(repo).setSnapshots(snapshot).get(); - assertThat(getSnapshotsResponse.getSnapshots().size(), equalTo(1)); - SnapshotInfo snapshotInfo = getSnapshotsResponse.getSnapshots().get(0); - assertThat(snapshotInfo.indices(), equalTo(Arrays.asList(indexName))); - - logger.info("--> restoring snapshot"); - RestoreSnapshotResponse response = client().admin().cluster().prepareRestoreSnapshot(repo, snapshot).setRestoreGlobalState(true).setWaitForCompletion(true).get(); - assertThat(response.status(), equalTo(RestStatus.OK)); - RestoreInfo restoreInfo = response.getRestoreInfo(); - assertThat(restoreInfo.successfulShards(), greaterThan(0)); - assertThat(restoreInfo.successfulShards(), equalTo(restoreInfo.totalShards())); - assertThat(restoreInfo.failedShards(), equalTo(0)); - String index = restoreInfo.indices().get(0); - assertThat(index, equalTo(indexName)); - - logger.info("--> check search"); - SearchResponse searchResponse = client().prepareSearch(index).get(); - assertThat(searchResponse.getHits().totalHits(), greaterThan(0L)); - - logger.info("--> cleanup"); - cluster().wipeIndices(restoreInfo.indices().toArray(new String[restoreInfo.indices().size()])); - cluster().wipeTemplates(); - } - private List repoVersions() throws Exception { return listRepoVersions("repo"); } @@ -245,7 +207,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 376feb305a171..1a36708f321bd 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,7 +87,8 @@ public void testFailUpgrade() { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.fromString("1.7.0")) .put(IndexMetaData.SETTING_VERSION_MINIMUM_COMPATIBLE, Version.CURRENT.luceneVersion.toString()).build()); - String message = expectThrows(IllegalStateException.class, () -> service.upgradeIndexMetaData(metaData)).getMessage(); + String message = expectThrows(IllegalStateException.class, () -> service.upgradeIndexMetaData(metaData, + Version.V_5_0_0.minimumIndexCompatibilityVersion())).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() + "."); @@ -96,7 +97,7 @@ public void testFailUpgrade() { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.fromString("2.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/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/core/src/test/resources/indices/bwc/unsupportedmissing-checksum-repo-2.3.4.zip b/core/src/test/resources/indices/bwc/unsupportedmissing-checksum-repo-2.3.4.zip deleted file mode 100644 index 9590f8dbd660fd56617c8e55b6bc73276aae4744..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11371 zcmdT~2{_bU`!*)V{@Rxi#+qzn%a$c3#=d4LYax>?k&HDD(G^ zIB+u`emP-;s*<0_IItg+E@0eFY=!>!j+mz>o9U;=T5o!C2G!a6t&mq(g1=(9y?DB= z<)v}nvUf;kaAenb_hd`Dcg$5scesxW+zRGiXM0u`Zsh|DBh%7yL>M^0U~nrxL)5kK z%%Dsabr(3y&hMC=dzrMt z$Ok+p4HHsK)8n%aWM!r%TO^ppA+y@gbdWO%3-@T#Toddf6W0_!Pen!XmGTors)(eQ z@~V*14af>nce-RU>|8^jSL2ayg%vy%)jY30CO7{P^C4h;eT~(8rREMd2(Xd)zoPiL zc>N_CCW6|SM-X=uH;5#>MIe#Www`#T#a1+%Fln(BOj%LL;b&XpD&dz9<*fkM3svOH zNGN^KcXWr@8NmEt4lV_PIZ0Kg)VgjuP|p}!TRFKH!VUZaEc`4Sbm0rxy>Y8?E$PeS z-37)BNlA<)W6{=Bx(f@$M~GMwTalg0<}uLBR`gsA8o}GT5LP6MM$GZDD1XVAf+Bf& z(TD|Dq6nW4u@5nO4VpK6fhD|^B@mhV_CaqL94MDoM62w|+`=3hfrf_jDsw>gFE&B7v?PpLZbsY;4*`bwH~VSp z918XTf*<~IMmqo?!QW$VIQ${2A@=gu-l1ytU8@={-mD z;^a3(6c2ujyf;csPfdoy$I`P-#8{^$k7Kx=fVAi)*f;Vpjzi&9gNf^#rm*l&X%!PJ z)hiR(T{`2GpAEatvv4Z^p*ec_A?u~dPw`w$kuUXE6zEstwDSYnGzHd3p!J*$%-~)& z%4ZynH>B2x?@BhIOzF9>g?75orA9Y>h&2Qprl?&zZ>Nn@C& z4AYJIqJOxoK8WlXfyuNA(csA&y&4J|954nngTGoH-L+UCoWdJ9FwtU)=1XM+1Sz1nqb}iIxXe)7{_hu2N{WGN%^KpsmXEH zJd@T5De3VAW9in$G1gmD2PTRL5}%i$_#*U~V!Wq&nd@ea7^yC|(GdxW5gt{KdbW~6 z8`|5qd~W#=zg=nd35RyFI5RLhEI9CiZAPFd1TV72Em1+NOyyPy!%Q+GZ{FQoQwfZY zZh#A_45h65XIb{oBhLqWGA8p(Gk)Bknu6S)mWpI%WwA(=C~IY%<;_FgB}9*FljNls z95`Uey8=ZYbU~tJ54O6jKx@KzeQH(?qEG7HdBC;s0C4Ay2uPBz;Zhh~NW>#1(5|qn z$P~;P0|^K!NbyKSaa>{VG3n8xQ=uP4uE7BZv%rKFWFR`4;g1HFQv@ZY@&&?zN%<@X zyoMwzEuV}Cs_CSCq-#wX5~Q7p+fq`Z_PUleVmE+J4RU@jox5}SJu;mGw3 zyB|G6U_I8RUL}vd<{G5-@vdJE4-^bq>9x8y=r2S?JE&+68GrP4dvHxR_jF{$vNuN4PB>rFi4?M1h@&k6(QK;j35mMvie+w~k%O zVJfDa8~*xQGcUGUH*FZ{OV>Nx5yBd>d}VA{K=CZUr#n{(7Y8>9waeXVAsa;sgU~{@ z>0Hnax)!SeB8p2ZF-ingoyiRRhh!f;5T>Dn#P`I9JoF;etBe2PN9xnV@;>#OqUFuZ zkETd9+dKz_;VXTj!KP`?XKM;#f=lO(%I5f|)nigB4L-W-z;ZM=uhY^ide8(nSnk+yUf~KTj(pSF(H7t$RVQ;gd?#g)(G#OSLKMbix!_e*dI{ z4BXfHhN}lrh=)aWCab-q@{#gbW8>nFO(o8rZPceu>CO;SWv;f(P7k%|TeRQIplb}Z zFOn}%a_%326gVW<7~HoyBqK!^)#ex^MP<^L$5nGZvd}4u;vr&P-gW`OGWI>HP{Gwu zmaTc+QfNYhj%qIuN1lOC$ogr<8+|g=(OFjoXgm#+ zB#;h=Rp$yoie>nfjwrI{=LnSYNGL@=zRDzGVUdaiC*{8}H=nX)q&Gvdnq+1%0sF*;LoZ{~5)s;3D3RgiB>Q^IL+mg)H^u)L4+3o0-uCmUqh zzfalwaT#&TzzrZ+4WOX@))Iu^s}Mw)>PeA1Ey2bL^gV@Gl-_)oMs)zwfY5QC>!Tht zmm7my_h80NgnWv!9>mCBLXBC(oyrry}qzA&=cfa6&)4!syHnO z@#Om3?;ITQRS{_RTodyjxAUV?xxVDNu)Wv$(R3y`j0<2}?n9l%91xtsUVB02K(t;2 zQ$v8$42g1yJ6DQ6_choOLBVuMg6=z!ABP~HZ2u?06c zi%}f}ylpBH&Wp zV9AMYNU+OiE>5km%Li&np4Vi27{QRCqq^PC)$m}Y^93RoNp!PplMUMB36Do%JAA?kyY z;yVp_*AGg8PwUADg?Rb+g;Y9Fqz`*lPPx@fSHHMNVVYeYQ39JqBFHG2C|_}ws|ZX6 z9r1_BFA5fFKw-qS0dmLg>*lCGn#;I<)gDB8V2Xn(*`Qr1OOn9-EyhhxkOYE3nJC&2$V0Q6QPiDK)uN^8jvL6aDYe`lA=SJ>2q#EQ)8V zTq^72b?l;2d+bJvIWBUqxhyrWXRS^24K$5D|M7Q^IpTXb=5#(+)x}H35Xa}3gImUk zz*ELZxm*~orEKo5Z=-zEDJ)hGny(Up@Y z`tK{>mt0Rzy?GumLb)r9GYZ1CcoAIUpCGu}KTL2TJPF=3;USnoxJn14Jk)$mhdj8D z_S$_z;FeD5I6lg)-4M8bj8U$}u2bpn68!hgjbD4;#m}>fcoAIkzbE*0d;!8viiA+3@5y zSSPBC-5(rzG)^!>;z>wXoF(@)@HW5IOUC^>H~SDrk4L=Pt<)Z^E4#A~BaX{p9syi^ zpmX+~wtMLUB_osLEV9$mEaFn)%;Gx!-3uZIyW~3Fc3dnh+HL`9Xkl7FumX<&6}XkJ zI~)dcu<}_bhS{KeI-zJxy9K(S>|y9nq0_A$rPHGwNh#DVoTq9$nK;ddVr-qsYh^+) zPM+rDXOw5wg(eI3Y6xi_^b&g)s*#Z7a4YoqlzP|-4?`QsVtFP-E)_JXcAT;HmR?Z) zrCLh6KzKx2Y*1-XIBM#{*Ox89gfYOtRk#pj^_jW;K2I+Lm;-#F_*Tp3e(N6IL?U#r z>By{$1@f|a<<&eD0`=vR61?S4Zb%M`_Ou=r z04sDWDJm*-fNx83m*ntBRdMo{@>DE8|1jFHu?0LJAmUTfLF}rm18QT|P?+I1?HuwR zbcesFq1pKs3wO2y%eJyWR8azYSyrjCNEp)|l#s1hzQPm}Ymt3+GgZn^)jXv<&xB?A z2M48Os_8lo35JVycI%kf_b+-yYRsM0>(NqSeDTyW&N?oI&N@BTEWywZe$?GbysvM( z+^|~T*TT+0*NLX@R#rnGM_{-MF$Wb0Fq%`{IjbH8#fL%UepBdR5sqX zFD?*mU0m38;b6BZ>}v~KrZ5~M$PH5ZX=?it1Ag{42JrJmxp^64V%WQRRxI;Um_T5Sc&GV|3k|pcew7i}*BO4!)<%dUa06^M zKk>uBzHP7*Lk}a>;c#NhCCuFe{BW@Q@jG!6F>)Xdod2LCF1$a_bnMAtAi{yS(T&Ch z@N06gd(k@`g5w6*!{lIhcXwj=5#w~i+8WdA#m^7yR_0EOPoUpmU>ckFVPLmlc4B0a z{{{ookiicFy9u%rBZvYg#vT(HTgBdqBk%`Kod5GgUZuf-x1l%U7dULSai>EDv^W9w zkjU6#;7$xiI-D4LoE&VqY$wi140Hc-mH89#3|H*%S!G@_;K17ui*Nz_T4k`MqMZ&w z7;ysZVREpQoShh7abxUpa Date: Wed, 30 Nov 2016 15:44:56 +0100 Subject: [PATCH 07/10] Fix test --- .../java/org/elasticsearch/gateway/GatewayIndexStateIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 From 0d1be73490d6c59deab3e79550735fa2c73d4113 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 30 Nov 2016 20:40:32 +0100 Subject: [PATCH 08/10] more feedback --- .../cluster/metadata/MetaDataIndexUpgradeService.java | 11 +++++++---- .../metadata/MetaDataIndexUpgradeServiceTests.java | 4 ++-- 2 files changed, 9 insertions(+), 6 deletions(-) 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 b9cf5de96e58e..2a8b80b9e689e 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -92,13 +92,16 @@ boolean isUpgraded(IndexMetaData indexMetaData) { } /** - * Elasticsearch 6.0 no longer supports indices with pre Lucene v5.0 (Elasticsearch v5.0.0.beta1) segments. All indices - * that were created before Elasticsearch v5.0.0.beta1 should be reindexed in Elasticsearch 5.x - * before they can be opened by this version of elasticsearch. */ + * 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 before [" + 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 + "."); } 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 638f472fc0090..a1dfdbc74f9ac 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java @@ -89,8 +89,8 @@ public void testFailUpgrade() { Version.CURRENT.luceneVersion.toString()).build()); String message = expectThrows(IllegalStateException.class, () -> service.upgradeIndexMetaData(metaData, Version.CURRENT.minimumIndexCompatibilityVersion())).getMessage(); - assertEquals(message, "The index [[foo/BOOM]] was created before [5.0.0]. It should be re-indexed in Elasticsearch 5.x " + - "before upgrading to " + Version.CURRENT.toString() + "."); + 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) From fbf4dbc59dd694281b1f27c58ef0894273c5931f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 30 Nov 2016 21:09:14 +0100 Subject: [PATCH 09/10] Use the max node version to determin the minIndexCompatibileVersion --- .../main/java/org/elasticsearch/Version.java | 14 +++++--- .../metadata/MetaDataCreateIndexService.java | 2 +- .../metadata/MetaDataIndexStateService.java | 2 +- .../cluster/node/DiscoveryNodes.java | 35 ++++++++++++++++--- .../discovery/zen/NodeJoinController.java | 2 +- .../gateway/LocalAllocateDangledIndices.java | 2 +- .../snapshots/RestoreService.java | 2 +- .../elasticsearch/transport/TcpTransport.java | 2 +- .../java/org/elasticsearch/VersionTests.java | 24 +++++++++++++ .../cluster/node/DiscoveryNodesTests.java | 19 ++++++++++ .../test/ESBackcompatTestCase.java | 2 +- 11 files changed, 89 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/Version.java b/core/src/main/java/org/elasticsearch/Version.java index 9f05f7af9f0e8..daae5a75575c9 100644 --- a/core/src/main/java/org/elasticsearch/Version.java +++ b/core/src/main/java/org/elasticsearch/Version.java @@ -217,12 +217,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 */ @@ -325,7 +330,7 @@ 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)); } /** @@ -340,7 +345,7 @@ public Version minimumIndexCompatibilityVersion() { bwcMajor = major - 1; } final int bwcMinor = 0; - return Version.smallest(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); + return Version.min(this, fromId(bwcMajor * 1000000 + bwcMinor * 10000 + 99)); } /** @@ -428,5 +433,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 b998c3542a21c..2a2c6c65b967c 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -161,7 +161,7 @@ public ClusterState execute(ClusterState currentState) { MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder() .blocks(currentState.blocks()); - Version minIndexCompatibilityVersion = currentState.getNodes().getSmallestNonClientNodeVersion() + final Version minIndexCompatibilityVersion = currentState.getNodes().getMaxNodeVersion() .minimumIndexCompatibilityVersion(); for (IndexMetaData closedMetaData : indicesToOpen) { final String indexName = closedMetaData.getIndex().getName(); 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/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 2b5ec3c940d06..4dce87aa86e70 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -437,7 +437,7 @@ public BatchResult execute(ClusterState currentState, List shardsBuilder = ImmutableOpenMap.builder(); - Version minIndexCompatibilityVersion = currentState.getNodes().getSmallestNonClientNodeVersion() + final Version minIndexCompatibilityVersion = currentState.getNodes().getMaxNodeVersion() .minimumIndexCompatibilityVersion(); for (Map.Entry indexEntry : renamedIndices.entrySet()) { String index = indexEntry.getValue(); 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 18fddaab80635..245fc4c048677 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -66,6 +66,30 @@ public void testVersionComparison() throws Exception { 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()); 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/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()); } From 31aee20e5acb6fc3cfbc62c3a3df0df0d3a1df44 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 30 Nov 2016 23:11:34 +0100 Subject: [PATCH 10/10] fix version constant --- core/src/test/java/org/elasticsearch/VersionTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/VersionTests.java b/core/src/test/java/org/elasticsearch/VersionTests.java index 245fc4c048677..51f7dd8f85769 100644 --- a/core/src/test/java/org/elasticsearch/VersionTests.java +++ b/core/src/test/java/org/elasticsearch/VersionTests.java @@ -93,7 +93,7 @@ public void testMax() { 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_0_UNRELEASED.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()); }