From 93fa50cad9b3e1c7913dc97dfb0ffb8a25539359 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 12:53:39 +0100 Subject: [PATCH 01/30] Move stats out of Zen1 package --- .../org/elasticsearch/cluster/coordination/Coordinator.java | 1 - .../coordination}/PendingClusterStateStats.java | 2 +- .../cluster/coordination/PublicationTransportHandler.java | 1 - .../coordination}/PublishClusterStateStats.java | 2 +- .../main/java/org/elasticsearch/discovery/DiscoveryStats.java | 4 ++-- .../discovery/zen/PendingClusterStatesQueue.java | 1 + .../discovery/zen/PublishClusterStateAction.java | 1 + .../action/admin/cluster/node/stats/NodeStatsTests.java | 4 ++-- .../elasticsearch/cluster/coordination/CoordinatorTests.java | 1 - .../discovery/zen/PublishClusterStateActionTests.java | 1 + 10 files changed, 9 insertions(+), 9 deletions(-) rename server/src/main/java/org/elasticsearch/{discovery/zen => cluster/coordination}/PendingClusterStateStats.java (98%) rename server/src/main/java/org/elasticsearch/{discovery/zen => cluster/coordination}/PublishClusterStateStats.java (98%) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 8f439bd51dd72..90dabab29d30d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -63,7 +63,6 @@ import org.elasticsearch.discovery.HandshakingTransportAddressConnector; import org.elasticsearch.discovery.PeerFinder; import org.elasticsearch.discovery.SeedHostsResolver; -import org.elasticsearch.discovery.zen.PendingClusterStateStats; import org.elasticsearch.discovery.SeedHostsProvider; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.TransportResponse.Empty; diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStateStats.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PendingClusterStateStats.java similarity index 98% rename from server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStateStats.java rename to server/src/main/java/org/elasticsearch/cluster/coordination/PendingClusterStateStats.java index a10d56c606de7..e8f0fc263047c 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStateStats.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PendingClusterStateStats.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.discovery.zen; +package org.elasticsearch.cluster.coordination; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java index 241b9b3f82231..86605737c8969 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java @@ -41,7 +41,6 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.discovery.zen.PublishClusterStateAction; -import org.elasticsearch.discovery.zen.PublishClusterStateStats; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BytesTransportRequest; import org.elasticsearch.transport.TransportChannel; diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateStats.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PublishClusterStateStats.java similarity index 98% rename from server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateStats.java rename to server/src/main/java/org/elasticsearch/cluster/coordination/PublishClusterStateStats.java index 36794e880f3ac..f74dbf9b1c500 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateStats.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PublishClusterStateStats.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.discovery.zen; +package org.elasticsearch.cluster.coordination; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryStats.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryStats.java index 1032b18f3a8dc..044912e9ef278 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryStats.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryStats.java @@ -25,8 +25,8 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.discovery.zen.PendingClusterStateStats; -import org.elasticsearch.discovery.zen.PublishClusterStateStats; +import org.elasticsearch.cluster.coordination.PendingClusterStateStats; +import org.elasticsearch.cluster.coordination.PublishClusterStateStats; import java.io.IOException; diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java b/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java index 018258066de8d..d471a32d5a9f3 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.coordination.PendingClusterStateStats; import org.elasticsearch.cluster.node.DiscoveryNode; import java.util.ArrayList; diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java b/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java index 1cb2528607b7e..107053816e86b 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.IncompatibleClusterStateVersionException; +import org.elasticsearch.cluster.coordination.PublishClusterStateStats; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.bytes.BytesReference; diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java index 8f51fb08dd23f..a0aab680bc039 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -23,8 +23,8 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.discovery.DiscoveryStats; -import org.elasticsearch.discovery.zen.PendingClusterStateStats; -import org.elasticsearch.discovery.zen.PublishClusterStateStats; +import org.elasticsearch.cluster.coordination.PendingClusterStateStats; +import org.elasticsearch.cluster.coordination.PublishClusterStateStats; import org.elasticsearch.http.HttpStats; import org.elasticsearch.indices.breaker.AllCircuitBreakerStats; import org.elasticsearch.indices.breaker.CircuitBreakerStats; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index f9925f4561b58..7df02b63e90ea 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -65,7 +65,6 @@ import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.discovery.SeedHostsProvider.HostsResolver; -import org.elasticsearch.discovery.zen.PublishClusterStateStats; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.MetaStateService; import org.elasticsearch.gateway.MockGatewayMetaState; diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java index 2f7a26af7cb58..3a8880da68d31 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.coordination.PublishClusterStateStats; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; From 51ca5b0a4f299a792647e6ead9a6846a5fc790fc Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 13:29:10 +0100 Subject: [PATCH 02/30] move ValidateJoinRequest --- .../cluster/coordination/JoinHelper.java | 6 +-- .../coordination/ValidateJoinRequest.java | 52 +++++++++++++++++++ .../discovery/zen/MembershipAction.java | 29 +---------- .../discovery/zen/ZenDiscoveryUnitTests.java | 5 +- 4 files changed, 60 insertions(+), 32 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/ValidateJoinRequest.java diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index a9309e9fe638a..ad3ab9c414718 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -131,7 +131,7 @@ public ClusterTasksResult execute(ClusterState currentSta }); transportService.registerRequestHandler(VALIDATE_JOIN_ACTION_NAME, - MembershipAction.ValidateJoinRequest::new, ThreadPool.Names.GENERIC, + ValidateJoinRequest::new, ThreadPool.Names.GENERIC, (request, channel, task) -> { final ClusterState localState = currentStateSupplier.get(); if (localState.metaData().clusterUUIDCommitted() && @@ -145,7 +145,7 @@ public ClusterTasksResult execute(ClusterState currentSta }); transportService.registerRequestHandler(MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME, - MembershipAction.ValidateJoinRequest::new, ThreadPool.Names.GENERIC, + ValidateJoinRequest::new, ThreadPool.Names.GENERIC, (request, channel, task) -> { joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState())); channel.sendResponse(Empty.INSTANCE); @@ -276,7 +276,7 @@ public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, Acti actionName = VALIDATE_JOIN_ACTION_NAME; } transportService.sendRequest(node, actionName, - new MembershipAction.ValidateJoinRequest(state), + new ValidateJoinRequest(state), TransportRequestOptions.builder().withTimeout(joinTimeout).build(), new EmptyTransportResponseHandler(ThreadPool.Names.GENERIC) { @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ValidateJoinRequest.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ValidateJoinRequest.java new file mode 100644 index 0000000000000..dec4a13c67d1e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ValidateJoinRequest.java @@ -0,0 +1,52 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.TransportRequest; + +import java.io.IOException; + +public class ValidateJoinRequest extends TransportRequest { + private ClusterState state; + + public ValidateJoinRequest() {} + + public ValidateJoinRequest(ClusterState state) { + this.state = state; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.state = ClusterState.readFrom(in, null); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + this.state.writeTo(out); + } + + public ClusterState getState() { + return state; + } +} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java b/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java index 550b25083fb96..9b8d9f14c2e8d 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.coordination.ValidateJoinRequest; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -159,32 +160,6 @@ public void onFailure(Exception e) { } } - public static class ValidateJoinRequest extends TransportRequest { - private ClusterState state; - - public ValidateJoinRequest() {} - - public ValidateJoinRequest(ClusterState state) { - this.state = state; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - this.state = ClusterState.readFrom(in, null); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - this.state.writeTo(out); - } - - public ClusterState getState() { - return state; - } - } - static class ValidateJoinRequestRequestHandler implements TransportRequestHandler { private final Supplier localNodeSupplier; private final Collection> joinValidators; @@ -199,7 +174,7 @@ static class ValidateJoinRequestRequestHandler implements TransportRequestHandle public void messageReceived(ValidateJoinRequest request, TransportChannel channel, Task task) throws Exception { DiscoveryNode node = localNodeSupplier.get(); assert node != null : "local node is null"; - joinValidators.stream().forEach(action -> action.accept(node, request.state)); + joinValidators.stream().forEach(action -> action.accept(node, request.getState())); channel.sendResponse(TransportResponse.Empty.INSTANCE); } } diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 084ba62c4792d..49163f9aa1fb1 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.coordination.JoinTaskExecutor; import org.elasticsearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor; +import org.elasticsearch.cluster.coordination.ValidateJoinRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -421,13 +422,13 @@ public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { .routingTable(RoutingTable.builder().add(indexRoutingTable).build()); if (incompatible) { IllegalStateException ex = expectThrows(IllegalStateException.class, () -> - request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), null, null)); + request.messageReceived(new ValidateJoinRequest(stateBuilder.build()), null, null)); assertEquals("index [test] version not supported: " + VersionUtils.getPreviousVersion(Version.CURRENT.minimumIndexCompatibilityVersion()) + " minimum compatible index version is: " + Version.CURRENT.minimumIndexCompatibilityVersion(), ex.getMessage()); } else { AtomicBoolean sendResponse = new AtomicBoolean(false); - request.messageReceived(new MembershipAction.ValidateJoinRequest(stateBuilder.build()), new TransportChannel() { + request.messageReceived(new ValidateJoinRequest(stateBuilder.build()), new TransportChannel() { @Override public String getProfileName() { From 642116839761534b5b473fb5e5f329b42dddab67 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 13:43:30 +0100 Subject: [PATCH 03/30] move hosts resolving to SeedHostsResolver --- .../common/settings/ClusterSettings.java | 10 +- .../discovery/SeedHostsResolver.java | 130 +++++++++++++++++- .../discovery/zen/UnicastZenPing.java | 125 +---------------- .../FileBasedSeedHostsProviderTests.java | 5 +- .../discovery/zen/UnicastZenPingTests.java | 11 +- 5 files changed, 143 insertions(+), 138 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 61c3dd9adadad..4bcb9b78fd3c4 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -66,10 +66,10 @@ import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.PeerFinder; +import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.FaultDetection; -import org.elasticsearch.discovery.zen.UnicastZenPing; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -406,10 +406,10 @@ public void apply(Settings value, Settings current, Settings previous) { ZenDiscovery.MAX_PENDING_CLUSTER_STATES_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING, - UnicastZenPing.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, - UnicastZenPing.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, - UnicastZenPing.LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING, - UnicastZenPing.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT, + SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, + SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, + SeedHostsResolver.LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING, + SeedHostsResolver.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT, SearchService.DEFAULT_KEEPALIVE_SETTING, SearchService.KEEPALIVE_INTERVAL_SETTING, SearchService.MAX_KEEPALIVE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java index b656cc288e37f..c13697965ede5 100644 --- a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java +++ b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java @@ -23,24 +23,45 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SetOnce; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.discovery.PeerFinder.ConfiguredHostsResolver; -import org.elasticsearch.discovery.zen.UnicastZenPing; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; +import java.util.stream.Collectors; public class SeedHostsResolver extends AbstractLifecycleComponent implements ConfiguredHostsResolver { + public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING = + Setting.intSetting("discovery.zen.ping.unicast.concurrent_connects", 10, 0, Setting.Property.NodeScope, Setting.Property.Deprecated); + public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT = + Setting.positiveTimeSetting("discovery.zen.ping.unicast.hosts.resolve_timeout", TimeValue.timeValueSeconds(5), + Setting.Property.NodeScope, Setting.Property.Deprecated); + public static final Setting DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING = + Setting.intSetting("discovery.seed_resolver.max_concurrent_resolvers", 10, 0, Setting.Property.NodeScope); + public static final Setting DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING = + Setting.positiveTimeSetting("discovery.seed_resolver.timeout", TimeValue.timeValueSeconds(5), Setting.Property.NodeScope); + private static final Logger logger = LogManager.getLogger(SeedHostsResolver.class); private final Settings settings; @@ -58,8 +79,109 @@ public SeedHostsResolver(String nodeName, Settings settings, TransportService tr this.nodeName = nodeName; this.transportService = transportService; this.hostsProvider = seedProvider; - resolveTimeout = UnicastZenPing.getResolveTimeout(settings); - concurrentConnects = UnicastZenPing.getMaxConcurrentResolvers(settings); + resolveTimeout = getResolveTimeout(settings); + concurrentConnects = getMaxConcurrentResolvers(settings); + } + + public static int getMaxConcurrentResolvers(Settings settings) { + if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.exists(settings)) { + if (DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.exists(settings)) { + throw new IllegalArgumentException("it is forbidden to set both [" + + DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.getKey() + "] and [" + + LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.getKey() + "]"); + } + return LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings); + } + return DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.get(settings); + } + + public static TimeValue getResolveTimeout(Settings settings) { + if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.exists(settings)) { + if (DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.exists(settings)) { + throw new IllegalArgumentException("it is forbidden to set both [" + + DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.getKey() + "] and [" + + LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.getKey() + "]"); + } + return LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.get(settings); + } + return DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.get(settings); + } + + /** + * Resolves a list of hosts to a list of transport addresses. Each host is resolved into a transport address (or a collection of + * addresses if the number of ports is greater than one). Host lookups are done in parallel using specified executor service up + * to the specified resolve timeout. + * + * @param executorService the executor service used to parallelize hostname lookups + * @param logger logger used for logging messages regarding hostname lookups + * @param hosts the hosts to resolve + * @param limitPortCounts the number of ports to resolve (should be 1 for non-local transport) + * @param transportService the transport service + * @param resolveTimeout the timeout before returning from hostname lookups + * @return a list of resolved transport addresses + */ + public static List resolveHostsLists( + final ExecutorService executorService, + final Logger logger, + final List hosts, + final int limitPortCounts, + final TransportService transportService, + final TimeValue resolveTimeout) { + Objects.requireNonNull(executorService); + Objects.requireNonNull(logger); + Objects.requireNonNull(hosts); + Objects.requireNonNull(transportService); + Objects.requireNonNull(resolveTimeout); + if (resolveTimeout.nanos() < 0) { + throw new IllegalArgumentException("resolve timeout must be non-negative but was [" + resolveTimeout + "]"); + } + // create tasks to submit to the executor service; we will wait up to resolveTimeout for these tasks to complete + final List> callables = + hosts + .stream() + .map(hn -> (Callable) () -> transportService.addressesFromString(hn, limitPortCounts)) + .collect(Collectors.toList()); + final List> futures; + try { + futures = executorService.invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Collections.emptyList(); + } + final List transportAddresses = new ArrayList<>(); + final Set localAddresses = new HashSet<>(); + localAddresses.add(transportService.boundAddress().publishAddress()); + localAddresses.addAll(Arrays.asList(transportService.boundAddress().boundAddresses())); + // ExecutorService#invokeAll guarantees that the futures are returned in the iteration order of the tasks so we can associate the + // hostname with the corresponding task by iterating together + final Iterator it = hosts.iterator(); + for (final Future future : futures) { + final String hostname = it.next(); + if (!future.isCancelled()) { + assert future.isDone(); + try { + final TransportAddress[] addresses = future.get(); + logger.trace("resolved host [{}] to {}", hostname, addresses); + for (int addressId = 0; addressId < addresses.length; addressId++) { + final TransportAddress address = addresses[addressId]; + // no point in pinging ourselves + if (localAddresses.contains(address) == false) { + transportAddresses.add(address); + } + } + } catch (final ExecutionException e) { + assert e.getCause() != null; + final String message = "failed to resolve host [" + hostname + "]"; + logger.warn(message, e.getCause()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + // ignore + } + } else { + logger.warn("timed out after [{}] resolving host [{}]", resolveTimeout, hostname); + } + } + return Collections.unmodifiableList(transportAddresses); } @Override @@ -102,7 +224,7 @@ protected void doRun() { List providedAddresses = hostsProvider.getSeedAddresses((hosts, limitPortCounts) - -> UnicastZenPing.resolveHostsLists(executorService.get(), logger, hosts, limitPortCounts, + -> resolveHostsLists(executorService.get(), logger, hosts, limitPortCounts, transportService, resolveTimeout)); consumer.accept(providedAddresses); diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java b/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java index d0cd0e27976fd..0cb10fa4e0f65 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java @@ -46,6 +46,7 @@ import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.discovery.SeedHostsProvider; +import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.node.Node; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; @@ -68,18 +69,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Objects; import java.util.Queue; import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -98,17 +92,6 @@ public class UnicastZenPing implements ZenPing { public static final String ACTION_NAME = "internal:discovery/zen/unicast"; - public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING = - Setting.intSetting("discovery.zen.ping.unicast.concurrent_connects", 10, 0, Property.NodeScope, Property.Deprecated); - public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT = - Setting.positiveTimeSetting("discovery.zen.ping.unicast.hosts.resolve_timeout", TimeValue.timeValueSeconds(5), - Property.NodeScope, Property.Deprecated); - - public static final Setting DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING = - Setting.intSetting("discovery.seed_resolver.max_concurrent_resolvers", 10, 0, Property.NodeScope); - public static final Setting DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING = - Setting.positiveTimeSetting("discovery.seed_resolver.timeout", TimeValue.timeValueSeconds(5), Property.NodeScope); - private final ThreadPool threadPool; private final TransportService transportService; private final ClusterName clusterName; @@ -140,8 +123,8 @@ public UnicastZenPing(Settings settings, ThreadPool threadPool, TransportService this.hostsProvider = seedHostsProvider; this.contextProvider = contextProvider; - final int concurrentConnects = getMaxConcurrentResolvers(settings); - resolveTimeout = getResolveTimeout(settings); + final int concurrentConnects = SeedHostsResolver.getMaxConcurrentResolvers(settings); + resolveTimeout = SeedHostsResolver.getResolveTimeout(settings); nodeName = Node.NODE_NAME_SETTING.get(settings); logger.debug( "using max_concurrent_resolvers [{}], resolver timeout [{}]", @@ -162,85 +145,8 @@ public UnicastZenPing(Settings settings, ThreadPool threadPool, TransportService threadPool.getThreadContext()); } - /** - * Resolves a list of hosts to a list of transport addresses. Each host is resolved into a transport address (or a collection of - * addresses if the number of ports is greater than one). Host lookups are done in parallel using specified executor service up - * to the specified resolve timeout. - * - * @param executorService the executor service used to parallelize hostname lookups - * @param logger logger used for logging messages regarding hostname lookups - * @param hosts the hosts to resolve - * @param limitPortCounts the number of ports to resolve (should be 1 for non-local transport) - * @param transportService the transport service - * @param resolveTimeout the timeout before returning from hostname lookups - * @return a list of resolved transport addresses - */ - public static List resolveHostsLists( - final ExecutorService executorService, - final Logger logger, - final List hosts, - final int limitPortCounts, - final TransportService transportService, - final TimeValue resolveTimeout) { - Objects.requireNonNull(executorService); - Objects.requireNonNull(logger); - Objects.requireNonNull(hosts); - Objects.requireNonNull(transportService); - Objects.requireNonNull(resolveTimeout); - if (resolveTimeout.nanos() < 0) { - throw new IllegalArgumentException("resolve timeout must be non-negative but was [" + resolveTimeout + "]"); - } - // create tasks to submit to the executor service; we will wait up to resolveTimeout for these tasks to complete - final List> callables = - hosts - .stream() - .map(hn -> (Callable) () -> transportService.addressesFromString(hn, limitPortCounts)) - .collect(Collectors.toList()); - final List> futures; - try { - futures = executorService.invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return Collections.emptyList(); - } - final List transportAddresses = new ArrayList<>(); - final Set localAddresses = new HashSet<>(); - localAddresses.add(transportService.boundAddress().publishAddress()); - localAddresses.addAll(Arrays.asList(transportService.boundAddress().boundAddresses())); - // ExecutorService#invokeAll guarantees that the futures are returned in the iteration order of the tasks so we can associate the - // hostname with the corresponding task by iterating together - final Iterator it = hosts.iterator(); - for (final Future future : futures) { - final String hostname = it.next(); - if (!future.isCancelled()) { - assert future.isDone(); - try { - final TransportAddress[] addresses = future.get(); - logger.trace("resolved host [{}] to {}", hostname, addresses); - for (int addressId = 0; addressId < addresses.length; addressId++) { - final TransportAddress address = addresses[addressId]; - // no point in pinging ourselves - if (localAddresses.contains(address) == false) { - transportAddresses.add(address); - } - } - } catch (final ExecutionException e) { - assert e.getCause() != null; - final String message = "failed to resolve host [" + hostname + "]"; - logger.warn(message, e.getCause()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - // ignore - } - } else { - logger.warn("timed out after [{}] resolving host [{}]", resolveTimeout, hostname); - } - } - return Collections.unmodifiableList(transportAddresses); - } - private SeedHostsProvider.HostsResolver createHostsResolver() { - return (hosts, limitPortCounts) -> resolveHostsLists(unicastZenPingExecutorService, logger, hosts, + return (hosts, limitPortCounts) -> SeedHostsResolver.resolveHostsLists(unicastZenPingExecutorService, logger, hosts, limitPortCounts, transportService, resolveTimeout); } @@ -672,27 +578,4 @@ protected Version getVersion() { return Version.CURRENT; // for tests } - public static int getMaxConcurrentResolvers(Settings settings) { - if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.exists(settings)) { - if (DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.exists(settings)) { - throw new IllegalArgumentException("it is forbidden to set both [" - + DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.getKey() + "] and [" - + LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.getKey() + "]"); - } - return LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings); - } - return DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.get(settings); - } - - public static TimeValue getResolveTimeout(Settings settings) { - if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.exists(settings)) { - if (DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.exists(settings)) { - throw new IllegalArgumentException("it is forbidden to set both [" - + DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.getKey() + "] and [" - + LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.getKey() + "]"); - } - return LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.get(settings); - } - return DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.get(settings); - } } diff --git a/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java b/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java index 4df4fc53affd3..fc89bd1a2f30b 100644 --- a/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java @@ -27,7 +27,6 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.discovery.zen.UnicastZenPing; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; @@ -117,7 +116,7 @@ public void testEmptyUnicastHostsFile() throws Exception { public void testUnicastHostsDoesNotExist() { final FileBasedSeedHostsProvider provider = new FileBasedSeedHostsProvider(createTempDir().toAbsolutePath()); final List addresses = provider.getSeedAddresses((hosts, limitPortCounts) -> - UnicastZenPing.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService, + SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService, TimeValue.timeValueSeconds(10))); assertEquals(0, addresses.size()); } @@ -147,7 +146,7 @@ private List setupAndRunHostProvider(final List hostEn } return new FileBasedSeedHostsProvider(configPath).getSeedAddresses((hosts, limitPortCounts) -> - UnicastZenPing.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService, + SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService, TimeValue.timeValueSeconds(10))); } } diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java index 859c056292445..9baafbdd66854 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; @@ -397,7 +398,7 @@ public BoundTransportAddress boundAddress() { Collections.emptySet()); closeables.push(transportService); final int limitPortCounts = randomIntBetween(1, 10); - final List transportAddresses = UnicastZenPing.resolveHostsLists( + final List transportAddresses = SeedHostsResolver.resolveHostsLists( executorService, logger, Collections.singletonList("127.0.0.1"), @@ -441,7 +442,7 @@ public BoundTransportAddress boundAddress() { new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); - final List transportAddresses = UnicastZenPing.resolveHostsLists( + final List transportAddresses = SeedHostsResolver.resolveHostsLists( executorService, logger, Collections.singletonList(NetworkAddress.format(loopbackAddress)), @@ -492,7 +493,7 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi Collections.emptySet()); closeables.push(transportService); - final List transportAddresses = UnicastZenPing.resolveHostsLists( + final List transportAddresses = SeedHostsResolver.resolveHostsLists( executorService, logger, Arrays.asList(hostname), @@ -551,7 +552,7 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi closeables.push(transportService); final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(3, 5)); try { - final List transportAddresses = UnicastZenPing.resolveHostsLists( + final List transportAddresses = SeedHostsResolver.resolveHostsLists( executorService, logger, Arrays.asList("hostname1", "hostname2"), @@ -718,7 +719,7 @@ public BoundTransportAddress boundAddress() { new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); - final List transportAddresses = UnicastZenPing.resolveHostsLists( + final List transportAddresses = SeedHostsResolver.resolveHostsLists( executorService, logger, Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"), From b4f306374cec42113e49c65fc7cb896ddc0efb71 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 13:50:04 +0100 Subject: [PATCH 04/30] switch serializeFullClusterState --- .../admin/cluster/state/TransportClusterStateAction.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java index 75dc811f37db6..3eb3f1746b4b6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.coordination.PublicationTransportHandler; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetaData; @@ -42,8 +43,6 @@ import java.io.IOException; import java.util.function.Predicate; -import static org.elasticsearch.discovery.zen.PublishClusterStateAction.serializeFullClusterState; - public class TransportClusterStateAction extends TransportMasterNodeReadAction { @@ -185,7 +184,7 @@ private void buildResponse(final ClusterStateRequest request, } } listener.onResponse(new ClusterStateResponse(currentState.getClusterName(), builder.build(), - serializeFullClusterState(currentState, Version.CURRENT).length(), false)); + PublicationTransportHandler.serializeFullClusterState(currentState, Version.CURRENT).length(), false)); } From f8a946ec021117fdc8fa747aa936e84235fe9e55 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 13:51:52 +0100 Subject: [PATCH 05/30] not mention zen disco in ClusterState --- .../main/java/org/elasticsearch/cluster/ClusterState.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index 7660ca9da92fa..79493ea2e66e3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -60,7 +60,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; import java.io.IOException; import java.util.EnumSet; @@ -80,9 +79,8 @@ * The cluster state can be updated only on the master node. All updates are performed by on a * single thread and controlled by the {@link ClusterService}. After every update the * {@link Discovery#publish} method publishes a new version of the cluster state to all other nodes in the - * cluster. The actual publishing mechanism is delegated to the {@link Discovery#publish} method and depends on - * the type of discovery. In the Zen Discovery it is handled in the {@link PublishClusterStateAction#publish} method. The - * publishing mechanism can be overridden by other discovery. + * cluster. The actual publishing mechanism is delegated to the {@link Discovery#publish} method and depends on + * the type of discovery. *

* The cluster state implements the {@link Diffable} interface in order to support publishing of cluster state * differences instead of the entire state on each change. The publishing mechanism should only send differences From 193340e593778efebbaae37a51cb36853072da8f Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 13:54:38 +0100 Subject: [PATCH 06/30] remove mention of MasterFaultDetection --- .../org/elasticsearch/cluster/NodeConnectionsService.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java b/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java index d6c2824fdbb10..28249754b50f1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java +++ b/server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java @@ -21,6 +21,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.cluster.coordination.FollowersChecker; +import org.elasticsearch.cluster.coordination.LeaderChecker; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.component.AbstractLifecycleComponent; @@ -32,8 +34,6 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.KeyedLock; -import org.elasticsearch.discovery.zen.MasterFaultDetection; -import org.elasticsearch.discovery.zen.NodesFaultDetection; import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -51,8 +51,7 @@ * This component is responsible for connecting to nodes once they are added to the cluster state, and disconnect when they are * removed. Also, it periodically checks that all connections are still open and if needed restores them. * Note that this component is *not* responsible for removing nodes from the cluster if they disconnect / do not respond - * to pings. This is done by {@link NodesFaultDetection}. Master fault detection - * is done by {@link MasterFaultDetection}. + * to pings. This is done by {@link FollowersChecker}. Master fault detection is done by {@link LeaderChecker}. */ public class NodeConnectionsService extends AbstractLifecycleComponent { private static final Logger logger = LogManager.getLogger(NodeConnectionsService.class); From 764b4d5b4d8ebdec53a88c43bf2be317170d2fa9 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:23:11 +0100 Subject: [PATCH 07/30] Add SeedHostsResolverTests --- .../discovery/SeedHostsResolverTests.java | 293 +++++++++++++++++- 1 file changed, 291 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java b/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java index 3254d2e9201a7..0506f5c48e803 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java @@ -19,24 +19,57 @@ package org.elasticsearch.discovery; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.transport.nio.MockNioTransport; import org.junit.After; import org.junit.Before; +import org.mockito.Matchers; +import java.io.Closeable; +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.Stack; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.core.IsNull.nullValue; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; public class SeedHostsResolverTests extends ESTestCase { @@ -44,6 +77,10 @@ public class SeedHostsResolverTests extends ESTestCase { private List transportAddresses; private SeedHostsResolver seedHostsResolver; private ThreadPool threadPool; + private ExecutorService executorService; + // close in reverse order as opened + private Stack closeables; + @Before public void startResolver() { @@ -55,12 +92,29 @@ public void startResolver() { seedHostsResolver = new SeedHostsResolver("test_node", Settings.EMPTY, transportService, hostsResolver -> transportAddresses); seedHostsResolver.start(); + + final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory("[" + getClass().getName() + "]"); + executorService = + EsExecutors.newScaling( + getClass().getName() + "/" + getTestName(), 0, 2, 60, TimeUnit.SECONDS, threadFactory, threadPool.getThreadContext()); + closeables = new Stack<>(); } @After - public void stopResolver() { + public void stopResolver() throws IOException { seedHostsResolver.stop(); - threadPool.shutdown(); + try { + logger.info("shutting down..."); + // JDK stack is broken, it does not iterate in the expected order (http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4475301) + final List reverse = new ArrayList<>(); + while (!closeables.isEmpty()) { + reverse.add(closeables.pop()); + } + IOUtils.close(reverse); + } finally { + terminate(executorService); + terminate(threadPool); + } } public void testResolvesAddressesInBackgroundAndIgnoresConcurrentCalls() throws Exception { @@ -92,4 +146,239 @@ public void testResolvesAddressesInBackgroundAndIgnoresConcurrentCalls() throws assertTrue(endLatch.await(30, TimeUnit.SECONDS)); assertThat(resolvedAddressesRef.get(), equalTo(transportAddresses)); } + + public void testPortLimit() { + final NetworkService networkService = new NetworkService(Collections.emptyList()); + final Transport transport = new MockNioTransport( + Settings.EMPTY, + Version.CURRENT, + threadPool, + networkService, + PageCacheRecycler.NON_RECYCLING_INSTANCE, + new NamedWriteableRegistry(Collections.emptyList()), + new NoneCircuitBreakerService()) { + + @Override + public BoundTransportAddress boundAddress() { + return new BoundTransportAddress( + new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9500)}, + new TransportAddress(InetAddress.getLoopbackAddress(), 9500) + ); + } + }; + closeables.push(transport); + final TransportService transportService = + new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, + Collections.emptySet()); + closeables.push(transportService); + final int limitPortCounts = randomIntBetween(1, 10); + final List transportAddresses = SeedHostsResolver.resolveHostsLists( + executorService, + logger, + Collections.singletonList("127.0.0.1"), + limitPortCounts, + transportService, + TimeValue.timeValueSeconds(30)); + assertThat(transportAddresses, hasSize(limitPortCounts)); + final Set ports = new HashSet<>(); + for (final TransportAddress address : transportAddresses) { + assertTrue(address.address().getAddress().isLoopbackAddress()); + ports.add(address.getPort()); + } + assertThat(ports, equalTo(IntStream.range(9300, 9300 + limitPortCounts).mapToObj(m -> m).collect(Collectors.toSet()))); + } + + public void testRemovingLocalAddresses() { + final NetworkService networkService = new NetworkService(Collections.emptyList()); + final InetAddress loopbackAddress = InetAddress.getLoopbackAddress(); + final Transport transport = new MockNioTransport( + Settings.EMPTY, + Version.CURRENT, + threadPool, + networkService, + PageCacheRecycler.NON_RECYCLING_INSTANCE, + new NamedWriteableRegistry(Collections.emptyList()), + new NoneCircuitBreakerService()) { + + @Override + public BoundTransportAddress boundAddress() { + return new BoundTransportAddress( + new TransportAddress[]{ + new TransportAddress(loopbackAddress, 9300), + new TransportAddress(loopbackAddress, 9301) + }, + new TransportAddress(loopbackAddress, 9302) + ); + } + }; + closeables.push(transport); + final TransportService transportService = + new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, + Collections.emptySet()); + closeables.push(transportService); + final List transportAddresses = SeedHostsResolver.resolveHostsLists( + executorService, + logger, + Collections.singletonList(NetworkAddress.format(loopbackAddress)), + 10, + transportService, + TimeValue.timeValueSeconds(30)); + assertThat(transportAddresses, hasSize(7)); + final Set ports = new HashSet<>(); + for (final TransportAddress address : transportAddresses) { + assertTrue(address.address().getAddress().isLoopbackAddress()); + ports.add(address.getPort()); + } + assertThat(ports, equalTo(IntStream.range(9303, 9310).mapToObj(m -> m).collect(Collectors.toSet()))); + } + + public void testUnknownHost() { + final Logger logger = mock(Logger.class); + final NetworkService networkService = new NetworkService(Collections.emptyList()); + final String hostname = randomAlphaOfLength(8); + final UnknownHostException unknownHostException = new UnknownHostException(hostname); + final Transport transport = new MockNioTransport( + Settings.EMPTY, + Version.CURRENT, + threadPool, + networkService, + PageCacheRecycler.NON_RECYCLING_INSTANCE, + new NamedWriteableRegistry(Collections.emptyList()), + new NoneCircuitBreakerService()) { + + @Override + public BoundTransportAddress boundAddress() { + return new BoundTransportAddress( + new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9300)}, + new TransportAddress(InetAddress.getLoopbackAddress(), 9300) + ); + } + + @Override + public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { + throw unknownHostException; + } + + }; + closeables.push(transport); + + final TransportService transportService = + new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, + Collections.emptySet()); + closeables.push(transportService); + + final List transportAddresses = SeedHostsResolver.resolveHostsLists( + executorService, + logger, + Arrays.asList(hostname), + 1, + transportService, + TimeValue.timeValueSeconds(30) + ); + + assertThat(transportAddresses, empty()); + verify(logger).warn("failed to resolve host [" + hostname + "]", unknownHostException); + } + + public void testResolveTimeout() { + final Logger logger = mock(Logger.class); + final NetworkService networkService = new NetworkService(Collections.emptyList()); + final CountDownLatch latch = new CountDownLatch(1); + final Transport transport = new MockNioTransport( + Settings.EMPTY, + Version.CURRENT, + threadPool, + networkService, + PageCacheRecycler.NON_RECYCLING_INSTANCE, + new NamedWriteableRegistry(Collections.emptyList()), + new NoneCircuitBreakerService()) { + + @Override + public BoundTransportAddress boundAddress() { + return new BoundTransportAddress( + new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9500)}, + new TransportAddress(InetAddress.getLoopbackAddress(), 9500) + ); + } + + @Override + public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { + if ("hostname1".equals(address)) { + return new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}; + } else if ("hostname2".equals(address)) { + try { + latch.await(); + return new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}; + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } else { + throw new UnknownHostException(address); + } + } + + }; + closeables.push(transport); + + final TransportService transportService = + new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, + Collections.emptySet()); + closeables.push(transportService); + final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(3, 5)); + try { + final List transportAddresses = SeedHostsResolver.resolveHostsLists( + executorService, + logger, + Arrays.asList("hostname1", "hostname2"), + 1, + transportService, + resolveTimeout); + + assertThat(transportAddresses, hasSize(1)); + verify(logger).trace( + "resolved host [{}] to {}", "hostname1", + new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}); + verify(logger).warn("timed out after [{}] resolving host [{}]", resolveTimeout, "hostname2"); + verifyNoMoreInteractions(logger); + } finally { + latch.countDown(); + } + } + + public void testInvalidHosts() { + final Logger logger = mock(Logger.class); + final Transport transport = new MockNioTransport( + Settings.EMPTY, + Version.CURRENT, + threadPool, + new NetworkService(Collections.emptyList()), + PageCacheRecycler.NON_RECYCLING_INSTANCE, + new NamedWriteableRegistry(Collections.emptyList()), + new NoneCircuitBreakerService()) { + @Override + public BoundTransportAddress boundAddress() { + return new BoundTransportAddress( + new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9300)}, + new TransportAddress(InetAddress.getLoopbackAddress(), 9300) + ); + } + }; + closeables.push(transport); + + final TransportService transportService = + new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, + Collections.emptySet()); + closeables.push(transportService); + final List transportAddresses = SeedHostsResolver.resolveHostsLists( + executorService, + logger, + Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"), + 1, + transportService, + TimeValue.timeValueSeconds(30)); + assertThat(transportAddresses, hasSize(1)); // only one of the two is valid and will be used + assertThat(transportAddresses.get(0).getAddress(), equalTo("127.0.0.1")); + assertThat(transportAddresses.get(0).getPort(), equalTo(9301)); + verify(logger).warn(eq("failed to resolve host [127.0.0.1:9300:9300]"), Matchers.any(ExecutionException.class)); + } } From 6196455254b125bc7daa894348dfeafefdc41e05 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:27:18 +0100 Subject: [PATCH 08/30] move join validator test to zen2 --- .../cluster/coordination/Coordinator.java | 4 + .../discovery/DiscoveryModuleTests.java | 6 +- .../discovery/zen/UnicastZenPingTests.java | 941 ------------------ 3 files changed, 6 insertions(+), 945 deletions(-) delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 90dabab29d30d..9664de95c40a1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -1061,6 +1061,10 @@ private void cancelActivePublication(String reason) { } } + public Collection> getOnJoinValidators() { + return onJoinValidators; + } + public enum Mode { CANDIDATE, LEADER, FOLLOWER } diff --git a/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java b/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java index 30b0448e09fad..fe74a736fe3ba 100644 --- a/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java @@ -30,7 +30,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.gateway.GatewayMetaState; import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.test.ESTestCase; @@ -194,15 +193,14 @@ public void testLazyConstructionSeedsProvider() { public void testJoinValidator() { BiConsumer consumer = (a, b) -> {}; - // TODO: move to zen2 once join validators are implemented DiscoveryModule module = newModule(Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), - DiscoveryModule.ZEN_DISCOVERY_TYPE).build(), Collections.singletonList(new DiscoveryPlugin() { + DiscoveryModule.ZEN2_DISCOVERY_TYPE).build(), Collections.singletonList(new DiscoveryPlugin() { @Override public BiConsumer getJoinValidator() { return consumer; } })); - ZenDiscovery discovery = (ZenDiscovery) module.getDiscovery(); + Coordinator discovery = (Coordinator) module.getDiscovery(); Collection> onJoinValidators = discovery.getOnJoinValidators(); assertEquals(2, onJoinValidators.size()); assertTrue(onJoinValidators.contains(consumer)); diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java deleted file mode 100644 index 9baafbdd66854..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java +++ /dev/null @@ -1,941 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.Constants; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlocks; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNode.Role; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.network.NetworkAddress; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.SeedHostsResolver; -import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -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.Transport; -import org.elasticsearch.transport.TransportConnectionListener; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.TransportSettings; -import org.elasticsearch.transport.nio.MockNioTransport; -import org.junit.After; -import org.junit.Before; -import org.mockito.Matchers; - -import java.io.Closeable; -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiFunction; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; -import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.hasSize; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; - -public class UnicastZenPingTests extends ESTestCase { - - private ThreadPool threadPool; - private ExecutorService executorService; - // close in reverse order as opened - private Stack closeables; - - @Before - public void setUp() throws Exception { - super.setUp(); - threadPool = new TestThreadPool(getClass().getName()); - final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory("[" + getClass().getName() + "]"); - executorService = - EsExecutors.newScaling( - getClass().getName() + "/" + getTestName(), 0, 2, 60, TimeUnit.SECONDS, threadFactory, threadPool.getThreadContext()); - closeables = new Stack<>(); - } - - @After - public void tearDown() throws Exception { - try { - logger.info("shutting down..."); - // JDK stack is broken, it does not iterate in the expected order (http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4475301) - final List reverse = new ArrayList<>(); - while (!closeables.isEmpty()) { - reverse.add(closeables.pop()); - } - IOUtils.close(reverse); - } finally { - terminate(executorService); - terminate(threadPool); - super.tearDown(); - } - } - - public void testSimplePings() throws IOException, InterruptedException, ExecutionException { - // use ephemeral ports - final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build(); - final Settings settingsMismatch = - Settings.builder().put(settings).put("cluster.name", "mismatch").put(TransportSettings.PORT.getKey(), 0).build(); - - NetworkService networkService = new NetworkService(Collections.emptyList()); - - final BiFunction supplier = (s, v) -> new MockNioTransport( - s, - v, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()); - - NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier); - closeables.push(handleA.transportService); - NetworkHandle handleB = startServices(settings, threadPool, "UZP_B", Version.CURRENT, supplier); - closeables.push(handleB.transportService); - NetworkHandle handleC = startServices(settingsMismatch, threadPool, "UZP_C", Version.CURRENT, supplier); - closeables.push(handleC.transportService); - final Version versionD; - if (randomBoolean()) { - versionD = VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumCompatibilityVersion(), Version.CURRENT); - } else { - versionD = Version.CURRENT; - } - logger.info("UZP_D version set to [{}]", versionD); - NetworkHandle handleD = startServices(settingsMismatch, threadPool, "UZP_D", versionD, supplier); - closeables.push(handleD.transportService); - - final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomNonNegativeLong()).build(); - final ClusterState stateMismatch = ClusterState.builder(new ClusterName("mismatch")).version(randomNonNegativeLong()).build(); - - final Settings hostsSettings = Settings.builder() - .putList(DISCOVERY_SEED_HOSTS_SETTING.getKey(), - NetworkAddress.format(new InetSocketAddress(handleA.address.address().getAddress(), handleA.address.address().getPort())), - NetworkAddress.format(new InetSocketAddress(handleB.address.address().getAddress(), handleB.address.address().getPort())), - NetworkAddress.format(new InetSocketAddress(handleC.address.address().getAddress(), handleC.address.address().getPort())), - NetworkAddress.format(new InetSocketAddress(handleD.address.address().getAddress(), handleD.address.address().getPort()))) - .put("cluster.name", "test") - .build(); - - Settings hostsSettingsMismatch = Settings.builder().put(hostsSettings).put(settingsMismatch).build(); - ClusterState stateA = ClusterState.builder(state) - .blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)) - .nodes(DiscoveryNodes.builder().add(handleA.node).localNodeId("UZP_A")) - .build(); - TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, () -> stateA); - zenPingA.start(); - closeables.push(zenPingA); - - ClusterState stateB = ClusterState.builder(state) - .nodes(DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B")) - .build(); - TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, () -> stateB); - zenPingB.start(); - closeables.push(zenPingB); - - ClusterState stateC = ClusterState.builder(stateMismatch) - .nodes(DiscoveryNodes.builder().add(handleC.node).localNodeId("UZP_C")) - .build(); - TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleC, () -> stateC) { - @Override - protected Version getVersion() { - return versionD; - } - }; - zenPingC.start(); - closeables.push(zenPingC); - - ClusterState stateD = ClusterState.builder(stateMismatch) - .nodes(DiscoveryNodes.builder().add(handleD.node).localNodeId("UZP_D")) - .build(); - TestUnicastZenPing zenPingD = new TestUnicastZenPing(hostsSettingsMismatch, threadPool, handleD, () -> stateD); - zenPingD.start(); - closeables.push(zenPingD); - - logger.info("ping from UZP_A"); - Collection pingResponses = zenPingA.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - ZenPing.PingResponse ping = pingResponses.iterator().next(); - assertThat(ping.node().getId(), equalTo("UZP_B")); - assertThat(ping.getClusterStateVersion(), equalTo(state.version())); - assertPings(handleA, handleB); - assertNoPings(handleA, handleC); // mismatch, shouldn't ping - assertNoPings(handleA, handleD); // mismatch, shouldn't ping - - // ping again, this time from B, - logger.info("ping from UZP_B"); - pingResponses = zenPingB.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - ping = pingResponses.iterator().next(); - assertThat(ping.node().getId(), equalTo("UZP_A")); - assertThat(ping.getClusterStateVersion(), equalTo(ElectMasterService.MasterCandidate.UNRECOVERED_CLUSTER_VERSION)); - assertPings(handleB, handleA); - assertNoPings(handleB, handleC); // mismatch, shouldn't ping - assertNoPings(handleB, handleD); // mismatch, shouldn't ping - - logger.info("ping from UZP_C"); - pingResponses = zenPingC.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - assertNoPings(handleC, handleA); - assertNoPings(handleC, handleB); - assertPings(handleC, handleD); - - logger.info("ping from UZP_D"); - pingResponses = zenPingD.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - assertNoPings(handleD, handleA); - assertNoPings(handleD, handleB); - assertPings(handleD, handleC); - - zenPingC.close(); - handleD.counters.clear(); - logger.info("ping from UZP_D after closing UZP_C"); - pingResponses = zenPingD.pingAndWait().toList(); - // check that node does not respond to pings anymore after the ping service has been closed - assertThat(pingResponses.size(), equalTo(0)); - assertNoPings(handleD, handleA); - assertNoPings(handleD, handleB); - assertPings(handleD, handleC); - } - - public void testUnknownHostNotCached() throws ExecutionException, InterruptedException { - // use ephemeral ports - final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build(); - - final NetworkService networkService = new NetworkService(Collections.emptyList()); - - final Map addresses = new HashMap<>(); - final BiFunction supplier = (s, v) -> new MockNioTransport( - s, - v, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()) { - @Override - public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { - final TransportAddress[] transportAddresses = addresses.get(address); - if (transportAddresses == null) { - throw new UnknownHostException(address); - } else { - return transportAddresses; - } - } - }; - - final NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier); - closeables.push(handleA.transportService); - final NetworkHandle handleB = startServices(settings, threadPool, "UZP_B", Version.CURRENT, supplier); - closeables.push(handleB.transportService); - final NetworkHandle handleC = startServices(settings, threadPool, "UZP_C", Version.CURRENT, supplier); - closeables.push(handleC.transportService); - - addresses.put( - "UZP_A", - new TransportAddress[]{ - new TransportAddress( - new InetSocketAddress(handleA.address.address().getAddress(), handleA.address.address().getPort()))}); - addresses.put( - "UZP_C", - new TransportAddress[]{ - new TransportAddress( - new InetSocketAddress(handleC.address.address().getAddress(), handleC.address.address().getPort()))}); - - final Settings hostsSettings = Settings.builder() - .putList(DISCOVERY_SEED_HOSTS_SETTING.getKey(), "UZP_A", "UZP_B", "UZP_C") - .put("cluster.name", "test") - .build(); - - final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomNonNegativeLong()).build(); - - ClusterState stateA = ClusterState.builder(state) - .blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)) - .nodes(DiscoveryNodes.builder().add(handleA.node).localNodeId("UZP_A")) - .build(); - final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, () -> stateA); - zenPingA.start(); - closeables.push(zenPingA); - - ClusterState stateB = ClusterState.builder(state) - .nodes(DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B")) - .build(); - TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, () -> stateB); - zenPingB.start(); - closeables.push(zenPingB); - - ClusterState stateC = ClusterState.builder(state) - .nodes(DiscoveryNodes.builder().add(handleC.node).localNodeId("UZP_C")) - .build(); - TestUnicastZenPing zenPingC = new TestUnicastZenPing(hostsSettings, threadPool, handleC, () -> stateC); - zenPingC.start(); - closeables.push(zenPingC); - - // the presence of an unresolvable host should not prevent resolvable hosts from being pinged - { - final Collection pingResponses = zenPingA.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - ZenPing.PingResponse ping = pingResponses.iterator().next(); - assertThat(ping.node().getId(), equalTo("UZP_C")); - assertThat(ping.getClusterStateVersion(), equalTo(state.version())); - assertNoPings(handleA, handleB); - assertPings(handleA, handleC); - assertNull(handleA.counters.get(handleB.address)); - } - - final HashMap moreThan = new HashMap<>(); - // we should see at least one ping to UZP_B, and one more ping than we have already seen to UZP_C - moreThan.put(handleB.address, 0); - moreThan.put(handleC.address, handleA.counters.get(handleC.address).intValue()); - - // now allow UZP_B to be resolvable - addresses.put( - "UZP_B", - new TransportAddress[]{ - new TransportAddress( - new InetSocketAddress(handleB.address.address().getAddress(), handleB.address.address().getPort()))}); - - // now we should see pings to UZP_B; this establishes that host resolutions are not cached - { - handleA.counters.clear(); - final Collection secondPingResponses = zenPingA.pingAndWait().toList(); - assertThat(secondPingResponses.size(), equalTo(2)); - final Set ids = new HashSet<>(secondPingResponses.stream().map(p -> p.node().getId()).collect(Collectors.toList())); - assertThat(ids, equalTo(new HashSet<>(Arrays.asList("UZP_B", "UZP_C")))); - assertPings(handleA, handleB); - assertPings(handleA, handleC); - } - } - - public void testPortLimit() throws InterruptedException { - final NetworkService networkService = new NetworkService(Collections.emptyList()); - final Transport transport = new MockNioTransport( - Settings.EMPTY, - Version.CURRENT, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()) { - - @Override - public BoundTransportAddress boundAddress() { - return new BoundTransportAddress( - new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9500)}, - new TransportAddress(InetAddress.getLoopbackAddress(), 9500) - ); - } - }; - closeables.push(transport); - final TransportService transportService = - new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, - Collections.emptySet()); - closeables.push(transportService); - final int limitPortCounts = randomIntBetween(1, 10); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Collections.singletonList("127.0.0.1"), - limitPortCounts, - transportService, - TimeValue.timeValueSeconds(30)); - assertThat(transportAddresses, hasSize(limitPortCounts)); - final Set ports = new HashSet<>(); - for (final TransportAddress address : transportAddresses) { - assertTrue(address.address().getAddress().isLoopbackAddress()); - ports.add(address.getPort()); - } - assertThat(ports, equalTo(IntStream.range(9300, 9300 + limitPortCounts).mapToObj(m -> m).collect(Collectors.toSet()))); - } - - public void testRemovingLocalAddresses() throws InterruptedException { - final NetworkService networkService = new NetworkService(Collections.emptyList()); - final InetAddress loopbackAddress = InetAddress.getLoopbackAddress(); - final Transport transport = new MockNioTransport( - Settings.EMPTY, - Version.CURRENT, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()) { - - @Override - public BoundTransportAddress boundAddress() { - return new BoundTransportAddress( - new TransportAddress[]{ - new TransportAddress(loopbackAddress, 9300), - new TransportAddress(loopbackAddress, 9301) - }, - new TransportAddress(loopbackAddress, 9302) - ); - } - }; - closeables.push(transport); - final TransportService transportService = - new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, - Collections.emptySet()); - closeables.push(transportService); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Collections.singletonList(NetworkAddress.format(loopbackAddress)), - 10, - transportService, - TimeValue.timeValueSeconds(30)); - assertThat(transportAddresses, hasSize(7)); - final Set ports = new HashSet<>(); - for (final TransportAddress address : transportAddresses) { - assertTrue(address.address().getAddress().isLoopbackAddress()); - ports.add(address.getPort()); - } - assertThat(ports, equalTo(IntStream.range(9303, 9310).mapToObj(m -> m).collect(Collectors.toSet()))); - } - - public void testUnknownHost() throws InterruptedException { - final Logger logger = mock(Logger.class); - final NetworkService networkService = new NetworkService(Collections.emptyList()); - final String hostname = randomAlphaOfLength(8); - final UnknownHostException unknownHostException = new UnknownHostException(hostname); - final Transport transport = new MockNioTransport( - Settings.EMPTY, - Version.CURRENT, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()) { - - @Override - public BoundTransportAddress boundAddress() { - return new BoundTransportAddress( - new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9300)}, - new TransportAddress(InetAddress.getLoopbackAddress(), 9300) - ); - } - - @Override - public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { - throw unknownHostException; - } - - }; - closeables.push(transport); - - final TransportService transportService = - new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, - Collections.emptySet()); - closeables.push(transportService); - - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Arrays.asList(hostname), - 1, - transportService, - TimeValue.timeValueSeconds(30) - ); - - assertThat(transportAddresses, empty()); - verify(logger).warn("failed to resolve host [" + hostname + "]", unknownHostException); - } - - public void testResolveTimeout() throws InterruptedException { - final Logger logger = mock(Logger.class); - final NetworkService networkService = new NetworkService(Collections.emptyList()); - final CountDownLatch latch = new CountDownLatch(1); - final Transport transport = new MockNioTransport( - Settings.EMPTY, - Version.CURRENT, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()) { - - @Override - public BoundTransportAddress boundAddress() { - return new BoundTransportAddress( - new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9500)}, - new TransportAddress(InetAddress.getLoopbackAddress(), 9500) - ); - } - - @Override - public TransportAddress[] addressesFromString(String address, int perAddressLimit) throws UnknownHostException { - if ("hostname1".equals(address)) { - return new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}; - } else if ("hostname2".equals(address)) { - try { - latch.await(); - return new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}; - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } else { - throw new UnknownHostException(address); - } - } - - }; - closeables.push(transport); - - final TransportService transportService = - new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, - Collections.emptySet()); - closeables.push(transportService); - final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(3, 5)); - try { - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Arrays.asList("hostname1", "hostname2"), - 1, - transportService, - resolveTimeout); - - assertThat(transportAddresses, hasSize(1)); - verify(logger).trace( - "resolved host [{}] to {}", "hostname1", - new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}); - verify(logger).warn("timed out after [{}] resolving host [{}]", resolveTimeout, "hostname2"); - verifyNoMoreInteractions(logger); - } finally { - latch.countDown(); - } - } - - public void testResolveReuseExistingNodeConnections() throws ExecutionException, InterruptedException { - final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build(); - - NetworkService networkService = new NetworkService(Collections.emptyList()); - - final BiFunction supplier = (s, v) -> new MockNioTransport( - s, - Version.CURRENT, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()); - - NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier, EnumSet.allOf(Role.class)); - closeables.push(handleA.transportService); - NetworkHandle handleB = startServices(settings, threadPool, "UZP_B", Version.CURRENT, supplier, EnumSet.allOf(Role.class)); - closeables.push(handleB.transportService); - - final boolean useHosts = randomBoolean(); - final Settings.Builder hostsSettingsBuilder = Settings.builder().put("cluster.name", "test"); - if (useHosts) { - hostsSettingsBuilder.putList(DISCOVERY_SEED_HOSTS_SETTING.getKey(), - NetworkAddress.format(new InetSocketAddress(handleB.address.address().getAddress(), handleB.address.address().getPort())) - ); - } else { - hostsSettingsBuilder.put(DISCOVERY_SEED_HOSTS_SETTING.getKey(), (String) null); - } - final Settings hostsSettings = hostsSettingsBuilder.build(); - - final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomNonNegativeLong()).build(); - - // connection to reuse - handleA.transportService.connectToNode(handleB.node); - - // install a listener to check that no new connections are made - handleA.transportService.addConnectionListener(new TransportConnectionListener() { - @Override - public void onConnectionOpened(Transport.Connection connection) { - fail("should not open any connections. got [" + connection.getNode() + "]"); - } - }); - - final ClusterState stateA = ClusterState.builder(state) - .blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)) - .nodes(DiscoveryNodes.builder().add(handleA.node).add(handleB.node).localNodeId("UZP_A")) - .build(); - final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, () -> stateA); - zenPingA.start(); - closeables.push(zenPingA); - - final ClusterState stateB = ClusterState.builder(state) - .nodes(DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B")) - .build(); - TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, () -> stateB); - zenPingB.start(); - closeables.push(zenPingB); - - Collection pingResponses = zenPingA.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - ZenPing.PingResponse ping = pingResponses.iterator().next(); - assertThat(ping.node().getId(), equalTo("UZP_B")); - assertThat(ping.getClusterStateVersion(), equalTo(state.version())); - - } - - public void testPingingTemporalPings() throws ExecutionException, InterruptedException { - final Settings settings = Settings.builder().put("cluster.name", "test").put(TransportSettings.PORT.getKey(), 0).build(); - - NetworkService networkService = new NetworkService(Collections.emptyList()); - - final BiFunction supplier = (s, v) -> new MockNioTransport( - s, - v, - threadPool, - networkService, - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()); - - NetworkHandle handleA = startServices(settings, threadPool, "UZP_A", Version.CURRENT, supplier, EnumSet.allOf(Role.class)); - closeables.push(handleA.transportService); - NetworkHandle handleB = startServices(settings, threadPool, "UZP_B", Version.CURRENT, supplier, EnumSet.allOf(Role.class)); - closeables.push(handleB.transportService); - - final Settings hostsSettings = Settings.builder() - .put("cluster.name", "test") - .put(DISCOVERY_SEED_HOSTS_SETTING.getKey(), (String) null) // use nodes for simplicity - .build(); - - final ClusterState state = ClusterState.builder(new ClusterName("test")).version(randomNonNegativeLong()).build(); - final ClusterState stateA = ClusterState.builder(state) - .blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)) - .nodes(DiscoveryNodes.builder().add(handleA.node).add(handleB.node).localNodeId("UZP_A")).build(); - - final TestUnicastZenPing zenPingA = new TestUnicastZenPing(hostsSettings, threadPool, handleA, () -> stateA); - zenPingA.start(); - closeables.push(zenPingA); - - // Node B doesn't know about A! - final ClusterState stateB = ClusterState.builder(state).nodes( - DiscoveryNodes.builder().add(handleB.node).localNodeId("UZP_B")).build(); - TestUnicastZenPing zenPingB = new TestUnicastZenPing(hostsSettings, threadPool, handleB, () -> stateB); - zenPingB.start(); - closeables.push(zenPingB); - - { - logger.info("pinging from UZP_A so UZP_B will learn about it"); - Collection pingResponses = zenPingA.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - ZenPing.PingResponse ping = pingResponses.iterator().next(); - assertThat(ping.node().getId(), equalTo("UZP_B")); - assertThat(ping.getClusterStateVersion(), equalTo(state.version())); - } - { - logger.info("pinging from UZP_B"); - Collection pingResponses = zenPingB.pingAndWait().toList(); - assertThat(pingResponses.size(), equalTo(1)); - ZenPing.PingResponse ping = pingResponses.iterator().next(); - assertThat(ping.node().getId(), equalTo("UZP_A")); - assertThat(ping.getClusterStateVersion(), equalTo(-1L)); // A has a block - } - } - - public void testInvalidHosts() throws InterruptedException { - final Logger logger = mock(Logger.class); - final Transport transport = new MockNioTransport( - Settings.EMPTY, - Version.CURRENT, - threadPool, - new NetworkService(Collections.emptyList()), - PageCacheRecycler.NON_RECYCLING_INSTANCE, - new NamedWriteableRegistry(Collections.emptyList()), - new NoneCircuitBreakerService()) { - @Override - public BoundTransportAddress boundAddress() { - return new BoundTransportAddress( - new TransportAddress[]{new TransportAddress(InetAddress.getLoopbackAddress(), 9300)}, - new TransportAddress(InetAddress.getLoopbackAddress(), 9300) - ); - } - }; - closeables.push(transport); - - final TransportService transportService = - new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, - Collections.emptySet()); - closeables.push(transportService); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"), - 1, - transportService, - TimeValue.timeValueSeconds(30)); - assertThat(transportAddresses, hasSize(1)); // only one of the two is valid and will be used - assertThat(transportAddresses.get(0).getAddress(), equalTo("127.0.0.1")); - assertThat(transportAddresses.get(0).getPort(), equalTo(9301)); - verify(logger).warn(eq("failed to resolve host [127.0.0.1:9300:9300]"), Matchers.any(ExecutionException.class)); - } - - private void assertNoPings(final NetworkHandle fromNode, final NetworkHandle toNode) { - final AtomicInteger counter = fromNode.counters.getOrDefault(toNode.address, new AtomicInteger()); - final String onNodeName = fromNode.node.getName(); - assertNotNull("handle for [" + onNodeName + "] has no 'expected' counter", counter); - final String forNodeName = toNode.node.getName(); - assertThat("node [" + onNodeName + "] ping count to [" + forNodeName + "] is unexpected", - counter.get(), equalTo(0)); - } - - private void assertPings(final NetworkHandle fromNode, final NetworkHandle toNode) { - final AtomicInteger counter = fromNode.counters.getOrDefault(toNode.address, new AtomicInteger()); - final String onNodeName = fromNode.node.getName(); - assertNotNull("handle for [" + onNodeName + "] has no 'expected' counter", counter); - final String forNodeName = toNode.node.getName(); - if (Constants.WINDOWS) { - // Some of the ping attempts seem to sporadically fail on Windows (see https://github.com/elastic/elasticsearch/issues/28685) - // Anyhow, the point of the test is not to assert the exact number of pings, but to check if pinging has taken place or not - assertThat("node [" + onNodeName + "] ping count to [" + forNodeName + "] is unexpected", - counter.get(), greaterThan(0)); - } else { - assertThat("node [" + onNodeName + "] ping count to [" + forNodeName + "] is unexpected", - counter.get(), equalTo(3)); - } - - } - - private NetworkHandle startServices( - final Settings settings, - final ThreadPool threadPool, - final String nodeId, - final Version version, - final BiFunction supplier) { - return startServices(settings, threadPool, nodeId, version, supplier, emptySet()); - - } - - private NetworkHandle startServices( - final Settings settings, - final ThreadPool threadPool, - final String nodeId, - final Version version, - final BiFunction supplier, - final Set nodeRoles) { - final Settings nodeSettings = Settings.builder().put(settings) - .put("node.name", nodeId) - .put(TransportSettings.TRACE_LOG_INCLUDE_SETTING.getKey(), "internal:discovery/zen/unicast") - .build(); - final Transport transport = supplier.apply(nodeSettings, version); - final MockTransportService transportService = - new MockTransportService(nodeSettings, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> - new DiscoveryNode(nodeId, nodeId, boundAddress.publishAddress(), emptyMap(), nodeRoles, version), null, - Collections.emptySet()); - transportService.start(); - transportService.acceptIncomingRequests(); - final ConcurrentMap counters = ConcurrentCollections.newConcurrentMap(); - transportService.addTracer(new MockTransportService.Tracer() { - @Override - public void requestSent(DiscoveryNode node, long requestId, String action, TransportRequestOptions options) { - counters.computeIfAbsent(node.getAddress(), k -> new AtomicInteger()); - counters.get(node.getAddress()).incrementAndGet(); - } - }); - return new NetworkHandle(transport.boundAddress().publishAddress(), transportService, transportService.getLocalNode(), counters); - } - - private static class NetworkHandle { - - public final TransportAddress address; - public final TransportService transportService; - public final DiscoveryNode node; - public final ConcurrentMap counters; - - NetworkHandle( - final TransportAddress address, - final TransportService transportService, - final DiscoveryNode discoveryNode, - final ConcurrentMap counters) { - this.address = address; - this.transportService = transportService; - this.node = discoveryNode; - this.counters = counters; - } - } - - private static class TestUnicastZenPing extends UnicastZenPing { - - private static final Logger logger = LogManager.getLogger(TestUnicastZenPing.class); - - TestUnicastZenPing(Settings settings, ThreadPool threadPool, NetworkHandle networkHandle, - PingContextProvider contextProvider) { - super(Settings.builder().put("node.name", networkHandle.node.getName()).put(settings).build(), - threadPool, networkHandle.transportService, - new SettingsBasedSeedHostsProvider(settings, networkHandle.transportService), contextProvider); - } - - volatile CountDownLatch allTasksCompleted; - volatile AtomicInteger pendingTasks; - volatile CountDownLatch pingingRoundClosed; - - PingCollection pingAndWait() throws ExecutionException, InterruptedException { - allTasksCompleted = new CountDownLatch(1); - pingingRoundClosed = new CountDownLatch(1); - pendingTasks = new AtomicInteger(); - // mark the three sending rounds as ongoing - markTaskAsStarted("send pings"); - markTaskAsStarted("send pings"); - markTaskAsStarted("send pings"); - final AtomicReference response = new AtomicReference<>(); - ping(response::set, TimeValue.timeValueMillis(1), TimeValue.timeValueSeconds(30)); - pingingRoundClosed.await(); - final PingCollection result = response.get(); - assertNotNull("pinging didn't complete", result); - return result; - } - - @Override - protected void finishPingingRound(PingingRound pingingRound) { - // wait for all activity to finish before closing - try { - allTasksCompleted.await(); - } catch (InterruptedException e) { - // ok, finish anyway - } - super.finishPingingRound(pingingRound); - pingingRoundClosed.countDown(); - } - - @Override - protected void sendPings(TimeValue timeout, PingingRound pingingRound) { - super.sendPings(timeout, pingingRound); - markTaskAsCompleted("send pings"); - } - - @Override - protected void submitToExecutor(AbstractRunnable abstractRunnable) { - markTaskAsStarted("executor runnable"); - super.submitToExecutor(new AbstractRunnable() { - @Override - public void onRejection(Exception e) { - try { - super.onRejection(e); - } finally { - markTaskAsCompleted("executor runnable (rejected)"); - } - } - - @Override - public void onAfter() { - markTaskAsCompleted("executor runnable"); - } - - @Override - protected void doRun() throws Exception { - abstractRunnable.run(); - } - - @Override - public void onFailure(Exception e) { - // we shouldn't really end up here. - throw new AssertionError("unexpected error", e); - } - }); - } - - private void markTaskAsStarted(String task) { - logger.trace("task [{}] started. count [{}]", task, pendingTasks.incrementAndGet()); - } - - private void markTaskAsCompleted(String task) { - final int left = pendingTasks.decrementAndGet(); - logger.trace("task [{}] completed. count [{}]", task, left); - if (left == 0) { - allTasksCompleted.countDown(); - } - } - - @Override - protected TransportResponseHandler getPingResponseHandler(PingingRound pingingRound, DiscoveryNode node) { - markTaskAsStarted("ping [" + node + "]"); - TransportResponseHandler original = super.getPingResponseHandler(pingingRound, node); - return new TransportResponseHandler() { - @Override - public UnicastPingResponse read(StreamInput in) throws IOException { - return original.read(in); - } - - @Override - public void handleResponse(UnicastPingResponse response) { - original.handleResponse(response); - markTaskAsCompleted("ping [" + node + "]"); - } - - @Override - public void handleException(TransportException exp) { - original.handleException(exp); - markTaskAsCompleted("ping [" + node + "] (error)"); - } - - @Override - public String executor() { - return original.executor(); - } - }; - } - } - -} From 4f8b70e24ace328da4670eb525fdcbfcee6e07f2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:32:34 +0100 Subject: [PATCH 09/30] remove Zen1 dep from GatewayIndexStateIT --- .../org/elasticsearch/gateway/GatewayIndexStateIT.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index ebdae985a39c7..314b272543e49 100644 --- a/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -39,7 +39,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.indices.IndexClosedException; @@ -466,7 +465,7 @@ public void testArchiveBrokenClusterSettings() throws Exception { final MetaData metaData = state.getMetaData(); final MetaData brokenMeta = MetaData.builder(metaData).persistentSettings(Settings.builder() .put(metaData.persistentSettings()).put("this.is.unknown", true) - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), "broken").build()).build(); + .put(MetaData.SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey(), "broken").build()).build(); internalCluster().fullRestart(new RestartCallback(){ @Override public Settings onNodeStopped(String nodeName) throws Exception { @@ -480,7 +479,7 @@ public Settings onNodeStopped(String nodeName) throws Exception { state = client().admin().cluster().prepareState().get().getState(); assertEquals("true", state.metaData().persistentSettings().get("archived.this.is.unknown")); assertEquals("broken", state.metaData().persistentSettings().get("archived." - + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey())); + + MetaData.SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey())); // delete these settings client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder().putNull("archived.*")).get(); @@ -488,7 +487,7 @@ public Settings onNodeStopped(String nodeName) throws Exception { state = client().admin().cluster().prepareState().get().getState(); assertNull(state.metaData().persistentSettings().get("archived.this.is.unknown")); assertNull(state.metaData().persistentSettings().get("archived." - + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey())); + + MetaData.SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey())); assertHitCount(client().prepareSearch().setQuery(matchAllQuery()).get(), 1L); } } From e4e9c093daabc0986e46be0a38ade92a8825e282 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:42:35 +0100 Subject: [PATCH 10/30] remove zen1 dependency from server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java --- .../coordination/PublicationTests.java | 45 ++++++++++++++++++- 1 file changed, 44 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java index d332888c185ac..d4cab5110eef2 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java @@ -25,11 +25,12 @@ import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportResponse; @@ -45,6 +46,8 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -57,6 +60,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; public class PublicationTests extends ESTestCase { @@ -478,4 +482,43 @@ private static DiscoveryNode newNode(int nodeId, Map attributes, return ts.stream(); }); } + + public static class AssertingAckListener implements Discovery.AckListener { + private final List> errors = new CopyOnWriteArrayList<>(); + private final Set successfulAcks = Collections.synchronizedSet(new HashSet<>()); + private final CountDownLatch countDown; + private final CountDownLatch commitCountDown; + + public AssertingAckListener(int nodeCount) { + countDown = new CountDownLatch(nodeCount); + commitCountDown = new CountDownLatch(1); + } + + @Override + public void onCommit(TimeValue commitTime) { + commitCountDown.countDown(); + } + + @Override + public void onNodeAck(DiscoveryNode node, @Nullable Exception e) { + if (e != null) { + errors.add(new Tuple<>(node, e)); + } else { + successfulAcks.add(node); + } + countDown.countDown(); + } + + public Set await(long timeout, TimeUnit unit) throws InterruptedException { + assertThat(awaitErrors(timeout, unit), emptyIterable()); + assertTrue(commitCountDown.await(timeout, unit)); + return new HashSet<>(successfulAcks); + } + + public List> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException { + countDown.await(timeout, unit); + return errors; + } + + } } From 756dc42a06d695d2c688cf94a1af733e013900ad Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:45:52 +0100 Subject: [PATCH 11/30] move MembershipActionTests.java --- .../coordination/JoinTaskExecutorTests.java} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename server/src/test/java/org/elasticsearch/{discovery/zen/MembershipActionTests.java => cluster/coordination/JoinTaskExecutorTests.java} (98%) diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/MembershipActionTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinTaskExecutorTests.java similarity index 98% rename from server/src/test/java/org/elasticsearch/discovery/zen/MembershipActionTests.java rename to server/src/test/java/org/elasticsearch/cluster/coordination/JoinTaskExecutorTests.java index cafe050726c54..35fa5786bbda3 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/MembershipActionTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinTaskExecutorTests.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.discovery.zen; +package org.elasticsearch.cluster.coordination; import org.elasticsearch.Version; import org.elasticsearch.cluster.coordination.JoinTaskExecutor; @@ -36,7 +36,7 @@ import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.elasticsearch.test.VersionUtils.randomVersionBetween; -public class MembershipActionTests extends ESTestCase { +public class JoinTaskExecutorTests extends ESTestCase { public void testPreventJoinClusterWithNewerIndices() { Settings.builder().build(); From 5e4a0e7f25d5a08cf18e98377d605b124c9219fe Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:46:24 +0100 Subject: [PATCH 12/30] remove zen discovery --- .../coordination/FollowersChecker.java | 19 +- .../cluster/coordination/JoinHelper.java | 41 +- .../coordination/JoinTaskExecutor.java | 5 - .../cluster/coordination/LeaderChecker.java | 28 +- .../PublicationTransportHandler.java | 28 +- .../common/settings/ClusterSettings.java | 18 - .../discovery/DiscoveryModule.java | 6 +- .../discovery/zen/ElectMasterService.java | 223 --- .../discovery/zen/FaultDetection.java | 116 -- .../discovery/zen/MasterFaultDetection.java | 444 ------ .../discovery/zen/MembershipAction.java | 214 --- .../discovery/zen/NodeJoinController.java | 381 ----- .../discovery/zen/NodesFaultDetection.java | 370 ----- .../zen/PendingClusterStatesQueue.java | 310 ---- .../discovery/zen/PingContextProvider.java | 28 - .../zen/PublishClusterStateAction.java | 664 --------- .../discovery/zen/UnicastZenPing.java | 581 -------- .../discovery/zen/ZenDiscovery.java | 1251 ----------------- .../elasticsearch/discovery/zen/ZenPing.java | 196 --- .../org/elasticsearch/gateway/Gateway.java | 5 +- .../elasticsearch/gateway/GatewayService.java | 3 - .../ack/AckClusterUpdateSettingsIT.java | 207 --- .../cluster/coordination/ZenDiscoveryIT.java | 15 - .../discovery/DiscoveryDisruptionIT.java | 22 +- .../discovery/MasterDisruptionIT.java | 38 - .../discovery/ZenFaultDetectionTests.java | 359 ----- .../single/SingleNodeDiscoveryIT.java | 62 - .../zen/ElectMasterServiceTests.java | 157 --- .../zen/NodeJoinControllerTests.java | 868 ------------ .../zen/PendingClusterStatesQueueTests.java | 268 ---- .../zen/PublishClusterStateActionTests.java | 962 ------------- .../discovery/zen/ZenDiscoveryUnitTests.java | 602 -------- .../discovery/zen/ZenPingTests.java | 90 -- .../elasticsearch/test/ESIntegTestCase.java | 28 +- .../test/InternalTestCluster.java | 85 -- 35 files changed, 12 insertions(+), 8682 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/NodesFaultDetection.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/PingContextProvider.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java delete mode 100644 server/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java delete mode 100644 server/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueueTests.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/FollowersChecker.java b/server/src/main/java/org/elasticsearch/cluster/coordination/FollowersChecker.java index 92bc9767d3c7a..4a9be561ae96d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/FollowersChecker.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/FollowersChecker.java @@ -22,8 +22,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.Coordinator.Mode; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -33,7 +31,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.discovery.zen.NodesFaultDetection; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TransportChannel; @@ -114,10 +111,6 @@ public FollowersChecker(Settings settings, TransportService transportService, updateFastResponseState(0, Mode.CANDIDATE); transportService.registerRequestHandler(FOLLOWER_CHECK_ACTION_NAME, Names.SAME, false, false, FollowerCheckRequest::new, (request, transportChannel, task) -> handleFollowerCheck(request, transportChannel)); - transportService.registerRequestHandler( - NodesFaultDetection.PING_ACTION_NAME, NodesFaultDetection.PingRequest::new, Names.SAME, false, false, - (request, channel, task) -> // TODO: check that we're a follower of the requesting node? - channel.sendResponse(new NodesFaultDetection.PingResponse())); transportService.addConnectionListener(new TransportConnectionListener() { @Override public void onNodeDisconnected(DiscoveryNode node) { @@ -304,17 +297,7 @@ private void handleWakeUp() { final FollowerCheckRequest request = new FollowerCheckRequest(fastResponseState.term, transportService.getLocalNode()); logger.trace("handleWakeUp: checking {} with {}", discoveryNode, request); - final String actionName; - final TransportRequest transportRequest; - if (Coordinator.isZen1Node(discoveryNode)) { - actionName = NodesFaultDetection.PING_ACTION_NAME; - transportRequest = new NodesFaultDetection.PingRequest(discoveryNode, ClusterName.CLUSTER_NAME_SETTING.get(settings), - transportService.getLocalNode(), ClusterState.UNKNOWN_VERSION); - } else { - actionName = FOLLOWER_CHECK_ACTION_NAME; - transportRequest = request; - } - transportService.sendRequest(discoveryNode, actionName, transportRequest, + transportService.sendRequest(discoveryNode, FOLLOWER_CHECK_ACTION_NAME, request, TransportRequestOptions.builder().withTimeout(followerCheckTimeout).withType(Type.PING).build(), new TransportResponseHandler() { @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index ad3ab9c414718..66d91cb03f226 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -37,8 +37,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.discovery.zen.MembershipAction; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.EmptyTransportResponseHandler; @@ -117,11 +115,6 @@ public ClusterTasksResult execute(ClusterState currentSta transportService.registerRequestHandler(JOIN_ACTION_NAME, ThreadPool.Names.GENERIC, false, false, JoinRequest::new, (request, channel, task) -> joinHandler.accept(request, transportJoinCallback(request, channel))); - transportService.registerRequestHandler(MembershipAction.DISCOVERY_JOIN_ACTION_NAME, MembershipAction.JoinRequest::new, - ThreadPool.Names.GENERIC, false, false, - (request, channel, task) -> joinHandler.accept(new JoinRequest(request.getNode(), Optional.empty()), // treat as non-voting join - transportJoinCallback(request, channel))); - transportService.registerRequestHandler(START_JOIN_ACTION_NAME, Names.GENERIC, false, false, StartJoinRequest::new, (request, channel, task) -> { @@ -143,21 +136,6 @@ public ClusterTasksResult execute(ClusterState currentSta joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState())); channel.sendResponse(Empty.INSTANCE); }); - - transportService.registerRequestHandler(MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME, - ValidateJoinRequest::new, ThreadPool.Names.GENERIC, - (request, channel, task) -> { - joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState())); - channel.sendResponse(Empty.INSTANCE); - }); - - transportService.registerRequestHandler( - ZenDiscovery.DISCOVERY_REJOIN_ACTION_NAME, ZenDiscovery.RejoinClusterRequest::new, ThreadPool.Names.SAME, - (request, channel, task) -> channel.sendResponse(Empty.INSTANCE)); // TODO: do we need to implement anything here? - - transportService.registerRequestHandler( - MembershipAction.DISCOVERY_LEAVE_ACTION_NAME, MembershipAction.LeaveRequest::new, ThreadPool.Names.SAME, - (request, channel, task) -> channel.sendResponse(Empty.INSTANCE)); // TODO: do we need to implement anything here? } private JoinCallback transportJoinCallback(TransportRequest request, TransportChannel channel) { @@ -200,16 +178,7 @@ public void sendJoinRequest(DiscoveryNode destination, Optional optionalJo final Tuple dedupKey = Tuple.tuple(destination, joinRequest); if (pendingOutgoingJoins.add(dedupKey)) { logger.debug("attempting to join {} with {}", destination, joinRequest); - final String actionName; - final TransportRequest transportRequest; - if (Coordinator.isZen1Node(destination)) { - actionName = MembershipAction.DISCOVERY_JOIN_ACTION_NAME; - transportRequest = new MembershipAction.JoinRequest(transportService.getLocalNode()); - } else { - actionName = JOIN_ACTION_NAME; - transportRequest = joinRequest; - } - transportService.sendRequest(destination, actionName, transportRequest, + transportService.sendRequest(destination, JOIN_ACTION_NAME, joinRequest, TransportRequestOptions.builder().withTimeout(joinTimeout).build(), new TransportResponseHandler() { @Override @@ -269,13 +238,7 @@ public String executor() { } public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, ActionListener listener) { - final String actionName; - if (Coordinator.isZen1Node(node)) { - actionName = MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME; - } else { - actionName = VALIDATE_JOIN_ACTION_NAME; - } - transportService.sendRequest(node, actionName, + transportService.sendRequest(node, VALIDATE_JOIN_ACTION_NAME, new ValidateJoinRequest(state), TransportRequestOptions.builder().withTimeout(joinTimeout).build(), new EmptyTransportResponseHandler(ThreadPool.Names.GENERIC) { diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java index ef83b9191d094..cc7128fcdcd4c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java @@ -30,7 +30,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import java.util.ArrayList; @@ -47,8 +46,6 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor { - try { - handleLeaderCheck(new LeaderCheckRequest(request.sourceNode)); - } catch (CoordinationStateRejectedException e) { - throw new MasterFaultDetection.ThisIsNotTheMasterYouAreLookingForException(e.getMessage()); - } - channel.sendResponse(new MasterFaultDetection.MasterPingResponseResponse()); - }); - transportService.addConnectionListener(new TransportConnectionListener() { @Override public void onNodeDisconnected(DiscoveryNode node) { @@ -217,21 +205,7 @@ void handleWakeUp() { logger.trace("checking {} with [{}] = {}", leader, LEADER_CHECK_TIMEOUT_SETTING.getKey(), leaderCheckTimeout); - final String actionName; - final TransportRequest transportRequest; - if (Coordinator.isZen1Node(leader)) { - actionName = MasterFaultDetection.MASTER_PING_ACTION_NAME; - transportRequest = new MasterFaultDetection.MasterPingRequest( - transportService.getLocalNode(), leader, ClusterName.CLUSTER_NAME_SETTING.get(settings)); - } else { - actionName = LEADER_CHECK_ACTION_NAME; - transportRequest = new LeaderCheckRequest(transportService.getLocalNode()); - } - // TODO lag detection: - // In the PoC, the leader sent its current version to the follower in the response to a LeaderCheck, so the follower - // could detect if it was lagging. We'd prefer this to be implemented on the leader, so the response is just - // TransportResponse.Empty here. - transportService.sendRequest(leader, actionName, transportRequest, + transportService.sendRequest(leader, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(transportService.getLocalNode()), TransportRequestOptions.builder().withTimeout(leaderCheckTimeout).withType(Type.PING).build(), new TransportResponseHandler() { diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java index 86605737c8969..e526b9699b429 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java @@ -40,12 +40,10 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BytesTransportRequest; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; @@ -54,7 +52,6 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; -import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; @@ -194,16 +191,7 @@ public String toString() { @Override public void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest applyCommitRequest, ActionListener responseActionListener) { - final String actionName; - final TransportRequest transportRequest; - if (Coordinator.isZen1Node(destination)) { - actionName = PublishClusterStateAction.COMMIT_ACTION_NAME; - transportRequest = new PublishClusterStateAction.CommitClusterStateRequest(newState.stateUUID()); - } else { - actionName = COMMIT_STATE_ACTION_NAME; - transportRequest = applyCommitRequest; - } - transportService.sendRequest(destination, actionName, transportRequest, stateRequestOptions, + transportService.sendRequest(destination, COMMIT_STATE_ACTION_NAME, applyCommitRequest, stateRequestOptions, new TransportResponseHandler() { @Override @@ -267,19 +255,7 @@ public String executor() { return ThreadPool.Names.GENERIC; } }; - final String actionName; - final TransportResponseHandler transportResponseHandler; - if (Coordinator.isZen1Node(node)) { - actionName = PublishClusterStateAction.SEND_ACTION_NAME; - transportResponseHandler = publishWithJoinResponseHandler.wrap(empty -> new PublishWithJoinResponse( - new PublishResponse(clusterState.term(), clusterState.version()), - Optional.of(new Join(node, transportService.getLocalNode(), clusterState.term(), clusterState.term(), - clusterState.version()))), in -> TransportResponse.Empty.INSTANCE); - } else { - actionName = PUBLISH_STATE_ACTION_NAME; - transportResponseHandler = publishWithJoinResponseHandler; - } - transportService.sendRequest(node, actionName, request, stateRequestOptions, transportResponseHandler); + transportService.sendRequest(node, PUBLISH_STATE_ACTION_NAME, request, stateRequestOptions, publishWithJoinResponseHandler); } catch (Exception e) { logger.warn(() -> new ParameterizedMessage("error sending cluster state to {}", node), e); responseActionListener.onFailure(e); diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 4bcb9b78fd3c4..a60648f773c37 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -68,9 +68,6 @@ import org.elasticsearch.discovery.PeerFinder; import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; -import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.FaultDetection; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.GatewayService; @@ -291,7 +288,6 @@ public void apply(Settings value, Settings current, Settings previous) { ClusterService.USER_DEFINED_META_DATA, SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING, SearchService.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS, - ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, RemoteClusterAware.REMOTE_CLUSTERS_SEEDS, RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_SEEDS, @@ -390,20 +386,6 @@ public void apply(Settings value, Settings current, Settings previous) { DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING, - FaultDetection.PING_RETRIES_SETTING, - FaultDetection.PING_TIMEOUT_SETTING, - FaultDetection.REGISTER_CONNECTION_LISTENER_SETTING, - FaultDetection.PING_INTERVAL_SETTING, - FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING, - ZenDiscovery.PING_TIMEOUT_SETTING, - ZenDiscovery.JOIN_TIMEOUT_SETTING, - ZenDiscovery.JOIN_RETRY_ATTEMPTS_SETTING, - ZenDiscovery.JOIN_RETRY_DELAY_SETTING, - ZenDiscovery.MAX_PINGS_FROM_ANOTHER_MASTER_SETTING, - ZenDiscovery.SEND_LEAVE_REQUEST_SETTING, - ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING, - ZenDiscovery.MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING, - ZenDiscovery.MAX_PENDING_CLUSTER_STATES_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 10f10e1040d99..8f8189676ff0f 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -19,8 +19,8 @@ package org.elasticsearch.discovery; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.coordination.Coordinator; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -37,7 +37,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.discovery.single.SingleNodeDiscovery; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.gateway.GatewayMetaState; import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.threadpool.ThreadPool; @@ -128,9 +127,6 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic }; Map> discoveryTypes = new HashMap<>(); - discoveryTypes.put(ZEN_DISCOVERY_TYPE, - () -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, - clusterSettings, seedHostsProvider, allocationService, joinValidators, gatewayMetaState)); discoveryTypes.put(ZEN2_DISCOVERY_TYPE, () -> new Coordinator(NODE_NAME_SETTING.get(settings), settings, clusterSettings, transportService, namedWriteableRegistry, allocationService, masterService, () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), seedHostsProvider, clusterApplier, diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java b/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java deleted file mode 100644 index 7e0f0cfca2a99..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java +++ /dev/null @@ -1,223 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import com.carrotsearch.hppc.ObjectContainer; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.CollectionUtils; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -public class ElectMasterService { - - private static final Logger logger = LogManager.getLogger(ElectMasterService.class); - - public static final Setting DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING = - Setting.intSetting("discovery.zen.minimum_master_nodes", -1, Property.Dynamic, Property.NodeScope, Property.Deprecated); - - private volatile int minimumMasterNodes; - - /** - * a class to encapsulate all the information about a candidate in a master election - * that is needed to decided which of the candidates should win - */ - public static class MasterCandidate { - - public static final long UNRECOVERED_CLUSTER_VERSION = -1; - - final DiscoveryNode node; - - final long clusterStateVersion; - - public MasterCandidate(DiscoveryNode node, long clusterStateVersion) { - Objects.requireNonNull(node); - assert clusterStateVersion >= -1 : "got: " + clusterStateVersion; - assert node.isMasterNode(); - this.node = node; - this.clusterStateVersion = clusterStateVersion; - } - - public DiscoveryNode getNode() { - return node; - } - - public long getClusterStateVersion() { - return clusterStateVersion; - } - - @Override - public String toString() { - return "Candidate{" + - "node=" + node + - ", clusterStateVersion=" + clusterStateVersion + - '}'; - } - - /** - * compares two candidates to indicate which the a better master. - * A higher cluster state version is better - * - * @return -1 if c1 is a batter candidate, 1 if c2. - */ - public static int compare(MasterCandidate c1, MasterCandidate c2) { - // we explicitly swap c1 and c2 here. the code expects "better" is lower in a sorted - // list, so if c2 has a higher cluster state version, it needs to come first. - int ret = Long.compare(c2.clusterStateVersion, c1.clusterStateVersion); - if (ret == 0) { - ret = compareNodes(c1.getNode(), c2.getNode()); - } - return ret; - } - } - - public ElectMasterService(Settings settings) { - this.minimumMasterNodes = DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings); - logger.debug("using minimum_master_nodes [{}]", minimumMasterNodes); - } - - public void minimumMasterNodes(int minimumMasterNodes) { - this.minimumMasterNodes = minimumMasterNodes; - } - - public int minimumMasterNodes() { - return minimumMasterNodes; - } - - public int countMasterNodes(Iterable nodes) { - int count = 0; - for (DiscoveryNode node : nodes) { - if (node.isMasterNode()) { - count++; - } - } - return count; - } - - public boolean hasEnoughCandidates(Collection candidates) { - if (candidates.isEmpty()) { - return false; - } - if (minimumMasterNodes < 1) { - return true; - } - assert candidates.stream().map(MasterCandidate::getNode).collect(Collectors.toSet()).size() == candidates.size() : - "duplicates ahead: " + candidates; - return candidates.size() >= minimumMasterNodes; - } - - /** - * Elects a new master out of the possible nodes, returning it. Returns {@code null} - * if no master has been elected. - */ - public MasterCandidate electMaster(Collection candidates) { - assert hasEnoughCandidates(candidates); - List sortedCandidates = new ArrayList<>(candidates); - sortedCandidates.sort(MasterCandidate::compare); - return sortedCandidates.get(0); - } - - /** selects the best active master to join, where multiple are discovered */ - public DiscoveryNode tieBreakActiveMasters(Collection activeMasters) { - return activeMasters.stream().min(ElectMasterService::compareNodes).get(); - } - - public boolean hasEnoughMasterNodes(Iterable nodes) { - final int count = countMasterNodes(nodes); - return count > 0 && (minimumMasterNodes < 0 || count >= minimumMasterNodes); - } - - public boolean hasTooManyMasterNodes(Iterable nodes) { - final int count = countMasterNodes(nodes); - return count > 1 && minimumMasterNodes <= count / 2; - } - - public void logMinimumMasterNodesWarningIfNecessary(ClusterState oldState, ClusterState newState) { - // check if min_master_nodes setting is too low and log warning - if (hasTooManyMasterNodes(oldState.nodes()) == false && hasTooManyMasterNodes(newState.nodes())) { - logger.warn("value for setting \"{}\" is too low. This can result in data loss! Please set it to at least a quorum of master-" + - "eligible nodes (current value: [{}], total number of master-eligible nodes used for publishing in this round: [{}])", - ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minimumMasterNodes(), - newState.getNodes().getMasterNodes().size()); - } - } - - /** - * Returns the given nodes sorted by likelihood of being elected as master, most likely first. - * Non-master nodes are not removed but are rather put in the end - */ - static List sortByMasterLikelihood(Iterable nodes) { - ArrayList sortedNodes = CollectionUtils.iterableAsArrayList(nodes); - CollectionUtil.introSort(sortedNodes, ElectMasterService::compareNodes); - return sortedNodes; - } - - /** - * Returns a list of the next possible masters. - */ - public DiscoveryNode[] nextPossibleMasters(ObjectContainer nodes, int numberOfPossibleMasters) { - List sortedNodes = sortedMasterNodes(Arrays.asList(nodes.toArray(DiscoveryNode.class))); - if (sortedNodes == null) { - return new DiscoveryNode[0]; - } - List nextPossibleMasters = new ArrayList<>(numberOfPossibleMasters); - int counter = 0; - for (DiscoveryNode nextPossibleMaster : sortedNodes) { - if (++counter >= numberOfPossibleMasters) { - break; - } - nextPossibleMasters.add(nextPossibleMaster); - } - return nextPossibleMasters.toArray(new DiscoveryNode[nextPossibleMasters.size()]); - } - - private List sortedMasterNodes(Iterable nodes) { - List possibleNodes = CollectionUtils.iterableAsArrayList(nodes); - if (possibleNodes.isEmpty()) { - return null; - } - // clean non master nodes - possibleNodes.removeIf(node -> !node.isMasterNode()); - CollectionUtil.introSort(possibleNodes, ElectMasterService::compareNodes); - return possibleNodes; - } - - /** master nodes go before other nodes, with a secondary sort by id **/ - private static int compareNodes(DiscoveryNode o1, DiscoveryNode o2) { - if (o1.isMasterNode() && !o2.isMasterNode()) { - return -1; - } - if (!o1.isMasterNode() && o2.isMasterNode()) { - return 1; - } - return o1.getId().compareTo(o2.getId()); - } -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java b/server/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java deleted file mode 100644 index 424dc6d31b0b3..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import java.io.Closeable; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportConnectionListener; -import org.elasticsearch.transport.TransportService; - -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; - -/** - * A base class for {@link MasterFaultDetection} & {@link NodesFaultDetection}, - * making sure both use the same setting. - */ -public abstract class FaultDetection implements Closeable { - - private static final Logger logger = LogManager.getLogger(FaultDetection.class); - - public static final Setting CONNECT_ON_NETWORK_DISCONNECT_SETTING = - Setting.boolSetting("discovery.zen.fd.connect_on_network_disconnect", false, Property.NodeScope, Property.Deprecated); - public static final Setting PING_INTERVAL_SETTING = - Setting.positiveTimeSetting("discovery.zen.fd.ping_interval", timeValueSeconds(1), Property.NodeScope, Property.Deprecated); - public static final Setting PING_TIMEOUT_SETTING = - Setting.timeSetting("discovery.zen.fd.ping_timeout", timeValueSeconds(30), Property.NodeScope, Property.Deprecated); - public static final Setting PING_RETRIES_SETTING = - Setting.intSetting("discovery.zen.fd.ping_retries", 3, Property.NodeScope, Property.Deprecated); - public static final Setting REGISTER_CONNECTION_LISTENER_SETTING = - Setting.boolSetting("discovery.zen.fd.register_connection_listener", true, Property.NodeScope, Property.Deprecated); - - protected final ThreadPool threadPool; - protected final ClusterName clusterName; - protected final TransportService transportService; - - // used mainly for testing, should always be true - protected final boolean registerConnectionListener; - protected final FDConnectionListener connectionListener; - protected final boolean connectOnNetworkDisconnect; - - protected final TimeValue pingInterval; - protected final TimeValue pingRetryTimeout; - protected final int pingRetryCount; - - public FaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService, ClusterName clusterName) { - this.threadPool = threadPool; - this.transportService = transportService; - this.clusterName = clusterName; - - this.connectOnNetworkDisconnect = CONNECT_ON_NETWORK_DISCONNECT_SETTING.get(settings); - this.pingInterval = PING_INTERVAL_SETTING.get(settings); - this.pingRetryTimeout = PING_TIMEOUT_SETTING.get(settings); - this.pingRetryCount = PING_RETRIES_SETTING.get(settings); - this.registerConnectionListener = REGISTER_CONNECTION_LISTENER_SETTING.get(settings); - - this.connectionListener = new FDConnectionListener(); - if (registerConnectionListener) { - transportService.addConnectionListener(connectionListener); - } - } - - @Override - public void close() { - transportService.removeConnectionListener(connectionListener); - } - - /** - * This method will be called when the {@link org.elasticsearch.transport.TransportService} raised a node disconnected event - */ - abstract void handleTransportDisconnect(DiscoveryNode node); - - private class FDConnectionListener implements TransportConnectionListener { - @Override - public void onNodeDisconnected(DiscoveryNode node) { - AbstractRunnable runnable = new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.warn("failed to handle transport disconnect for node: {}", node); - } - - @Override - protected void doRun() { - handleTransportDisconnect(node); - } - }; - threadPool.generic().execute(runnable); - } - } - -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java b/server/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java deleted file mode 100644 index 4c1fbcd702afa..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java +++ /dev/null @@ -1,444 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateUpdateTask; -import org.elasticsearch.cluster.NotMasterException; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Nullable; -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.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A fault detection that pings the master periodically to see if its alive. - */ -public class MasterFaultDetection extends FaultDetection { - - private static final Logger logger = LogManager.getLogger(MasterFaultDetection.class); - - public static final String MASTER_PING_ACTION_NAME = "internal:discovery/zen/fd/master_ping"; - - public interface Listener { - - /** called when pinging the master failed, like a timeout, transport disconnects etc */ - void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason); - - } - - private final MasterService masterService; - private final java.util.function.Supplier clusterStateSupplier; - private final CopyOnWriteArrayList listeners = new CopyOnWriteArrayList<>(); - - private volatile MasterPinger masterPinger; - - private final Object masterNodeMutex = new Object(); - - private volatile DiscoveryNode masterNode; - - private volatile int retryCount; - - private final AtomicBoolean notifiedMasterFailure = new AtomicBoolean(); - - public MasterFaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService, - java.util.function.Supplier clusterStateSupplier, MasterService masterService, - ClusterName clusterName) { - super(settings, threadPool, transportService, clusterName); - this.clusterStateSupplier = clusterStateSupplier; - this.masterService = masterService; - - logger.debug("[master] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]", pingInterval, pingRetryTimeout, - pingRetryCount); - - transportService.registerRequestHandler( - MASTER_PING_ACTION_NAME, MasterPingRequest::new, ThreadPool.Names.SAME, false, false, new MasterPingRequestHandler()); - } - - public DiscoveryNode masterNode() { - return this.masterNode; - } - - public void addListener(Listener listener) { - listeners.add(listener); - } - - public void removeListener(Listener listener) { - listeners.remove(listener); - } - - public void restart(DiscoveryNode masterNode, String reason) { - synchronized (masterNodeMutex) { - if (logger.isDebugEnabled()) { - logger.debug("[master] restarting fault detection against master [{}], reason [{}]", masterNode, reason); - } - innerStop(); - innerStart(masterNode); - } - } - - private void innerStart(final DiscoveryNode masterNode) { - this.masterNode = masterNode; - this.retryCount = 0; - this.notifiedMasterFailure.set(false); - if (masterPinger != null) { - masterPinger.stop(); - } - this.masterPinger = new MasterPinger(); - - // we start pinging slightly later to allow the chosen master to complete it's own master election - threadPool.schedule(masterPinger, pingInterval, ThreadPool.Names.SAME); - } - - public void stop(String reason) { - synchronized (masterNodeMutex) { - if (masterNode != null) { - if (logger.isDebugEnabled()) { - logger.debug("[master] stopping fault detection against master [{}], reason [{}]", masterNode, reason); - } - } - innerStop(); - } - } - - private void innerStop() { - // also will stop the next ping schedule - this.retryCount = 0; - if (masterPinger != null) { - masterPinger.stop(); - masterPinger = null; - } - this.masterNode = null; - } - - @Override - public void close() { - super.close(); - stop("closing"); - this.listeners.clear(); - } - - @Override - protected void handleTransportDisconnect(DiscoveryNode node) { - synchronized (masterNodeMutex) { - if (!node.equals(this.masterNode)) { - return; - } - if (connectOnNetworkDisconnect) { - try { - transportService.connectToNode(node); - // if all is well, make sure we restart the pinger - if (masterPinger != null) { - masterPinger.stop(); - } - this.masterPinger = new MasterPinger(); - // we use schedule with a 0 time value to run the pinger on the pool as it will run on later - threadPool.schedule(masterPinger, TimeValue.timeValueMillis(0), ThreadPool.Names.SAME); - } catch (Exception e) { - logger.trace("[master] [{}] transport disconnected (with verified connect)", masterNode); - notifyMasterFailure(masterNode, null, "transport disconnected (with verified connect)"); - } - } else { - logger.trace("[master] [{}] transport disconnected", node); - notifyMasterFailure(node, null, "transport disconnected"); - } - } - } - - private void notifyMasterFailure(final DiscoveryNode masterNode, final Throwable cause, final String reason) { - if (notifiedMasterFailure.compareAndSet(false, true)) { - try { - threadPool.generic().execute(() -> { - for (Listener listener : listeners) { - listener.onMasterFailure(masterNode, cause, reason); - } - }); - } catch (EsRejectedExecutionException e) { - logger.error("master failure notification was rejected, it's highly likely the node is shutting down", e); - } - stop("master failure, " + reason); - } - } - - private class MasterPinger implements Runnable { - - private volatile boolean running = true; - - public void stop() { - this.running = false; - } - - @Override - public void run() { - if (!running) { - // return and don't spawn... - return; - } - final DiscoveryNode masterToPing = masterNode; - if (masterToPing == null) { - // master is null, should not happen, but we are still running, so reschedule - threadPool.schedule(MasterPinger.this, pingInterval, ThreadPool.Names.SAME); - return; - } - - final MasterPingRequest request = new MasterPingRequest( - clusterStateSupplier.get().nodes().getLocalNode(), masterToPing, clusterName); - final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING) - .withTimeout(pingRetryTimeout).build(); - transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, - new TransportResponseHandler() { - @Override - public MasterPingResponseResponse read(StreamInput in) throws IOException { - return new MasterPingResponseResponse(in); - } - - @Override - public void handleResponse(MasterPingResponseResponse response) { - if (!running) { - return; - } - // reset the counter, we got a good result - MasterFaultDetection.this.retryCount = 0; - // check if the master node did not get switched on us..., if it did, we simply return with no reschedule - if (masterToPing.equals(MasterFaultDetection.this.masterNode())) { - // we don't stop on disconnection from master, we keep pinging it - threadPool.schedule(MasterPinger.this, pingInterval, ThreadPool.Names.SAME); - } - } - - @Override - public void handleException(TransportException exp) { - if (!running) { - return; - } - synchronized (masterNodeMutex) { - // check if the master node did not get switched on us... - if (masterToPing.equals(MasterFaultDetection.this.masterNode())) { - if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) { - handleTransportDisconnect(masterToPing); - return; - } else if (exp.getCause() instanceof NotMasterException) { - logger.debug("[master] pinging a master {} that is no longer a master", masterNode); - notifyMasterFailure(masterToPing, exp, "no longer master"); - return; - } else if (exp.getCause() instanceof ThisIsNotTheMasterYouAreLookingForException) { - logger.debug("[master] pinging a master {} that is not the master", masterNode); - notifyMasterFailure(masterToPing, exp,"not master"); - return; - } else if (exp.getCause() instanceof NodeDoesNotExistOnMasterException) { - logger.debug("[master] pinging a master {} but we do not exists on it, act as if its master failure" - , masterNode); - notifyMasterFailure(masterToPing, exp,"do not exists on master, act as master failure"); - return; - } - - int retryCount = ++MasterFaultDetection.this.retryCount; - logger.trace(() -> new ParameterizedMessage( - "[master] failed to ping [{}], retry [{}] out of [{}]", - masterNode, retryCount, pingRetryCount), exp); - if (retryCount >= pingRetryCount) { - logger.debug("[master] failed to ping [{}], tried [{}] times, each with maximum [{}] timeout", - masterNode, pingRetryCount, pingRetryTimeout); - // not good, failure - notifyMasterFailure(masterToPing, null, "failed to ping, tried [" + pingRetryCount - + "] times, each with maximum [" + pingRetryTimeout + "] timeout"); - } else { - // resend the request, not reschedule, rely on send timeout - transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, this); - } - } - } - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - } - ); - } - } - - /** Thrown when a ping reaches the wrong node */ - public static class ThisIsNotTheMasterYouAreLookingForException extends IllegalStateException { - - public ThisIsNotTheMasterYouAreLookingForException(String msg) { - super(msg); - } - - public ThisIsNotTheMasterYouAreLookingForException() { - } - - @Override - public Throwable fillInStackTrace() { - return null; - } - } - - static class NodeDoesNotExistOnMasterException extends IllegalStateException { - @Override - public Throwable fillInStackTrace() { - return null; - } - } - - private class MasterPingRequestHandler implements TransportRequestHandler { - - @Override - public void messageReceived(final MasterPingRequest request, final TransportChannel channel, Task task) throws Exception { - final DiscoveryNodes nodes = clusterStateSupplier.get().nodes(); - // check if we are really the same master as the one we seemed to be think we are - // this can happen if the master got "kill -9" and then another node started using the same port - if (!request.masterNode.equals(nodes.getLocalNode())) { - throw new ThisIsNotTheMasterYouAreLookingForException(); - } - - // ping from nodes of version < 1.4.0 will have the clustername set to null - if (request.clusterName != null && !request.clusterName.equals(clusterName)) { - logger.trace("master fault detection ping request is targeted for a different [{}] cluster then us [{}]", - request.clusterName, clusterName); - throw new ThisIsNotTheMasterYouAreLookingForException("master fault detection ping request is targeted for a different [" - + request.clusterName + "] cluster then us [" + clusterName + "]"); - } - - // when we are elected as master or when a node joins, we use a cluster state update thread - // to incorporate that information in the cluster state. That cluster state is published - // before we make it available locally. This means that a master ping can come from a node - // that has already processed the new CS but it is not known locally. - // Therefore, if we fail we have to check again under a cluster state thread to make sure - // all processing is finished. - // - - if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.sourceNode)) { - logger.trace("checking ping from {} under a cluster state thread", request.sourceNode); - masterService.submitStateUpdateTask("master ping (from: " + request.sourceNode + ")", new ClusterStateUpdateTask() { - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - // if we are no longer master, fail... - DiscoveryNodes nodes = currentState.nodes(); - if (!nodes.nodeExists(request.sourceNode)) { - throw new NodeDoesNotExistOnMasterException(); - } - return currentState; - } - - @Override - public void onNoLongerMaster(String source) { - onFailure(source, new NotMasterException("local node is not master")); - } - - @Override - public void onFailure(String source, @Nullable Exception e) { - if (e == null) { - e = new ElasticsearchException("unknown error while processing ping"); - } - try { - channel.sendResponse(e); - } catch (IOException inner) { - inner.addSuppressed(e); - logger.warn("error while sending ping response", inner); - } - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - try { - channel.sendResponse(new MasterPingResponseResponse()); - } catch (IOException e) { - logger.warn("error while sending ping response", e); - } - } - }); - } else { - // send a response, and note if we are connected to the master or not - channel.sendResponse(new MasterPingResponseResponse()); - } - } - } - - - public static class MasterPingRequest extends TransportRequest { - - public DiscoveryNode sourceNode; - - private DiscoveryNode masterNode; - private ClusterName clusterName; - - public MasterPingRequest() { - } - - public MasterPingRequest(DiscoveryNode sourceNode, DiscoveryNode masterNode, ClusterName clusterName) { - this.sourceNode = sourceNode; - this.masterNode = masterNode; - this.clusterName = clusterName; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - sourceNode = new DiscoveryNode(in); - masterNode = new DiscoveryNode(in); - clusterName = new ClusterName(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - sourceNode.writeTo(out); - masterNode.writeTo(out); - clusterName.writeTo(out); - } - } - - public static class MasterPingResponseResponse extends TransportResponse { - - public MasterPingResponseResponse() { - } - - public MasterPingResponseResponse(StreamInput in) throws IOException { - super(in); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java b/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java deleted file mode 100644 index 9b8d9f14c2e8d..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.coordination.ValidateJoinRequest; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.EmptyTransportResponseHandler; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.Collection; -import java.util.concurrent.TimeUnit; -import java.util.function.BiConsumer; -import java.util.function.Supplier; - -public class MembershipAction { - - private static final Logger logger = LogManager.getLogger(MembershipAction.class); - - public static final String DISCOVERY_JOIN_ACTION_NAME = "internal:discovery/zen/join"; - public static final String DISCOVERY_JOIN_VALIDATE_ACTION_NAME = "internal:discovery/zen/join/validate"; - public static final String DISCOVERY_LEAVE_ACTION_NAME = "internal:discovery/zen/leave"; - - public interface JoinCallback { - void onSuccess(); - - void onFailure(Exception e); - } - - public interface MembershipListener { - void onJoin(DiscoveryNode node, JoinCallback callback); - - void onLeave(DiscoveryNode node); - } - - private final TransportService transportService; - - private final MembershipListener listener; - - public MembershipAction(TransportService transportService, MembershipListener listener, - Collection> joinValidators) { - this.transportService = transportService; - this.listener = listener; - - - transportService.registerRequestHandler(DISCOVERY_JOIN_ACTION_NAME, JoinRequest::new, - ThreadPool.Names.GENERIC, new JoinRequestRequestHandler()); - transportService.registerRequestHandler(DISCOVERY_JOIN_VALIDATE_ACTION_NAME, - () -> new ValidateJoinRequest(), ThreadPool.Names.GENERIC, - new ValidateJoinRequestRequestHandler(transportService::getLocalNode, joinValidators)); - transportService.registerRequestHandler(DISCOVERY_LEAVE_ACTION_NAME, LeaveRequest::new, - ThreadPool.Names.GENERIC, new LeaveRequestRequestHandler()); - } - - public void sendLeaveRequest(DiscoveryNode masterNode, DiscoveryNode node) { - transportService.sendRequest(node, DISCOVERY_LEAVE_ACTION_NAME, new LeaveRequest(masterNode), - EmptyTransportResponseHandler.INSTANCE_SAME); - } - - public void sendLeaveRequestBlocking(DiscoveryNode masterNode, DiscoveryNode node, TimeValue timeout) { - transportService.submitRequest(masterNode, DISCOVERY_LEAVE_ACTION_NAME, new LeaveRequest(node), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(timeout.millis(), TimeUnit.MILLISECONDS); - } - - public void sendJoinRequestBlocking(DiscoveryNode masterNode, DiscoveryNode node, TimeValue timeout) { - transportService.submitRequest(masterNode, DISCOVERY_JOIN_ACTION_NAME, new JoinRequest(node), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(timeout.millis(), TimeUnit.MILLISECONDS); - } - - /** - * Validates the join request, throwing a failure if it failed. - */ - public void sendValidateJoinRequestBlocking(DiscoveryNode node, ClusterState state, TimeValue timeout) { - transportService.submitRequest(node, DISCOVERY_JOIN_VALIDATE_ACTION_NAME, new ValidateJoinRequest(state), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(timeout.millis(), TimeUnit.MILLISECONDS); - } - - public static class JoinRequest extends TransportRequest { - - private DiscoveryNode node; - - public DiscoveryNode getNode() { - return node; - } - - public JoinRequest() { - } - - public JoinRequest(DiscoveryNode node) { - this.node = node; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - node = new DiscoveryNode(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - node.writeTo(out); - } - } - - - private class JoinRequestRequestHandler implements TransportRequestHandler { - - @Override - public void messageReceived(final JoinRequest request, final TransportChannel channel, Task task) throws Exception { - listener.onJoin(request.getNode(), new JoinCallback() { - @Override - public void onSuccess() { - try { - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } catch (Exception e) { - onFailure(e); - } - } - - @Override - public void onFailure(Exception e) { - try { - channel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.warn("failed to send back failure on join request", inner); - } - } - }); - } - } - - static class ValidateJoinRequestRequestHandler implements TransportRequestHandler { - private final Supplier localNodeSupplier; - private final Collection> joinValidators; - - ValidateJoinRequestRequestHandler(Supplier localNodeSupplier, - Collection> joinValidators) { - this.localNodeSupplier = localNodeSupplier; - this.joinValidators = joinValidators; - } - - @Override - public void messageReceived(ValidateJoinRequest request, TransportChannel channel, Task task) throws Exception { - DiscoveryNode node = localNodeSupplier.get(); - assert node != null : "local node is null"; - joinValidators.stream().forEach(action -> action.accept(node, request.getState())); - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } - } - - public static class LeaveRequest extends TransportRequest { - - private DiscoveryNode node; - - public LeaveRequest() { - } - - private LeaveRequest(DiscoveryNode node) { - this.node = node; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - node = new DiscoveryNode(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - node.writeTo(out); - } - } - - private class LeaveRequestRequestHandler implements TransportRequestHandler { - - @Override - public void messageReceived(LeaveRequest request, TransportChannel channel, Task task) throws Exception { - listener.onLeave(request.node); - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java deleted file mode 100644 index 821b158bb31f4..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ /dev/null @@ -1,381 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.ClusterStateTaskListener; -import org.elasticsearch.cluster.NotMasterException; -import org.elasticsearch.cluster.coordination.JoinTaskExecutor; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * This class processes incoming join request (passed zia {@link ZenDiscovery}). Incoming nodes - * are directly added to the cluster state or are accumulated during master election. - */ -public class NodeJoinController { - - private static final Logger logger = LogManager.getLogger(NodeJoinController.class); - - private final MasterService masterService; - private final JoinTaskExecutor joinTaskExecutor; - - // this is set while trying to become a master - // mutation should be done under lock - private ElectionContext electionContext = null; - - - public NodeJoinController(Settings settings, MasterService masterService, AllocationService allocationService, - ElectMasterService electMaster) { - this.masterService = masterService; - joinTaskExecutor = new JoinTaskExecutor(settings, allocationService, logger) { - @Override - public void clusterStatePublished(ClusterChangedEvent event) { - electMaster.logMinimumMasterNodesWarningIfNecessary(event.previousState(), event.state()); - } - }; - } - - /** - * waits for enough incoming joins from master eligible nodes to complete the master election - *

- * You must start accumulating joins before calling this method. See {@link #startElectionContext()} - *

- * The method will return once the local node has been elected as master or some failure/timeout has happened. - * The exact outcome is communicated via the callback parameter, which is guaranteed to be called. - * - * @param requiredMasterJoins the number of joins from master eligible needed to complete the election - * @param timeValue how long to wait before failing. a timeout is communicated via the callback's onFailure method. - * @param callback the result of the election (success or failure) will be communicated by calling methods on this - * object - **/ - public void waitToBeElectedAsMaster(int requiredMasterJoins, TimeValue timeValue, final ElectionCallback callback) { - final CountDownLatch done = new CountDownLatch(1); - final ElectionCallback wrapperCallback = new ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - done.countDown(); - callback.onElectedAsMaster(state); - } - - @Override - public void onFailure(Throwable t) { - done.countDown(); - callback.onFailure(t); - } - }; - - ElectionContext myElectionContext = null; - - try { - // check what we have so far.. - // capture the context we add the callback to make sure we fail our own - synchronized (this) { - assert electionContext != null : "waitToBeElectedAsMaster is called we are not accumulating joins"; - myElectionContext = electionContext; - electionContext.onAttemptToBeElected(requiredMasterJoins, wrapperCallback); - checkPendingJoinsAndElectIfNeeded(); - } - - try { - if (done.await(timeValue.millis(), TimeUnit.MILLISECONDS)) { - // callback handles everything - return; - } - } catch (InterruptedException e) { - - } - if (logger.isTraceEnabled()) { - final int pendingNodes = myElectionContext.getPendingMasterJoinsCount(); - logger.trace("timed out waiting to be elected. waited [{}]. pending master node joins [{}]", timeValue, pendingNodes); - } - failContextIfNeeded(myElectionContext, "timed out waiting to be elected"); - } catch (Exception e) { - logger.error("unexpected failure while waiting for incoming joins", e); - if (myElectionContext != null) { - failContextIfNeeded(myElectionContext, "unexpected failure while waiting for pending joins [" + e.getMessage() + "]"); - } - } - } - - /** - * utility method to fail the given election context under the cluster state thread - */ - private synchronized void failContextIfNeeded(final ElectionContext context, final String reason) { - if (electionContext == context) { - stopElectionContext(reason); - } - } - - /** - * Accumulates any future incoming join request. Pending join requests will be processed in the final steps of becoming a - * master or when {@link #stopElectionContext(String)} is called. - */ - public synchronized void startElectionContext() { - logger.trace("starting an election context, will accumulate joins"); - assert electionContext == null : "double startElectionContext() calls"; - electionContext = new ElectionContext(); - } - - /** - * Stopped accumulating joins. All pending joins will be processed. Future joins will be processed immediately - */ - public void stopElectionContext(String reason) { - logger.trace("stopping election ([{}])", reason); - synchronized (this) { - assert electionContext != null : "stopElectionContext() called but not accumulating"; - electionContext.closeAndProcessPending(reason); - electionContext = null; - } - } - - /** - * processes or queues an incoming join request. - *

- * Note: doesn't do any validation. This should have been done before. - */ - public synchronized void handleJoinRequest(final DiscoveryNode node, final MembershipAction.JoinCallback callback) { - if (electionContext != null) { - electionContext.addIncomingJoin(node, callback); - checkPendingJoinsAndElectIfNeeded(); - } else { - masterService.submitStateUpdateTask("zen-disco-node-join", - new JoinTaskExecutor.Task(node, "no election context"), ClusterStateTaskConfig.build(Priority.URGENT), - joinTaskExecutor, new JoinTaskListener(callback, logger)); - } - } - - /** - * checks if there is an on going request to become master and if it has enough pending joins. If so, the node will - * become master via a ClusterState update task. - */ - private synchronized void checkPendingJoinsAndElectIfNeeded() { - assert electionContext != null : "election check requested but no active context"; - final int pendingMasterJoins = electionContext.getPendingMasterJoinsCount(); - if (electionContext.isEnoughPendingJoins(pendingMasterJoins) == false) { - if (logger.isTraceEnabled()) { - logger.trace("not enough joins for election. Got [{}], required [{}]", pendingMasterJoins, - electionContext.requiredMasterJoins); - } - } else { - if (logger.isTraceEnabled()) { - logger.trace("have enough joins for election. Got [{}], required [{}]", pendingMasterJoins, - electionContext.requiredMasterJoins); - } - electionContext.closeAndBecomeMaster(); - electionContext = null; // clear this out so future joins won't be accumulated - } - } - - public interface ElectionCallback { - /** - * called when the local node is successfully elected as master - * Guaranteed to be called on the cluster state update thread - **/ - void onElectedAsMaster(ClusterState state); - - /** - * called when the local node failed to be elected as master - * Guaranteed to be called on the cluster state update thread - **/ - void onFailure(Throwable t); - } - - class ElectionContext { - private ElectionCallback callback = null; - private int requiredMasterJoins = -1; - private final Map> joinRequestAccumulator = new HashMap<>(); - - final AtomicBoolean closed = new AtomicBoolean(); - - public synchronized void onAttemptToBeElected(int requiredMasterJoins, ElectionCallback callback) { - ensureOpen(); - assert this.requiredMasterJoins < 0; - assert this.callback == null; - this.requiredMasterJoins = requiredMasterJoins; - this.callback = callback; - } - - public synchronized void addIncomingJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) { - ensureOpen(); - joinRequestAccumulator.computeIfAbsent(node, n -> new ArrayList<>()).add(callback); - } - - - public synchronized boolean isEnoughPendingJoins(int pendingMasterJoins) { - final boolean hasEnough; - if (requiredMasterJoins < 0) { - // requiredMasterNodes is unknown yet, return false and keep on waiting - hasEnough = false; - } else { - assert callback != null : "requiredMasterJoins is set but not the callback"; - hasEnough = pendingMasterJoins >= requiredMasterJoins; - } - return hasEnough; - } - - private Map getPendingAsTasks(String reason) { - Map tasks = new HashMap<>(); - joinRequestAccumulator.entrySet().stream().forEach(e -> tasks.put( - new JoinTaskExecutor.Task(e.getKey(), reason), new JoinTaskListener(e.getValue(), logger))); - return tasks; - } - - public synchronized int getPendingMasterJoinsCount() { - int pendingMasterJoins = 0; - for (DiscoveryNode node : joinRequestAccumulator.keySet()) { - if (node.isMasterNode()) { - pendingMasterJoins++; - } - } - return pendingMasterJoins; - } - - public synchronized void closeAndBecomeMaster() { - assert callback != null : "becoming a master but the callback is not yet set"; - assert isEnoughPendingJoins(getPendingMasterJoinsCount()) : "becoming a master but pending joins of " - + getPendingMasterJoinsCount() + " are not enough. needs [" + requiredMasterJoins + "];"; - - innerClose(); - - Map tasks = getPendingAsTasks("become master"); - final String source = "zen-disco-elected-as-master ([" + tasks.size() + "] nodes joined)"; - - // noop listener, the election finished listener determines result - tasks.put(JoinTaskExecutor.newBecomeMasterTask(), (source1, e) -> {}); - tasks.put(JoinTaskExecutor.newFinishElectionTask(), electionFinishedListener); - masterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); - } - - public synchronized void closeAndProcessPending(String reason) { - innerClose(); - Map tasks = getPendingAsTasks(reason); - final String source = "zen-disco-election-stop [" + reason + "]"; - tasks.put(JoinTaskExecutor.newFinishElectionTask(), electionFinishedListener); - masterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor); - } - - private void innerClose() { - if (closed.getAndSet(true)) { - throw new AlreadyClosedException("election context is already closed"); - } - } - - private void ensureOpen() { - if (closed.get()) { - throw new AlreadyClosedException("election context is already closed"); - } - } - - private synchronized ElectionCallback getCallback() { - return callback; - } - - private void onElectedAsMaster(ClusterState state) { - assert MasterService.assertMasterUpdateThread(); - assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master"; - ElectionCallback callback = getCallback(); // get under lock - if (callback != null) { - callback.onElectedAsMaster(state); - } - } - - private void onFailure(Throwable t) { - assert MasterService.assertMasterUpdateThread(); - ElectionCallback callback = getCallback(); // get under lock - if (callback != null) { - callback.onFailure(t); - } - } - - private final ClusterStateTaskListener electionFinishedListener = new ClusterStateTaskListener() { - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - if (newState.nodes().isLocalNodeElectedMaster()) { - ElectionContext.this.onElectedAsMaster(newState); - } else { - onFailure(source, new NotMasterException("election stopped [" + source + "]")); - } - } - - @Override - public void onFailure(String source, Exception e) { - ElectionContext.this.onFailure(e); - } - }; - - } - - static class JoinTaskListener implements ClusterStateTaskListener { - final List callbacks; - private final Logger logger; - - JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) { - this(Collections.singletonList(callback), logger); - } - - JoinTaskListener(List callbacks, Logger logger) { - this.callbacks = callbacks; - this.logger = logger; - } - - @Override - public void onFailure(String source, Exception e) { - for (MembershipAction.JoinCallback callback : callbacks) { - try { - callback.onFailure(e); - } catch (Exception inner) { - logger.error(() -> new ParameterizedMessage("error handling task failure [{}]", e), inner); - } - } - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - for (MembershipAction.JoinCallback callback : callbacks) { - try { - callback.onSuccess(); - } catch (Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected error during [{}]", source), e); - } - } - } - } - -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/NodesFaultDetection.java b/server/src/main/java/org/elasticsearch/discovery/zen/NodesFaultDetection.java deleted file mode 100644 index 4b89c9f19dc59..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/NodesFaultDetection.java +++ /dev/null @@ -1,370 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -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.common.util.concurrent.EsRejectedExecutionException; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.Collections; -import java.util.Set; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.function.Supplier; - -import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; - -/** - * A fault detection of multiple nodes. - */ -public class NodesFaultDetection extends FaultDetection { - - private static final Logger logger = LogManager.getLogger(NodesFaultDetection.class); - - public static final String PING_ACTION_NAME = "internal:discovery/zen/fd/ping"; - - public abstract static class Listener { - - public void onNodeFailure(DiscoveryNode node, String reason) {} - - public void onPingReceived(PingRequest pingRequest) {} - - } - - private final CopyOnWriteArrayList listeners = new CopyOnWriteArrayList<>(); - - private final ConcurrentMap nodesFD = newConcurrentMap(); - - private final Supplier clusterStateSupplier; - - private volatile DiscoveryNode localNode; - - public NodesFaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService, - Supplier clusterStateSupplier, ClusterName clusterName) { - super(settings, threadPool, transportService, clusterName); - - this.clusterStateSupplier = clusterStateSupplier; - - logger.debug("[node ] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]", pingInterval, pingRetryTimeout, - pingRetryCount); - - transportService.registerRequestHandler( - PING_ACTION_NAME, PingRequest::new, ThreadPool.Names.SAME, false, false, new PingRequestHandler()); - } - - public void setLocalNode(DiscoveryNode localNode) { - this.localNode = localNode; - } - - public void addListener(Listener listener) { - listeners.add(listener); - } - - public void removeListener(Listener listener) { - listeners.remove(listener); - } - - /** - * Gets the current set of nodes involved in node fault detection. - * NB: For testing purposes. - */ - public Set getNodes() { - return Collections.unmodifiableSet(nodesFD.keySet()); - } - - /** - * make sure that nodes in clusterState are pinged. Any pinging to nodes which are not - * part of the cluster will be stopped - */ - public void updateNodesAndPing(ClusterState clusterState) { - // remove any nodes we don't need, this will cause their FD to stop - for (DiscoveryNode monitoredNode : nodesFD.keySet()) { - if (!clusterState.nodes().nodeExists(monitoredNode)) { - nodesFD.remove(monitoredNode); - } - } - // add any missing nodes - - for (DiscoveryNode node : clusterState.nodes()) { - if (node.equals(localNode)) { - // no need to monitor the local node - continue; - } - if (!nodesFD.containsKey(node)) { - NodeFD fd = new NodeFD(node); - // it's OK to overwrite an existing nodeFD - it will just stop and the new one will pick things up. - nodesFD.put(node, fd); - // we use schedule with a 0 time value to run the pinger on the pool as it will run on later - threadPool.schedule(fd, TimeValue.timeValueMillis(0), ThreadPool.Names.SAME); - } - } - } - - /** stops all pinging **/ - public NodesFaultDetection stop() { - nodesFD.clear(); - return this; - } - - @Override - public void close() { - super.close(); - stop(); - } - - @Override - protected void handleTransportDisconnect(DiscoveryNode node) { - NodeFD nodeFD = nodesFD.remove(node); - if (nodeFD == null) { - return; - } - if (connectOnNetworkDisconnect) { - NodeFD fd = new NodeFD(node); - try { - transportService.connectToNode(node); - nodesFD.put(node, fd); - // we use schedule with a 0 time value to run the pinger on the pool as it will run on later - threadPool.schedule(fd, TimeValue.timeValueMillis(0), ThreadPool.Names.SAME); - } catch (Exception e) { - logger.trace("[node ] [{}] transport disconnected (with verified connect)", node); - // clean up if needed, just to be safe.. - nodesFD.remove(node, fd); - notifyNodeFailure(node, "transport disconnected (with verified connect)"); - } - } else { - logger.trace("[node ] [{}] transport disconnected", node); - notifyNodeFailure(node, "transport disconnected"); - } - } - - private void notifyNodeFailure(final DiscoveryNode node, final String reason) { - try { - threadPool.generic().execute(new Runnable() { - @Override - public void run() { - for (Listener listener : listeners) { - listener.onNodeFailure(node, reason); - } - } - }); - } catch (EsRejectedExecutionException ex) { - logger.trace(() -> new ParameterizedMessage( - "[node ] [{}] ignoring node failure (reason [{}]). Local node is shutting down", node, reason), ex); - } - } - - private void notifyPingReceived(final PingRequest pingRequest) { - threadPool.generic().execute(new Runnable() { - - @Override - public void run() { - for (Listener listener : listeners) { - listener.onPingReceived(pingRequest); - } - } - - }); - } - - - private class NodeFD implements Runnable { - volatile int retryCount; - - private final DiscoveryNode node; - - private NodeFD(DiscoveryNode node) { - this.node = node; - } - - private boolean running() { - return NodeFD.this.equals(nodesFD.get(node)); - } - - private PingRequest newPingRequest() { - return new PingRequest(node, clusterName, localNode, clusterStateSupplier.get().version()); - } - - @Override - public void run() { - if (!running()) { - return; - } - final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING) - .withTimeout(pingRetryTimeout).build(); - transportService.sendRequest(node, PING_ACTION_NAME, newPingRequest(), options, new TransportResponseHandler() { - @Override - public PingResponse read(StreamInput in) throws IOException { - return new PingResponse(in); - } - - @Override - public void handleResponse(PingResponse response) { - if (!running()) { - return; - } - retryCount = 0; - threadPool.schedule(NodeFD.this, pingInterval, ThreadPool.Names.SAME); - } - - @Override - public void handleException(TransportException exp) { - if (!running()) { - return; - } - if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) { - handleTransportDisconnect(node); - return; - } - - retryCount++; - logger.trace( () -> new ParameterizedMessage( - "[node ] failed to ping [{}], retry [{}] out of [{}]", node, retryCount, pingRetryCount), exp); - if (retryCount >= pingRetryCount) { - logger.debug("[node ] failed to ping [{}], tried [{}] times, each with maximum [{}] timeout", node, - pingRetryCount, pingRetryTimeout); - // not good, failure - if (nodesFD.remove(node, NodeFD.this)) { - notifyNodeFailure(node, "failed to ping, tried [" + pingRetryCount + "] times, each with maximum [" - + pingRetryTimeout + "] timeout"); - } - } else { - // resend the request, not reschedule, rely on send timeout - transportService.sendRequest(node, PING_ACTION_NAME, newPingRequest(), options, this); - } - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - } - ); - } - } - - class PingRequestHandler implements TransportRequestHandler { - @Override - public void messageReceived(PingRequest request, TransportChannel channel, Task task) throws Exception { - // if we are not the node we are supposed to be pinged, send an exception - // this can happen when a kill -9 is sent, and another node is started using the same port - if (!localNode.equals(request.targetNode())) { - throw new IllegalStateException("Got pinged as node " + request.targetNode() + "], but I am node " + localNode ); - } - - // PingRequest will have clusterName set to null if it came from a node of version <1.4.0 - if (request.clusterName != null && !request.clusterName.equals(clusterName)) { - // Don't introduce new exception for bwc reasons - throw new IllegalStateException("Got pinged with cluster name [" + request.clusterName + "], but I'm part of cluster [" - + clusterName + "]"); - } - - notifyPingReceived(request); - - channel.sendResponse(new PingResponse()); - } - } - - - public static class PingRequest extends TransportRequest { - - // the (assumed) node we are pinging - private DiscoveryNode targetNode; - - private ClusterName clusterName; - - private DiscoveryNode masterNode; - - private long clusterStateVersion = ClusterState.UNKNOWN_VERSION; - - public PingRequest() { - } - - public PingRequest(DiscoveryNode targetNode, ClusterName clusterName, DiscoveryNode masterNode, long clusterStateVersion) { - this.targetNode = targetNode; - this.clusterName = clusterName; - this.masterNode = masterNode; - this.clusterStateVersion = clusterStateVersion; - } - - public DiscoveryNode targetNode() { - return targetNode; - } - - public ClusterName clusterName() { - return clusterName; - } - - public DiscoveryNode masterNode() { - return masterNode; - } - - public long clusterStateVersion() { - return clusterStateVersion; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - targetNode = new DiscoveryNode(in); - clusterName = new ClusterName(in); - masterNode = new DiscoveryNode(in); - clusterStateVersion = in.readLong(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - targetNode.writeTo(out); - clusterName.writeTo(out); - masterNode.writeTo(out); - out.writeLong(clusterStateVersion); - } - } - - public static class PingResponse extends TransportResponse { - - public PingResponse() { - } - - public PingResponse(StreamInput in) throws IOException { - super(in); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java b/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java deleted file mode 100644 index d471a32d5a9f3..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java +++ /dev/null @@ -1,310 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.coordination.PendingClusterStateStats; -import org.elasticsearch.cluster.node.DiscoveryNode; - -import java.util.ArrayList; -import java.util.Locale; -import java.util.Objects; - -/** - * A queue that holds all "in-flight" incoming cluster states from the master. Once a master commits a cluster - * state, it is made available via {@link #getNextClusterStateToProcess()}. The class also takes care of batching - * cluster states for processing and failures. - *

- * The queue is bound by {@link #maxQueueSize}. When the queue is at capacity and a new cluster state is inserted - * the oldest cluster state will be dropped. This is safe because: - * 1) Under normal operations, master will publish & commit a cluster state before processing - * another change (i.e., the queue length is 1) - * 2) If the master fails to commit a change, it will step down, causing a master election, which will flush the queue. - * 3) In general it's safe to process the incoming cluster state as a replacement to the cluster state that's dropped. - * a) If the dropped cluster is from the same master as the incoming one is, it is likely to be superseded by the - * incoming state (or another state in the queue). - * This is only not true in very extreme cases of out of order delivery. - * b) If the dropping cluster state is not from the same master, it means that: - * i) we are no longer following the master of the dropped cluster state but follow the incoming one - * ii) we are no longer following any master, in which case it doesn't matter which cluster state will be processed first. - *

- * The class is fully thread safe and can be used concurrently. - */ -public class PendingClusterStatesQueue { - - interface StateProcessedListener { - - void onNewClusterStateProcessed(); - - void onNewClusterStateFailed(Exception e); - } - - final ArrayList pendingStates = new ArrayList<>(); - final Logger logger; - final int maxQueueSize; - - public PendingClusterStatesQueue(Logger logger, int maxQueueSize) { - this.logger = logger; - this.maxQueueSize = maxQueueSize; - } - - /** Add an incoming, not yet committed cluster state */ - public synchronized void addPending(ClusterState state) { - pendingStates.add(new ClusterStateContext(state)); - if (pendingStates.size() > maxQueueSize) { - ClusterStateContext context = pendingStates.remove(0); - logger.warn("dropping pending state [{}]. more than [{}] pending states.", context, maxQueueSize); - if (context.committed()) { - context.listener.onNewClusterStateFailed(new ElasticsearchException("too many pending states ([{}] pending)", - maxQueueSize)); - } - } - } - - /** - * Mark a previously added cluster state as committed. This will make it available via {@link #getNextClusterStateToProcess()} - * When the cluster state is processed (or failed), the supplied listener will be called - **/ - public synchronized ClusterState markAsCommitted(String stateUUID, StateProcessedListener listener) { - final ClusterStateContext context = findState(stateUUID); - if (context == null) { - listener.onNewClusterStateFailed(new IllegalStateException("can't resolve cluster state with uuid" + - " [" + stateUUID + "] to commit")); - return null; - } - if (context.committed()) { - listener.onNewClusterStateFailed(new IllegalStateException("cluster state with uuid" + - " [" + stateUUID + "] is already committed")); - return null; - } - context.markAsCommitted(listener); - return context.state; - } - - /** - * mark that the processing of the given state has failed. All committed states that are - * {@link ClusterState#supersedes(ClusterState)}-ed by this failed state, will be failed as well - */ - public synchronized void markAsFailed(ClusterState state, Exception reason) { - final ClusterStateContext failedContext = findState(state.stateUUID()); - if (failedContext == null) { - throw new IllegalArgumentException("can't resolve failed cluster state with uuid [" + state.stateUUID() - + "], version [" + state.version() + "]"); - } - if (failedContext.committed() == false) { - throw new IllegalArgumentException("failed cluster state is not committed " + state); - } - - // fail all committed states which are batch together with the failed state - ArrayList statesToRemove = new ArrayList<>(); - for (int index = 0; index < pendingStates.size(); index++) { - final ClusterStateContext pendingContext = pendingStates.get(index); - if (pendingContext.committed() == false) { - continue; - } - final ClusterState pendingState = pendingContext.state; - if (pendingContext.equals(failedContext)) { - statesToRemove.add(pendingContext); - pendingContext.listener.onNewClusterStateFailed(reason); - } else if (state.supersedes(pendingState)) { - statesToRemove.add(pendingContext); - logger.debug("failing committed state {} together with state {}", pendingContext, failedContext); - pendingContext.listener.onNewClusterStateFailed(reason); - } - } - pendingStates.removeAll(statesToRemove); - assert findState(state.stateUUID()) == null : "state was marked as processed but can still be found in pending list " + state; - } - - /** - * indicates that a cluster state was successfully processed. Any committed state that is - * {@link ClusterState#supersedes(ClusterState)}-ed by the processed state will be marked as processed as well. - *

- * NOTE: successfully processing a state indicates we are following the master it came from. Any committed state - * from another master will be failed by this method - */ - public synchronized void markAsProcessed(ClusterState state) { - if (findState(state.stateUUID()) == null) { - throw new IllegalStateException("can't resolve processed cluster state with uuid [" + state.stateUUID() - + "], version [" + state.version() + "]"); - } - final DiscoveryNode currentMaster = state.nodes().getMasterNode(); - assert currentMaster != null : "processed cluster state mast have a master. " + state; - - // fail or remove any incoming state from a different master - // respond to any committed state from the same master with same or lower version (we processed a higher version) - ArrayList contextsToRemove = new ArrayList<>(); - for (int index = 0; index < pendingStates.size(); index++) { - final ClusterStateContext pendingContext = pendingStates.get(index); - final ClusterState pendingState = pendingContext.state; - final DiscoveryNode pendingMasterNode = pendingState.nodes().getMasterNode(); - if (Objects.equals(currentMaster, pendingMasterNode) == false) { - contextsToRemove.add(pendingContext); - if (pendingContext.committed()) { - // this is a committed state , warn - logger.warn("received a cluster state (uuid[{}]/v[{}]) from a different master than the current one," - + " rejecting (received {}, current {})", - pendingState.stateUUID(), pendingState.version(), pendingMasterNode, currentMaster); - pendingContext.listener.onNewClusterStateFailed( - new IllegalStateException("cluster state from a different master than the current one," + - " rejecting (received " + pendingMasterNode + ", current " + currentMaster + ")")); - } else { - logger.trace("removing non-committed state with uuid[{}]/v[{}] from [{}] - a state from" + - " [{}] was successfully processed", - pendingState.stateUUID(), pendingState.version(), pendingMasterNode, currentMaster); - } - } else if (pendingState.stateUUID().equals(state.stateUUID())) { - assert pendingContext.committed() : "processed cluster state is not committed " + state; - contextsToRemove.add(pendingContext); - pendingContext.listener.onNewClusterStateProcessed(); - } else if (state.version() >= pendingState.version()) { - logger.trace("processing pending state uuid[{}]/v[{}] together with state uuid[{}]/v[{}]", - pendingState.stateUUID(), pendingState.version(), state.stateUUID(), state.version() - ); - contextsToRemove.add(pendingContext); - if (pendingContext.committed()) { - pendingContext.listener.onNewClusterStateProcessed(); - } - } - } - // now ack the processed state - pendingStates.removeAll(contextsToRemove); - assert findState(state.stateUUID()) == null : "state was marked as processed but can still be found in pending list " + state; - - } - - ClusterStateContext findState(String stateUUID) { - for (int i = 0; i < pendingStates.size(); i++) { - final ClusterStateContext context = pendingStates.get(i); - if (context.stateUUID().equals(stateUUID)) { - return context; - } - } - return null; - } - - /** clear the incoming queue. any committed state will be failed - */ - public synchronized void failAllStatesAndClear(Exception reason) { - for (ClusterStateContext pendingState : pendingStates) { - if (pendingState.committed()) { - pendingState.listener.onNewClusterStateFailed(reason); - } - } - pendingStates.clear(); - } - - /** - * Gets the next committed state to process. - *

- * The method tries to batch operation by getting the cluster state the highest possible committed states - * which succeeds the first committed state in queue (i.e., it comes from the same master). - */ - public synchronized ClusterState getNextClusterStateToProcess() { - if (pendingStates.isEmpty()) { - return null; - } - - ClusterStateContext stateToProcess = null; - int index = 0; - for (; index < pendingStates.size(); index++) { - ClusterStateContext potentialState = pendingStates.get(index); - if (potentialState.committed()) { - stateToProcess = potentialState; - break; - } - } - if (stateToProcess == null) { - return null; - } - - // now try to find the highest committed state from the same master - for (; index < pendingStates.size(); index++) { - ClusterStateContext potentialState = pendingStates.get(index); - - if (potentialState.state.supersedes(stateToProcess.state) && potentialState.committed()) { - // we found a new one - stateToProcess = potentialState; - } - } - assert stateToProcess.committed() : "should only return committed cluster state. found " + stateToProcess.state; - return stateToProcess.state; - } - - /** returns all pending states, committed or not */ - public synchronized ClusterState[] pendingClusterStates() { - ArrayList states = new ArrayList<>(); - for (ClusterStateContext context : pendingStates) { - states.add(context.state); - } - return states.toArray(new ClusterState[states.size()]); - } - - static class ClusterStateContext { - final ClusterState state; - StateProcessedListener listener; - - ClusterStateContext(ClusterState clusterState) { - this.state = clusterState; - } - - void markAsCommitted(StateProcessedListener listener) { - if (this.listener != null) { - throw new IllegalStateException(toString() + "is already committed"); - } - this.listener = listener; - } - - boolean committed() { - return listener != null; - } - - public String stateUUID() { - return state.stateUUID(); - } - - @Override - public String toString() { - return String.format( - Locale.ROOT, - "[uuid[%s], v[%d], m[%s]]", - stateUUID(), - state.version(), - state.nodes().getMasterNodeId() - ); - } - } - - public synchronized PendingClusterStateStats stats() { - - // calculate committed cluster state - int committed = 0; - for (ClusterStateContext clusterStatsContext : pendingStates) { - if (clusterStatsContext.committed()) { - committed += 1; - } - } - - return new PendingClusterStateStats(pendingStates.size(), pendingStates.size() - committed, committed); - } - -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PingContextProvider.java b/server/src/main/java/org/elasticsearch/discovery/zen/PingContextProvider.java deleted file mode 100644 index 7567b69cfe459..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PingContextProvider.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.elasticsearch.cluster.ClusterState; - -public interface PingContextProvider { - - /** return the current cluster state of the node */ - ClusterState clusterState(); -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java b/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java deleted file mode 100644 index 107053816e86b..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java +++ /dev/null @@ -1,664 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.Diff; -import org.elasticsearch.cluster.IncompatibleClusterStateVersionException; -import org.elasticsearch.cluster.coordination.PublishClusterStateStats; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.Compressor; -import org.elasticsearch.common.compress.CompressorFactory; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.AckClusterStatePublishResponseHandler; -import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler; -import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.BytesTransportRequest; -import org.elasticsearch.transport.EmptyTransportResponseHandler; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; - -public class PublishClusterStateAction { - - private static final Logger logger = LogManager.getLogger(PublishClusterStateAction.class); - - public static final String SEND_ACTION_NAME = "internal:discovery/zen/publish/send"; - public static final String COMMIT_ACTION_NAME = "internal:discovery/zen/publish/commit"; - - // -> no need to put a timeout on the options, because we want the state response to eventually be received - // and not log an error if it arrives after the timeout - private final TransportRequestOptions stateRequestOptions = TransportRequestOptions.builder() - .withType(TransportRequestOptions.Type.STATE).build(); - - public interface IncomingClusterStateListener { - - /** - * called when a new incoming cluster state has been received. - * Should validate the incoming state and throw an exception if it's not a valid successor state. - */ - void onIncomingClusterState(ClusterState incomingState); - - /** - * called when a cluster state has been committed and is ready to be processed - */ - void onClusterStateCommitted(String stateUUID, ActionListener processedListener); - } - - private final TransportService transportService; - private final NamedWriteableRegistry namedWriteableRegistry; - private final IncomingClusterStateListener incomingClusterStateListener; - private final DiscoverySettings discoverySettings; - - private final AtomicLong fullClusterStateReceivedCount = new AtomicLong(); - private final AtomicLong incompatibleClusterStateDiffReceivedCount = new AtomicLong(); - private final AtomicLong compatibleClusterStateDiffReceivedCount = new AtomicLong(); - - public PublishClusterStateAction( - TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, - IncomingClusterStateListener incomingClusterStateListener, - DiscoverySettings discoverySettings) { - this.transportService = transportService; - this.namedWriteableRegistry = namedWriteableRegistry; - this.incomingClusterStateListener = incomingClusterStateListener; - this.discoverySettings = discoverySettings; - transportService.registerRequestHandler(SEND_ACTION_NAME, BytesTransportRequest::new, ThreadPool.Names.SAME, false, false, - new SendClusterStateRequestHandler()); - transportService.registerRequestHandler(COMMIT_ACTION_NAME, CommitClusterStateRequest::new, ThreadPool.Names.SAME, false, false, - new CommitClusterStateRequestHandler()); - } - - /** - * publishes a cluster change event to other nodes. if at least minMasterNodes acknowledge the change it is committed and will - * be processed by the master and the other nodes. - *

- * The method is guaranteed to throw a {@link FailedToCommitClusterStateException} - * if the change is not committed and should be rejected. - * Any other exception signals the something wrong happened but the change is committed. - */ - public void publish(final ClusterChangedEvent clusterChangedEvent, final int minMasterNodes, - final Discovery.AckListener ackListener) throws FailedToCommitClusterStateException { - final DiscoveryNodes nodes; - final SendingController sendingController; - final Set nodesToPublishTo; - final Map serializedStates; - final Map serializedDiffs; - final boolean sendFullVersion; - try { - nodes = clusterChangedEvent.state().nodes(); - nodesToPublishTo = new HashSet<>(nodes.getSize()); - DiscoveryNode localNode = nodes.getLocalNode(); - final int totalMasterNodes = nodes.getMasterNodes().size(); - for (final DiscoveryNode node : nodes) { - if (node.equals(localNode) == false) { - nodesToPublishTo.add(node); - } - } - sendFullVersion = !discoverySettings.getPublishDiff() || clusterChangedEvent.previousState() == null; - serializedStates = new HashMap<>(); - serializedDiffs = new HashMap<>(); - - // we build these early as a best effort not to commit in the case of error. - // sadly this is not water tight as it may that a failed diff based publishing to a node - // will cause a full serialization based on an older version, which may fail after the - // change has been committed. - buildDiffAndSerializeStates(clusterChangedEvent.state(), clusterChangedEvent.previousState(), - nodesToPublishTo, sendFullVersion, serializedStates, serializedDiffs); - - final BlockingClusterStatePublishResponseHandler publishResponseHandler = - new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener); - sendingController = new SendingController(clusterChangedEvent.state(), minMasterNodes, - totalMasterNodes, publishResponseHandler); - } catch (Exception e) { - throw new FailedToCommitClusterStateException("unexpected error while preparing to publish", e); - } - - try { - innerPublish(clusterChangedEvent, nodesToPublishTo, sendingController, ackListener, sendFullVersion, serializedStates, - serializedDiffs); - } catch (FailedToCommitClusterStateException t) { - throw t; - } catch (Exception e) { - // try to fail committing, in cause it's still on going - if (sendingController.markAsFailed("unexpected error", e)) { - // signal the change should be rejected - throw new FailedToCommitClusterStateException("unexpected error", e); - } else { - throw e; - } - } - } - - private void innerPublish(final ClusterChangedEvent clusterChangedEvent, final Set nodesToPublishTo, - final SendingController sendingController, final Discovery.AckListener ackListener, - final boolean sendFullVersion, final Map serializedStates, - final Map serializedDiffs) { - - final ClusterState clusterState = clusterChangedEvent.state(); - final ClusterState previousState = clusterChangedEvent.previousState(); - final TimeValue publishTimeout = discoverySettings.getPublishTimeout(); - - final long publishingStartInNanos = System.nanoTime(); - - for (final DiscoveryNode node : nodesToPublishTo) { - // try and serialize the cluster state once (or per version), so we don't serialize it - // per node when we send it over the wire, compress it while we are at it... - // we don't send full version if node didn't exist in the previous version of cluster state - if (sendFullVersion || !previousState.nodes().nodeExists(node)) { - sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController); - } else { - sendClusterStateDiff(clusterState, serializedDiffs, serializedStates, node, publishTimeout, sendingController); - } - } - - sendingController.waitForCommit(discoverySettings.getCommitTimeout()); - - final long commitTime = System.nanoTime() - publishingStartInNanos; - - ackListener.onCommit(TimeValue.timeValueNanos(commitTime)); - - try { - long timeLeftInNanos = Math.max(0, publishTimeout.nanos() - commitTime); - final BlockingClusterStatePublishResponseHandler publishResponseHandler = sendingController.getPublishResponseHandler(); - sendingController.setPublishingTimedOut(!publishResponseHandler.awaitAllNodes(TimeValue.timeValueNanos(timeLeftInNanos))); - if (sendingController.getPublishingTimedOut()) { - DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes(); - // everyone may have just responded - if (pendingNodes.length > 0) { - logger.warn("timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})", - clusterState.version(), publishTimeout, pendingNodes); - } - } - // The failure is logged under debug when a sending failed. we now log a summary. - Set failedNodes = publishResponseHandler.getFailedNodes(); - if (failedNodes.isEmpty() == false) { - logger.warn("publishing cluster state with version [{}] failed for the following nodes: [{}]", - clusterChangedEvent.state().version(), failedNodes); - } - } catch (InterruptedException e) { - // ignore & restore interrupt - Thread.currentThread().interrupt(); - } - } - - private void buildDiffAndSerializeStates(ClusterState clusterState, ClusterState previousState, Set nodesToPublishTo, - boolean sendFullVersion, Map serializedStates, - Map serializedDiffs) { - Diff diff = null; - for (final DiscoveryNode node : nodesToPublishTo) { - try { - if (sendFullVersion || !previousState.nodes().nodeExists(node)) { - // will send a full reference - if (serializedStates.containsKey(node.getVersion()) == false) { - serializedStates.put(node.getVersion(), serializeFullClusterState(clusterState, node.getVersion())); - } - } else { - // will send a diff - if (diff == null) { - diff = clusterState.diff(previousState); - } - if (serializedDiffs.containsKey(node.getVersion()) == false) { - serializedDiffs.put(node.getVersion(), serializeDiffClusterState(diff, node.getVersion())); - } - } - } catch (IOException e) { - throw new ElasticsearchException("failed to serialize cluster_state for publishing to node {}", e, node); - } - } - } - - private void sendFullClusterState(ClusterState clusterState, Map serializedStates, - DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) { - BytesReference bytes = serializedStates.get(node.getVersion()); - if (bytes == null) { - try { - bytes = serializeFullClusterState(clusterState, node.getVersion()); - serializedStates.put(node.getVersion(), bytes); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to serialize cluster_state before publishing it to node {}", node), e); - sendingController.onNodeSendFailed(node, e); - return; - } - } - sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, false, serializedStates); - } - - private void sendClusterStateDiff(ClusterState clusterState, - Map serializedDiffs, Map serializedStates, - DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) { - BytesReference bytes = serializedDiffs.get(node.getVersion()); - assert bytes != null : "failed to find serialized diff for node " + node + " of version [" + node.getVersion() + "]"; - sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, true, serializedStates); - } - - private void sendClusterStateToNode(final ClusterState clusterState, BytesReference bytes, - final DiscoveryNode node, - final TimeValue publishTimeout, - final SendingController sendingController, - final boolean sendDiffs, final Map serializedStates) { - try { - - transportService.sendRequest(node, SEND_ACTION_NAME, - new BytesTransportRequest(bytes, node.getVersion()), - stateRequestOptions, - new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - - @Override - public void handleResponse(TransportResponse.Empty response) { - if (sendingController.getPublishingTimedOut()) { - logger.debug("node {} responded for cluster state [{}] (took longer than [{}])", node, - clusterState.version(), publishTimeout); - } - sendingController.onNodeSendAck(node); - } - - @Override - public void handleException(TransportException exp) { - if (sendDiffs && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) { - logger.debug("resending full cluster state to node {} reason {}", node, exp.getDetailedMessage()); - sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController); - } else { - logger.debug(() -> new ParameterizedMessage("failed to send cluster state to {}", node), exp); - sendingController.onNodeSendFailed(node, exp); - } - } - }); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("error sending cluster state to {}", node), e); - sendingController.onNodeSendFailed(node, e); - } - } - - private void sendCommitToNode(final DiscoveryNode node, final ClusterState clusterState, final SendingController sendingController) { - try { - logger.trace("sending commit for cluster state (uuid: [{}], version [{}]) to [{}]", - clusterState.stateUUID(), clusterState.version(), node); - transportService.sendRequest(node, COMMIT_ACTION_NAME, - new CommitClusterStateRequest(clusterState.stateUUID()), - stateRequestOptions, - new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - - @Override - public void handleResponse(TransportResponse.Empty response) { - if (sendingController.getPublishingTimedOut()) { - logger.debug("node {} responded to cluster state commit [{}]", node, clusterState.version()); - } - sendingController.getPublishResponseHandler().onResponse(node); - } - - @Override - public void handleException(TransportException exp) { - logger.debug(() -> new ParameterizedMessage("failed to commit cluster state (uuid [{}], version [{}]) to {}", - clusterState.stateUUID(), clusterState.version(), node), exp); - sendingController.getPublishResponseHandler().onFailure(node, exp); - } - }); - } catch (Exception t) { - logger.warn(() -> new ParameterizedMessage("error sending cluster state commit (uuid [{}], version [{}]) to {}", - clusterState.stateUUID(), clusterState.version(), node), t); - sendingController.getPublishResponseHandler().onFailure(node, t); - } - } - - - public static BytesReference serializeFullClusterState(ClusterState clusterState, Version nodeVersion) throws IOException { - BytesStreamOutput bStream = new BytesStreamOutput(); - try (StreamOutput stream = CompressorFactory.COMPRESSOR.streamOutput(bStream)) { - stream.setVersion(nodeVersion); - stream.writeBoolean(true); - clusterState.writeTo(stream); - } - return bStream.bytes(); - } - - public static BytesReference serializeDiffClusterState(Diff diff, Version nodeVersion) throws IOException { - BytesStreamOutput bStream = new BytesStreamOutput(); - try (StreamOutput stream = CompressorFactory.COMPRESSOR.streamOutput(bStream)) { - stream.setVersion(nodeVersion); - stream.writeBoolean(false); - diff.writeTo(stream); - } - return bStream.bytes(); - } - - private Object lastSeenClusterStateMutex = new Object(); - private ClusterState lastSeenClusterState; - - protected void handleIncomingClusterStateRequest(BytesTransportRequest request, TransportChannel channel) throws IOException { - Compressor compressor = CompressorFactory.compressor(request.bytes()); - StreamInput in = request.bytes().streamInput(); - final ClusterState incomingState; - synchronized (lastSeenClusterStateMutex) { - try { - if (compressor != null) { - in = compressor.streamInput(in); - } - in = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry); - in.setVersion(request.version()); - // If true we received full cluster state - otherwise diffs - if (in.readBoolean()) { - incomingState = ClusterState.readFrom(in, transportService.getLocalNode()); - fullClusterStateReceivedCount.incrementAndGet(); - logger.debug("received full cluster state version [{}] with size [{}]", incomingState.version(), - request.bytes().length()); - } else if (lastSeenClusterState != null) { - Diff diff = ClusterState.readDiffFrom(in, lastSeenClusterState.nodes().getLocalNode()); - incomingState = diff.apply(lastSeenClusterState); - compatibleClusterStateDiffReceivedCount.incrementAndGet(); - logger.debug("received diff cluster state version [{}] with uuid [{}], diff size [{}]", - incomingState.version(), incomingState.stateUUID(), request.bytes().length()); - } else { - logger.debug("received diff for but don't have any local cluster state - requesting full state"); - throw new IncompatibleClusterStateVersionException("have no local cluster state"); - } - } catch (IncompatibleClusterStateVersionException e) { - incompatibleClusterStateDiffReceivedCount.incrementAndGet(); - throw e; - } catch (Exception e) { - logger.warn("unexpected error while deserializing an incoming cluster state", e); - throw e; - } finally { - IOUtils.close(in); - } - incomingClusterStateListener.onIncomingClusterState(incomingState); - lastSeenClusterState = incomingState; - } - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } - - protected void handleCommitRequest(CommitClusterStateRequest request, final TransportChannel channel) { - incomingClusterStateListener.onClusterStateCommitted(request.stateUUID, new ActionListener() { - - @Override - public void onResponse(Void ignore) { - try { - // send a response to the master to indicate that this cluster state has been processed post committing it. - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } catch (Exception e) { - logger.debug("failed to send response on cluster state processed", e); - onFailure(e); - } - } - - @Override - public void onFailure(Exception e) { - try { - channel.sendResponse(e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.debug("failed to send response on cluster state processed", inner); - } - } - }); - } - - private class SendClusterStateRequestHandler implements TransportRequestHandler { - - @Override - public void messageReceived(BytesTransportRequest request, final TransportChannel channel, Task task) throws Exception { - handleIncomingClusterStateRequest(request, channel); - } - } - - private class CommitClusterStateRequestHandler implements TransportRequestHandler { - @Override - public void messageReceived(CommitClusterStateRequest request, final TransportChannel channel, Task task) throws Exception { - handleCommitRequest(request, channel); - } - } - - public static class CommitClusterStateRequest extends TransportRequest { - - public String stateUUID; - - public CommitClusterStateRequest() { - } - - public CommitClusterStateRequest(String stateUUID) { - this.stateUUID = stateUUID; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - stateUUID = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeString(stateUUID); - } - } - - - /** - * Coordinates acknowledgments of the sent cluster state from the different nodes. Commits the change - * after `minimum_master_nodes` have successfully responded or fails the entire change. After committing - * the cluster state, will trigger a commit message to all nodes that responded previously and responds immediately - * to all future acknowledgments. - */ - class SendingController { - - private final ClusterState clusterState; - - public BlockingClusterStatePublishResponseHandler getPublishResponseHandler() { - return publishResponseHandler; - } - - private final BlockingClusterStatePublishResponseHandler publishResponseHandler; - final ArrayList sendAckedBeforeCommit = new ArrayList<>(); - - // writes and reads of these are protected under synchronization - final CountDownLatch committedOrFailedLatch; // 0 count indicates that a decision was made w.r.t committing or failing - boolean committed; // true if cluster state was committed - int neededMastersToCommit; // number of master nodes acks still needed before committing - int pendingMasterNodes; // how many master node still need to respond - - // an external marker to note that the publishing process is timed out. This is useful for proper logging. - final AtomicBoolean publishingTimedOut = new AtomicBoolean(); - - private SendingController(ClusterState clusterState, int minMasterNodes, int totalMasterNodes, - BlockingClusterStatePublishResponseHandler publishResponseHandler) { - this.clusterState = clusterState; - this.publishResponseHandler = publishResponseHandler; - this.neededMastersToCommit = Math.max(0, minMasterNodes - 1); // we are one of the master nodes - this.pendingMasterNodes = totalMasterNodes - 1; - if (this.neededMastersToCommit > this.pendingMasterNodes) { - throw new FailedToCommitClusterStateException("not enough masters to ack sent cluster state." + - "[{}] needed , have [{}]", neededMastersToCommit, pendingMasterNodes); - } - this.committed = neededMastersToCommit == 0; - this.committedOrFailedLatch = new CountDownLatch(committed ? 0 : 1); - } - - public void waitForCommit(TimeValue commitTimeout) { - boolean timedout = false; - try { - timedout = committedOrFailedLatch.await(commitTimeout.millis(), TimeUnit.MILLISECONDS) == false; - } catch (InterruptedException e) { - // the commit check bellow will either translate to an exception or we are committed and we can safely continue - } - - if (timedout) { - markAsFailed("timed out waiting for commit (commit timeout [" + commitTimeout + "])"); - } - if (isCommitted() == false) { - throw new FailedToCommitClusterStateException("{} enough masters to ack sent cluster state. [{}] left", - timedout ? "timed out while waiting for" : "failed to get", neededMastersToCommit); - } - } - - public synchronized boolean isCommitted() { - return committed; - } - - public synchronized void onNodeSendAck(DiscoveryNode node) { - if (committed) { - assert sendAckedBeforeCommit.isEmpty(); - sendCommitToNode(node, clusterState, this); - } else if (committedOrFailed()) { - logger.trace("ignoring ack from [{}] for cluster state version [{}]. already failed", node, clusterState.version()); - } else { - // we're still waiting - sendAckedBeforeCommit.add(node); - if (node.isMasterNode()) { - checkForCommitOrFailIfNoPending(node); - } - } - } - - private synchronized boolean committedOrFailed() { - return committedOrFailedLatch.getCount() == 0; - } - - /** - * check if enough master node responded to commit the change. fails the commit - * if there are no more pending master nodes but not enough acks to commit. - */ - private synchronized void checkForCommitOrFailIfNoPending(DiscoveryNode masterNode) { - logger.trace("master node {} acked cluster state version [{}]. processing ... (current pending [{}], needed [{}])", - masterNode, clusterState.version(), pendingMasterNodes, neededMastersToCommit); - neededMastersToCommit--; - if (neededMastersToCommit == 0) { - if (markAsCommitted()) { - for (DiscoveryNode nodeToCommit : sendAckedBeforeCommit) { - sendCommitToNode(nodeToCommit, clusterState, this); - } - sendAckedBeforeCommit.clear(); - } - } - decrementPendingMasterAcksAndChangeForFailure(); - } - - private synchronized void decrementPendingMasterAcksAndChangeForFailure() { - pendingMasterNodes--; - if (pendingMasterNodes == 0 && neededMastersToCommit > 0) { - markAsFailed("no more pending master nodes, but failed to reach needed acks ([" + neededMastersToCommit + "] left)"); - } - } - - public synchronized void onNodeSendFailed(DiscoveryNode node, Exception e) { - if (node.isMasterNode()) { - logger.trace("master node {} failed to ack cluster state version [{}]. " + - "processing ... (current pending [{}], needed [{}])", - node, clusterState.version(), pendingMasterNodes, neededMastersToCommit); - decrementPendingMasterAcksAndChangeForFailure(); - } - publishResponseHandler.onFailure(node, e); - } - - /** - * tries and commit the current state, if a decision wasn't made yet - * - * @return true if successful - */ - private synchronized boolean markAsCommitted() { - if (committedOrFailed()) { - return committed; - } - logger.trace("committing version [{}]", clusterState.version()); - committed = true; - committedOrFailedLatch.countDown(); - return true; - } - - /** - * tries marking the publishing as failed, if a decision wasn't made yet - * - * @return true if the publishing was failed and the cluster state is *not* committed - **/ - private synchronized boolean markAsFailed(String details, Exception reason) { - if (committedOrFailed()) { - return committed == false; - } - logger.trace(() -> new ParameterizedMessage("failed to commit version [{}]. {}", - clusterState.version(), details), reason); - committed = false; - committedOrFailedLatch.countDown(); - return true; - } - - /** - * tries marking the publishing as failed, if a decision wasn't made yet - * - * @return true if the publishing was failed and the cluster state is *not* committed - **/ - private synchronized boolean markAsFailed(String reason) { - if (committedOrFailed()) { - return committed == false; - } - logger.trace("failed to commit version [{}]. {}", clusterState.version(), reason); - committed = false; - committedOrFailedLatch.countDown(); - return true; - } - - public boolean getPublishingTimedOut() { - return publishingTimedOut.get(); - } - - public void setPublishingTimedOut(boolean isTimedOut) { - publishingTimedOut.set(isTimedOut); - } - } - - public PublishClusterStateStats stats() { - return new PublishClusterStateStats( - fullClusterStateReceivedCount.get(), - incompatibleClusterStateDiffReceivedCount.get(), - compatibleClusterStateDiffReceivedCount.get()); - } -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java b/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java deleted file mode 100644 index 0cb10fa4e0f65..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java +++ /dev/null @@ -1,581 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import com.carrotsearch.hppc.cursors.ObjectCursor; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.CollectionUtils; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor; -import org.elasticsearch.common.util.concurrent.KeyedLock; -import org.elasticsearch.discovery.SeedHostsProvider; -import org.elasticsearch.discovery.SeedHostsResolver; -import org.elasticsearch.node.Node; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectTransportException; -import org.elasticsearch.transport.ConnectionProfile; -import org.elasticsearch.transport.NodeNotConnectedException; -import org.elasticsearch.transport.RemoteTransportException; -import org.elasticsearch.transport.Transport.Connection; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; - -public class UnicastZenPing implements ZenPing { - - private static final Logger logger = LogManager.getLogger(UnicastZenPing.class); - - public static final String ACTION_NAME = "internal:discovery/zen/unicast"; - - private final ThreadPool threadPool; - private final TransportService transportService; - private final ClusterName clusterName; - - private final PingContextProvider contextProvider; - - private final AtomicInteger pingingRoundIdGenerator = new AtomicInteger(); - - private final Map activePingingRounds = newConcurrentMap(); - - // a list of temporal responses a node will return for a request (holds responses from other nodes) - private final Queue temporalResponses = ConcurrentCollections.newQueue(); - - private final SeedHostsProvider hostsProvider; - - protected final EsThreadPoolExecutor unicastZenPingExecutorService; - - private final TimeValue resolveTimeout; - - private final String nodeName; - - private volatile boolean closed = false; - - public UnicastZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, - SeedHostsProvider seedHostsProvider, PingContextProvider contextProvider) { - this.threadPool = threadPool; - this.transportService = transportService; - this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); - this.hostsProvider = seedHostsProvider; - this.contextProvider = contextProvider; - - final int concurrentConnects = SeedHostsResolver.getMaxConcurrentResolvers(settings); - resolveTimeout = SeedHostsResolver.getResolveTimeout(settings); - nodeName = Node.NODE_NAME_SETTING.get(settings); - logger.debug( - "using max_concurrent_resolvers [{}], resolver timeout [{}]", - concurrentConnects, - resolveTimeout); - - transportService.registerRequestHandler(ACTION_NAME, ThreadPool.Names.SAME, UnicastPingRequest::new, - new UnicastPingRequestHandler()); - - final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings, "[unicast_connect]"); - unicastZenPingExecutorService = EsExecutors.newScaling( - nodeName + "/" + "unicast_connect", - 0, - concurrentConnects, - 60, - TimeUnit.SECONDS, - threadFactory, - threadPool.getThreadContext()); - } - - private SeedHostsProvider.HostsResolver createHostsResolver() { - return (hosts, limitPortCounts) -> SeedHostsResolver.resolveHostsLists(unicastZenPingExecutorService, logger, hosts, - limitPortCounts, transportService, resolveTimeout); - } - - @Override - public void close() { - ThreadPool.terminate(unicastZenPingExecutorService, 10, TimeUnit.SECONDS); - Releasables.close(activePingingRounds.values()); - closed = true; - } - - @Override - public void start() { - } - - /** - * Clears the list of cached ping responses. - */ - public void clearTemporalResponses() { - temporalResponses.clear(); - } - - /** - * Sends three rounds of pings notifying the specified {@link Consumer} when pinging is complete. Pings are sent after resolving - * configured unicast hosts to their IP address (subject to DNS caching within the JVM). A batch of pings is sent, then another batch - * of pings is sent at half the specified {@link TimeValue}, and then another batch of pings is sent at the specified {@link TimeValue}. - * The pings that are sent carry a timeout of 1.25 times the specified {@link TimeValue}. When pinging each node, a connection and - * handshake is performed, with a connection timeout of the specified {@link TimeValue}. - * - * @param resultsConsumer the callback when pinging is complete - * @param duration the timeout for various components of the pings - */ - @Override - public void ping(final Consumer resultsConsumer, final TimeValue duration) { - ping(resultsConsumer, duration, duration); - } - - /** - * a variant of {@link #ping(Consumer, TimeValue)}, but allows separating the scheduling duration - * from the duration used for request level time outs. This is useful for testing - */ - protected void ping(final Consumer resultsConsumer, - final TimeValue scheduleDuration, - final TimeValue requestDuration) { - final List seedAddresses = new ArrayList<>(); - seedAddresses.addAll(hostsProvider.getSeedAddresses(createHostsResolver())); - final DiscoveryNodes nodes = contextProvider.clusterState().nodes(); - // add all possible master nodes that were active in the last known cluster configuration - for (ObjectCursor masterNode : nodes.getMasterNodes().values()) { - seedAddresses.add(masterNode.value.getAddress()); - } - - final ConnectionProfile connectionProfile = - ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, requestDuration, requestDuration, - TimeValue.MINUS_ONE, null); - final PingingRound pingingRound = new PingingRound(pingingRoundIdGenerator.incrementAndGet(), seedAddresses, resultsConsumer, - nodes.getLocalNode(), connectionProfile); - activePingingRounds.put(pingingRound.id(), pingingRound); - final AbstractRunnable pingSender = new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - if (e instanceof AlreadyClosedException == false) { - logger.warn("unexpected error while pinging", e); - } - } - - @Override - protected void doRun() throws Exception { - sendPings(requestDuration, pingingRound); - } - }; - threadPool.generic().execute(pingSender); - threadPool.schedule(pingSender, TimeValue.timeValueMillis(scheduleDuration.millis() / 3), ThreadPool.Names.GENERIC); - threadPool.schedule(pingSender, TimeValue.timeValueMillis(scheduleDuration.millis() / 3 * 2), ThreadPool.Names.GENERIC); - threadPool.schedule(new AbstractRunnable() { - @Override - protected void doRun() throws Exception { - finishPingingRound(pingingRound); - } - - @Override - public void onFailure(Exception e) { - logger.warn("unexpected error while finishing pinging round", e); - } - }, scheduleDuration, ThreadPool.Names.GENERIC); - } - - // for testing - protected void finishPingingRound(PingingRound pingingRound) { - pingingRound.close(); - } - - protected class PingingRound implements Releasable { - private final int id; - private final Map tempConnections = new HashMap<>(); - private final KeyedLock connectionLock = new KeyedLock<>(true); - private final PingCollection pingCollection; - private final List seedAddresses; - private final Consumer pingListener; - private final DiscoveryNode localNode; - private final ConnectionProfile connectionProfile; - - private AtomicBoolean closed = new AtomicBoolean(false); - - PingingRound(int id, List seedAddresses, Consumer resultsConsumer, DiscoveryNode localNode, - ConnectionProfile connectionProfile) { - this.id = id; - this.seedAddresses = Collections.unmodifiableList(seedAddresses.stream().distinct().collect(Collectors.toList())); - this.pingListener = resultsConsumer; - this.localNode = localNode; - this.connectionProfile = connectionProfile; - this.pingCollection = new PingCollection(); - } - - public int id() { - return this.id; - } - - public boolean isClosed() { - return this.closed.get(); - } - - public List getSeedAddresses() { - ensureOpen(); - return seedAddresses; - } - - public Connection getOrConnect(DiscoveryNode node) throws IOException { - Connection result; - try (Releasable ignore = connectionLock.acquire(node.getAddress())) { - result = tempConnections.get(node.getAddress()); - if (result == null) { - ensureOpen(); - boolean success = false; - logger.trace("[{}] opening connection to [{}]", id(), node); - result = transportService.openConnection(node, connectionProfile); - try { - transportService.handshake(result, connectionProfile.getHandshakeTimeout().millis()); - synchronized (this) { - // acquire lock and check if closed, to prevent leaving an open connection after closing - ensureOpen(); - Connection existing = tempConnections.put(node.getAddress(), result); - assert existing == null; - success = true; - } - } finally { - if (success == false) { - logger.trace("[{}] closing connection to [{}] due to failure", id(), node); - IOUtils.closeWhileHandlingException(result); - } - } - } - } - return result; - } - - private void ensureOpen() { - if (isClosed()) { - throw new AlreadyClosedException("pinging round [" + id + "] is finished"); - } - } - - public void addPingResponseToCollection(PingResponse pingResponse) { - if (localNode.equals(pingResponse.node()) == false) { - pingCollection.addPing(pingResponse); - } - } - - @Override - public void close() { - List toClose = null; - synchronized (this) { - if (closed.compareAndSet(false, true)) { - activePingingRounds.remove(id); - toClose = new ArrayList<>(tempConnections.values()); - tempConnections.clear(); - } - } - if (toClose != null) { - // we actually closed - try { - pingListener.accept(pingCollection); - } finally { - IOUtils.closeWhileHandlingException(toClose); - } - } - } - - public ConnectionProfile getConnectionProfile() { - return connectionProfile; - } - } - - - protected void sendPings(final TimeValue timeout, final PingingRound pingingRound) { - final ClusterState lastState = contextProvider.clusterState(); - final UnicastPingRequest pingRequest = new UnicastPingRequest(pingingRound.id(), timeout, createPingResponse(lastState)); - - List temporalAddresses = temporalResponses.stream().map(pingResponse -> { - assert clusterName.equals(pingResponse.clusterName()) : - "got a ping request from a different cluster. expected " + clusterName + " got " + pingResponse.clusterName(); - return pingResponse.node().getAddress(); - }).collect(Collectors.toList()); - - final Stream uniqueAddresses = Stream.concat(pingingRound.getSeedAddresses().stream(), - temporalAddresses.stream()).distinct(); - - // resolve what we can via the latest cluster state - final Set nodesToPing = uniqueAddresses - .map(address -> { - DiscoveryNode foundNode = lastState.nodes().findByAddress(address); - if (foundNode != null && transportService.nodeConnected(foundNode)) { - return foundNode; - } else { - return new DiscoveryNode( - address.toString(), - address, - emptyMap(), - emptySet(), - Version.CURRENT.minimumCompatibilityVersion()); - } - }).collect(Collectors.toSet()); - - nodesToPing.forEach(node -> sendPingRequestToNode(node, timeout, pingingRound, pingRequest)); - } - - private void sendPingRequestToNode(final DiscoveryNode node, TimeValue timeout, final PingingRound pingingRound, - final UnicastPingRequest pingRequest) { - submitToExecutor(new AbstractRunnable() { - @Override - protected void doRun() throws Exception { - Connection connection = null; - if (transportService.nodeConnected(node)) { - try { - // concurrency can still cause disconnects - connection = transportService.getConnection(node); - } catch (NodeNotConnectedException e) { - logger.trace("[{}] node [{}] just disconnected, will create a temp connection", pingingRound.id(), node); - } - } - - if (connection == null) { - connection = pingingRound.getOrConnect(node); - } - - logger.trace("[{}] sending to {}", pingingRound.id(), node); - transportService.sendRequest(connection, ACTION_NAME, pingRequest, - TransportRequestOptions.builder().withTimeout((long) (timeout.millis() * 1.25)).build(), - getPingResponseHandler(pingingRound, node)); - } - - @Override - public void onFailure(Exception e) { - if (e instanceof ConnectTransportException || e instanceof AlreadyClosedException) { - // can't connect to the node - this is more common path! - logger.trace(() -> new ParameterizedMessage("[{}] failed to ping {}", pingingRound.id(), node), e); - } else if (e instanceof RemoteTransportException) { - // something went wrong on the other side - logger.debug(() -> new ParameterizedMessage( - "[{}] received a remote error as a response to ping {}", pingingRound.id(), node), e); - } else { - logger.warn(() -> new ParameterizedMessage("[{}] failed send ping to {}", pingingRound.id(), node), e); - } - } - - @Override - public void onRejection(Exception e) { - // The RejectedExecutionException can come from the fact unicastZenPingExecutorService is at its max down in sendPings - // But don't bail here, we can retry later on after the send ping has been scheduled. - logger.debug("Ping execution rejected", e); - } - }); - } - - // for testing - protected void submitToExecutor(AbstractRunnable abstractRunnable) { - unicastZenPingExecutorService.execute(abstractRunnable); - } - - // for testing - protected TransportResponseHandler getPingResponseHandler(final PingingRound pingingRound, - final DiscoveryNode node) { - return new TransportResponseHandler() { - - @Override - public UnicastPingResponse read(StreamInput in) throws IOException { - return new UnicastPingResponse(in); - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - - @Override - public void handleResponse(UnicastPingResponse response) { - logger.trace("[{}] received response from {}: {}", pingingRound.id(), node, Arrays.toString(response.pingResponses)); - if (pingingRound.isClosed()) { - if (logger.isTraceEnabled()) { - logger.trace("[{}] skipping received response from {}. already closed", pingingRound.id(), node); - } - } else { - Stream.of(response.pingResponses).forEach(pingingRound::addPingResponseToCollection); - } - } - - @Override - public void handleException(TransportException exp) { - if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException || - exp.getCause() instanceof AlreadyClosedException) { - // ok, not connected... - logger.trace(() -> new ParameterizedMessage("failed to connect to {}", node), exp); - } else if (closed == false) { - logger.warn(() -> new ParameterizedMessage("failed to send ping to [{}]", node), exp); - } - } - }; - } - - private UnicastPingResponse handlePingRequest(final UnicastPingRequest request) { - assert clusterName.equals(request.pingResponse.clusterName()) : - "got a ping request from a different cluster. expected " + clusterName + " got " + request.pingResponse.clusterName(); - temporalResponses.add(request.pingResponse); - // add to any ongoing pinging - activePingingRounds.values().forEach(p -> p.addPingResponseToCollection(request.pingResponse)); - threadPool.schedule(() -> temporalResponses.remove(request.pingResponse), - TimeValue.timeValueMillis(request.timeout.millis() * 2), ThreadPool.Names.SAME); - - List pingResponses = CollectionUtils.iterableAsArrayList(temporalResponses); - pingResponses.add(createPingResponse(contextProvider.clusterState())); - - return new UnicastPingResponse(request.id, pingResponses.toArray(new PingResponse[pingResponses.size()])); - } - - class UnicastPingRequestHandler implements TransportRequestHandler { - - @Override - public void messageReceived(UnicastPingRequest request, TransportChannel channel, Task task) throws Exception { - if (closed) { - throw new AlreadyClosedException("node is shutting down"); - } - if (request.pingResponse.clusterName().equals(clusterName)) { - channel.sendResponse(handlePingRequest(request)); - } else { - throw new IllegalStateException( - String.format( - Locale.ROOT, - "mismatched cluster names; request: [%s], local: [%s]", - request.pingResponse.clusterName().value(), - clusterName.value())); - } - } - - } - - public static class UnicastPingRequest extends TransportRequest { - - public final int id; - public final TimeValue timeout; - public final PingResponse pingResponse; - - public UnicastPingRequest(int id, TimeValue timeout, PingResponse pingResponse) { - this.id = id; - this.timeout = timeout; - this.pingResponse = pingResponse; - } - - public UnicastPingRequest(StreamInput in) throws IOException { - super(in); - id = in.readInt(); - timeout = in.readTimeValue(); - pingResponse = new PingResponse(in); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeInt(id); - out.writeTimeValue(timeout); - pingResponse.writeTo(out); - } - } - - private PingResponse createPingResponse(ClusterState clusterState) { - DiscoveryNodes discoNodes = clusterState.nodes(); - return new PingResponse(discoNodes.getLocalNode(), discoNodes.getMasterNode(), clusterState); - } - - public static class UnicastPingResponse extends TransportResponse { - - final int id; - - public final PingResponse[] pingResponses; - - public UnicastPingResponse(int id, PingResponse[] pingResponses) { - this.id = id; - this.pingResponses = pingResponses; - } - - public UnicastPingResponse(StreamInput in) throws IOException { - id = in.readInt(); - pingResponses = new PingResponse[in.readVInt()]; - for (int i = 0; i < pingResponses.length; i++) { - pingResponses[i] = new PingResponse(in); - } - } - - @Override - public void readFrom(StreamInput in) throws IOException { - throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeInt(id); - out.writeVInt(pingResponses.length); - for (PingResponse pingResponse : pingResponses) { - pingResponse.writeTo(out); - } - } - } - - protected Version getVersion() { - return Version.CURRENT; // for tests - } - -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java deleted file mode 100644 index 716a0b97b6d5d..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ /dev/null @@ -1,1251 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.NotMasterException; -import org.elasticsearch.cluster.block.ClusterBlocks; -import org.elasticsearch.cluster.coordination.JoinTaskExecutor; -import org.elasticsearch.cluster.coordination.NoMasterBlockService; -import org.elasticsearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterApplier; -import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; -import org.elasticsearch.cluster.service.ClusterApplierService; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.component.Lifecycle; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.logging.LoggerMessageFormat; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.discovery.DiscoveryStats; -import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; -import org.elasticsearch.discovery.SeedHostsProvider; -import org.elasticsearch.discovery.zen.PublishClusterStateAction.IncomingClusterStateListener; -import org.elasticsearch.gateway.GatewayMetaState; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.EmptyTransportResponseHandler; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestHandler; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportService; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Locale; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; -import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; - -public class ZenDiscovery extends AbstractLifecycleComponent implements Discovery, PingContextProvider, IncomingClusterStateListener { - private static final Logger logger = LogManager.getLogger(ZenDiscovery.class); - - public static final Setting PING_TIMEOUT_SETTING = - Setting.positiveTimeSetting("discovery.zen.ping_timeout", timeValueSeconds(3), Property.NodeScope); - public static final Setting JOIN_TIMEOUT_SETTING = - Setting.timeSetting("discovery.zen.join_timeout", - settings -> TimeValue.timeValueMillis(PING_TIMEOUT_SETTING.get(settings).millis() * 20), - TimeValue.timeValueMillis(0), Property.NodeScope, Property.Deprecated); - public static final Setting JOIN_RETRY_ATTEMPTS_SETTING = - Setting.intSetting("discovery.zen.join_retry_attempts", 3, 1, Property.NodeScope, Property.Deprecated); - public static final Setting JOIN_RETRY_DELAY_SETTING = - Setting.positiveTimeSetting("discovery.zen.join_retry_delay", TimeValue.timeValueMillis(100), - Property.NodeScope, Property.Deprecated); - public static final Setting MAX_PINGS_FROM_ANOTHER_MASTER_SETTING = - Setting.intSetting("discovery.zen.max_pings_from_another_master", 3, 1, Property.NodeScope, Property.Deprecated); - public static final Setting SEND_LEAVE_REQUEST_SETTING = - Setting.boolSetting("discovery.zen.send_leave_request", true, Property.NodeScope, Property.Deprecated); - public static final Setting MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING = - Setting.timeSetting("discovery.zen.master_election.wait_for_joins_timeout", - settings -> TimeValue.timeValueMillis(JOIN_TIMEOUT_SETTING.get(settings).millis() / 2), TimeValue.timeValueMillis(0), - Property.NodeScope, Property.Deprecated); - public static final Setting MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING = - Setting.boolSetting("discovery.zen.master_election.ignore_non_master_pings", false, Property.NodeScope, Property.Deprecated); - public static final Setting MAX_PENDING_CLUSTER_STATES_SETTING = - Setting.intSetting("discovery.zen.publish.max_pending_cluster_states", 25, 1, Property.NodeScope, Property.Deprecated); - - public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin"; - - private final TransportService transportService; - private final MasterService masterService; - private final DiscoverySettings discoverySettings; - private final NoMasterBlockService noMasterBlockService; - protected final ZenPing zenPing; // protected to allow tests access - private final MasterFaultDetection masterFD; - private final NodesFaultDetection nodesFD; - private final PublishClusterStateAction publishClusterState; - private final MembershipAction membership; - private final ClusterName clusterName; - private final ThreadPool threadPool; - - private final TimeValue pingTimeout; - private final TimeValue joinTimeout; - - /** how many retry attempts to perform if join request failed with an retryable error */ - private final int joinRetryAttempts; - /** how long to wait before performing another join attempt after a join request failed with an retryable error */ - private final TimeValue joinRetryDelay; - - /** how many pings from *another* master to tolerate before forcing a rejoin on other or local master */ - private final int maxPingsFromAnotherMaster; - - // a flag that should be used only for testing - private final boolean sendLeaveRequest; - - private final ElectMasterService electMaster; - - private final boolean masterElectionIgnoreNonMasters; - private final TimeValue masterElectionWaitForJoinsTimeout; - - private final JoinThreadControl joinThreadControl; - - private final PendingClusterStatesQueue pendingStatesQueue; - - private final NodeJoinController nodeJoinController; - private final NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor; - private final ClusterApplier clusterApplier; - private final AtomicReference committedState; // last committed cluster state - private final Object stateMutex = new Object(); - private final Collection> onJoinValidators; - - public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, ClusterApplier clusterApplier, - ClusterSettings clusterSettings, SeedHostsProvider hostsProvider, AllocationService allocationService, - Collection> onJoinValidators, GatewayMetaState gatewayMetaState) { - this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators); - this.masterService = masterService; - this.clusterApplier = clusterApplier; - this.transportService = transportService; - this.discoverySettings = new DiscoverySettings(settings, clusterSettings); - this.noMasterBlockService = new NoMasterBlockService(settings, clusterSettings); - this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider); - this.electMaster = new ElectMasterService(settings); - this.pingTimeout = PING_TIMEOUT_SETTING.get(settings); - this.joinTimeout = JOIN_TIMEOUT_SETTING.get(settings); - this.joinRetryAttempts = JOIN_RETRY_ATTEMPTS_SETTING.get(settings); - this.joinRetryDelay = JOIN_RETRY_DELAY_SETTING.get(settings); - this.maxPingsFromAnotherMaster = MAX_PINGS_FROM_ANOTHER_MASTER_SETTING.get(settings); - this.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings); - this.threadPool = threadPool; - this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); - this.committedState = new AtomicReference<>(); - - this.masterElectionIgnoreNonMasters = MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING.get(settings); - this.masterElectionWaitForJoinsTimeout = MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.get(settings); - - logger.debug("using ping_timeout [{}], join.timeout [{}], master_election.ignore_non_master [{}]", - this.pingTimeout, joinTimeout, masterElectionIgnoreNonMasters); - - clusterSettings.addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, - this::handleMinimumMasterNodesChanged, (value) -> { - final ClusterState clusterState = this.clusterState(); - int masterNodes = clusterState.nodes().getMasterNodes().size(); - // the purpose of this validation is to make sure that the master doesn't step down - // due to a change in master nodes, which also means that there is no way to revert - // an accidental change. Since we validate using the current cluster state (and - // not the one from which the settings come from) we have to be careful and only - // validate if the local node is already a master. Doing so all the time causes - // subtle issues. For example, a node that joins a cluster has no nodes in its - // current cluster state. When it receives a cluster state from the master with - // a dynamic minimum master nodes setting int it, we must make sure we don't reject - // it. - - if (clusterState.nodes().isLocalNodeElectedMaster() && value > masterNodes) { - throw new IllegalArgumentException("cannot set " - + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " to more than the current" + - " master nodes count [" + masterNodes + "]"); - } - }); - - this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, this::clusterState, masterService, clusterName); - this.masterFD.addListener(new MasterNodeFailureListener()); - this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, this::clusterState, clusterName); - this.nodesFD.addListener(new NodeFaultDetectionListener()); - this.pendingStatesQueue = new PendingClusterStatesQueue(logger, MAX_PENDING_CLUSTER_STATES_SETTING.get(settings)); - - this.publishClusterState = - new PublishClusterStateAction( - transportService, - namedWriteableRegistry, - this, - discoverySettings); - this.membership = new MembershipAction(transportService, new MembershipListener(), onJoinValidators); - this.joinThreadControl = new JoinThreadControl(); - - this.nodeJoinController = new NodeJoinController(settings, masterService, allocationService, electMaster); - this.nodeRemovalExecutor = new ZenNodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::submitRejoin, logger); - - masterService.setClusterStateSupplier(this::clusterState); - - transportService.registerRequestHandler( - DISCOVERY_REJOIN_ACTION_NAME, RejoinClusterRequest::new, ThreadPool.Names.SAME, new RejoinClusterRequestHandler()); - - if (clusterApplier instanceof ClusterApplierService) { - ((ClusterApplierService) clusterApplier).addLowPriorityApplier(gatewayMetaState); - } - } - - // protected to allow overriding in tests - protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, - SeedHostsProvider hostsProvider) { - return new UnicastZenPing(settings, threadPool, transportService, hostsProvider, this); - } - - @Override - protected void doStart() { - DiscoveryNode localNode = transportService.getLocalNode(); - assert localNode != null; - synchronized (stateMutex) { - // set initial state - assert committedState.get() == null; - assert localNode != null; - ClusterState.Builder builder = ClusterState.builder(clusterName); - ClusterState initialState = builder - .blocks(ClusterBlocks.builder() - .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK) - .addGlobalBlock(noMasterBlockService.getNoMasterBlock())) - .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())) - .build(); - committedState.set(initialState); - clusterApplier.setInitialState(initialState); - nodesFD.setLocalNode(localNode); - joinThreadControl.start(); - } - zenPing.start(); - } - - @Override - public void startInitialJoin() { - // start the join thread from a cluster state update. See {@link JoinThreadControl} for details. - synchronized (stateMutex) { - // do the join on a different thread, the caller of this method waits for 30s anyhow till it is discovered - joinThreadControl.startNewThreadIfNotRunning(); - } - } - - @Override - protected void doStop() { - joinThreadControl.stop(); - masterFD.stop("zen disco stop"); - nodesFD.stop(); - Releasables.close(zenPing); // stop any ongoing pinging - DiscoveryNodes nodes = clusterState().nodes(); - if (sendLeaveRequest) { - if (nodes.getMasterNode() == null) { - // if we don't know who the master is, nothing to do here - } else if (!nodes.isLocalNodeElectedMaster()) { - try { - membership.sendLeaveRequestBlocking(nodes.getMasterNode(), nodes.getLocalNode(), TimeValue.timeValueSeconds(1)); - } catch (Exception e) { - logger.debug(() -> new ParameterizedMessage("failed to send leave request to master [{}]", nodes.getMasterNode()), e); - } - } else { - // we're master -> let other potential master we left and start a master election now rather then wait for masterFD - DiscoveryNode[] possibleMasters = electMaster.nextPossibleMasters(nodes.getNodes().values(), 5); - for (DiscoveryNode possibleMaster : possibleMasters) { - if (nodes.getLocalNode().equals(possibleMaster)) { - continue; - } - try { - membership.sendLeaveRequest(nodes.getLocalNode(), possibleMaster); - } catch (Exception e) { - logger.debug(() -> new ParameterizedMessage("failed to send leave request from master [{}] to possible master [{}]", - nodes.getMasterNode(), possibleMaster), e); - } - } - } - } - } - - @Override - protected void doClose() throws IOException { - IOUtils.close(masterFD, nodesFD); - } - - @Override - public ClusterState clusterState() { - ClusterState clusterState = committedState.get(); - assert clusterState != null : "accessing cluster state before it is set"; - return clusterState; - } - - @Override - public void publish(ClusterChangedEvent clusterChangedEvent, ActionListener publishListener, AckListener ackListener) { - ClusterState newState = clusterChangedEvent.state(); - assert newState.getNodes().isLocalNodeElectedMaster() : "Shouldn't publish state when not master " + clusterChangedEvent.source(); - - try { - - // state got changed locally (maybe because another master published to us) - if (clusterChangedEvent.previousState() != this.committedState.get()) { - throw new FailedToCommitClusterStateException("state was mutated while calculating new CS update"); - } - - pendingStatesQueue.addPending(newState); - - publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener); - } catch (FailedToCommitClusterStateException t) { - // cluster service logs a WARN message - logger.debug("failed to publish cluster state version [{}] (not enough nodes acknowledged, min master nodes [{}])", - newState.version(), electMaster.minimumMasterNodes()); - - synchronized (stateMutex) { - pendingStatesQueue.failAllStatesAndClear( - new ElasticsearchException("failed to publish cluster state")); - - rejoin("zen-disco-failed-to-publish"); - } - - publishListener.onFailure(t); - return; - } - - final DiscoveryNode localNode = newState.getNodes().getLocalNode(); - final AtomicBoolean processedOrFailed = new AtomicBoolean(); - pendingStatesQueue.markAsCommitted(newState.stateUUID(), - new PendingClusterStatesQueue.StateProcessedListener() { - @Override - public void onNewClusterStateProcessed() { - processedOrFailed.set(true); - publishListener.onResponse(null); - ackListener.onNodeAck(localNode, null); - } - - @Override - public void onNewClusterStateFailed(Exception e) { - processedOrFailed.set(true); - publishListener.onFailure(e); - ackListener.onNodeAck(localNode, e); - logger.warn(() -> new ParameterizedMessage( - "failed while applying cluster state locally [{}]", clusterChangedEvent.source()), e); - } - }); - - synchronized (stateMutex) { - if (clusterChangedEvent.previousState() != this.committedState.get()) { - publishListener.onFailure( - new FailedToCommitClusterStateException("local state was mutated while CS update was published to other nodes") - ); - return; - } - - boolean sentToApplier = processNextCommittedClusterState("master " + newState.nodes().getMasterNode() + - " committed version [" + newState.version() + "] source [" + clusterChangedEvent.source() + "]"); - if (sentToApplier == false && processedOrFailed.get() == false) { - assert false : "cluster state published locally neither processed nor failed: " + newState; - logger.warn("cluster state with version [{}] that is published locally has neither been processed nor failed", - newState.version()); - publishListener.onFailure(new FailedToCommitClusterStateException("cluster state that is published locally has neither " + - "been processed nor failed")); - } - } - } - - /** - * Gets the current set of nodes involved in the node fault detection. - * NB: for testing purposes - */ - Set getFaultDetectionNodes() { - return nodesFD.getNodes(); - } - - @Override - public DiscoveryStats stats() { - return new DiscoveryStats(pendingStatesQueue.stats(), publishClusterState.stats()); - } - - public DiscoverySettings getDiscoverySettings() { - return discoverySettings; - } - - /** - * returns true if zen discovery is started and there is a currently a background thread active for (re)joining - * the cluster used for testing. - */ - public boolean joiningCluster() { - return joinThreadControl.joinThreadActive(); - } - - // used for testing - public ClusterState[] pendingClusterStates() { - return pendingStatesQueue.pendingClusterStates(); - } - - PendingClusterStatesQueue pendingClusterStatesQueue() { - return pendingStatesQueue; - } - - /** - * the main function of a join thread. This function is guaranteed to join the cluster - * or spawn a new join thread upon failure to do so. - */ - private void innerJoinCluster() { - DiscoveryNode masterNode = null; - final Thread currentThread = Thread.currentThread(); - nodeJoinController.startElectionContext(); - while (masterNode == null && joinThreadControl.joinThreadActive(currentThread)) { - masterNode = findMaster(); - } - - if (!joinThreadControl.joinThreadActive(currentThread)) { - logger.trace("thread is no longer in currentJoinThread. Stopping."); - return; - } - - if (transportService.getLocalNode().equals(masterNode)) { - final int requiredJoins = Math.max(0, electMaster.minimumMasterNodes() - 1); // we count as one - logger.debug("elected as master, waiting for incoming joins ([{}] needed)", requiredJoins); - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, masterElectionWaitForJoinsTimeout, - new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - synchronized (stateMutex) { - joinThreadControl.markThreadAsDone(currentThread); - } - } - - @Override - public void onFailure(Throwable t) { - logger.trace("failed while waiting for nodes to join, rejoining", t); - synchronized (stateMutex) { - joinThreadControl.markThreadAsDoneAndStartNew(currentThread); - } - } - } - - ); - } else { - // process any incoming joins (they will fail because we are not the master) - nodeJoinController.stopElectionContext(masterNode + " elected"); - - // send join request - final boolean success = joinElectedMaster(masterNode); - - synchronized (stateMutex) { - if (success) { - DiscoveryNode currentMasterNode = this.clusterState().getNodes().getMasterNode(); - if (currentMasterNode == null) { - // Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have - // a valid master. - logger.debug("no master node is set, despite of join request completing. retrying pings."); - joinThreadControl.markThreadAsDoneAndStartNew(currentThread); - } else if (currentMasterNode.equals(masterNode) == false) { - // update cluster state - joinThreadControl.stopRunningThreadAndRejoin("master_switched_while_finalizing_join"); - } - - joinThreadControl.markThreadAsDone(currentThread); - } else { - // failed to join. Try again... - joinThreadControl.markThreadAsDoneAndStartNew(currentThread); - } - } - } - } - - /** - * Join a newly elected master. - * - * @return true if successful - */ - private boolean joinElectedMaster(DiscoveryNode masterNode) { - try { - // first, make sure we can connect to the master - transportService.connectToNode(masterNode); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to connect to master [{}], retrying...", masterNode), e); - return false; - } - int joinAttempt = 0; // we retry on illegal state if the master is not yet ready - while (true) { - try { - logger.trace("joining master {}", masterNode); - membership.sendJoinRequestBlocking(masterNode, transportService.getLocalNode(), joinTimeout); - return true; - } catch (Exception e) { - final Throwable unwrap = ExceptionsHelper.unwrapCause(e); - if (unwrap instanceof NotMasterException) { - if (++joinAttempt == this.joinRetryAttempts) { - logger.info("failed to send join request to master [{}], reason [{}], tried [{}] times", masterNode, - ExceptionsHelper.detailedMessage(e), joinAttempt); - return false; - } else { - logger.trace("master {} failed with [{}]. retrying... (attempts done: [{}])", masterNode, - ExceptionsHelper.detailedMessage(e), joinAttempt); - } - } else { - if (logger.isTraceEnabled()) { - logger.trace(() -> new ParameterizedMessage("failed to send join request to master [{}]", masterNode), e); - } else { - logger.info("failed to send join request to master [{}], reason [{}]", masterNode, - ExceptionsHelper.detailedMessage(e)); - } - return false; - } - } - - try { - Thread.sleep(this.joinRetryDelay.millis()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - - private void submitRejoin(String source) { - synchronized (stateMutex) { - rejoin(source); - } - } - - // visible for testing - void setCommittedState(ClusterState clusterState) { - synchronized (stateMutex) { - committedState.set(clusterState); - } - } - - private void removeNode(final DiscoveryNode node, final String source, final String reason) { - masterService.submitStateUpdateTask( - source + "(" + node + "), reason(" + reason + ")", - new NodeRemovalClusterStateTaskExecutor.Task(node, reason), - ClusterStateTaskConfig.build(Priority.IMMEDIATE), - nodeRemovalExecutor, - nodeRemovalExecutor); - } - - private void handleLeaveRequest(final DiscoveryNode node) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a node failure - return; - } - if (localNodeMaster()) { - removeNode(node, "zen-disco-node-left", "left"); - } else if (node.equals(clusterState().nodes().getMasterNode())) { - handleMasterGone(node, null, "shut_down"); - } - } - - private void handleNodeFailure(final DiscoveryNode node, final String reason) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a node failure - return; - } - if (!localNodeMaster()) { - // nothing to do here... - return; - } - removeNode(node, "zen-disco-node-failed", reason); - } - - private void handleMinimumMasterNodesChanged(final int minimumMasterNodes) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a node failure - return; - } - final int prevMinimumMasterNode = ZenDiscovery.this.electMaster.minimumMasterNodes(); - ZenDiscovery.this.electMaster.minimumMasterNodes(minimumMasterNodes); - if (!localNodeMaster()) { - // We only set the new value. If the master doesn't see enough nodes it will revoke it's mastership. - return; - } - synchronized (stateMutex) { - // check if we have enough master nodes, if not, we need to move into joining the cluster again - if (!electMaster.hasEnoughMasterNodes(committedState.get().nodes())) { - rejoin("not enough master nodes on change of minimum_master_nodes from [" + prevMinimumMasterNode + "] to [" + - minimumMasterNodes + "]"); - } - } - } - - private void handleMasterGone(final DiscoveryNode masterNode, final Throwable cause, final String reason) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a master failure - return; - } - if (localNodeMaster()) { - // we might get this on both a master telling us shutting down, and then the disconnect failure - return; - } - - logger.info(() -> new ParameterizedMessage("master_left [{}], reason [{}]", masterNode, reason), cause); - - synchronized (stateMutex) { - if (localNodeMaster() == false && masterNode.equals(committedState.get().nodes().getMasterNode())) { - // flush any pending cluster states from old master, so it will not be set as master again - pendingStatesQueue.failAllStatesAndClear(new ElasticsearchException("master left [{}]", reason)); - rejoin("master left (reason = " + reason + ")"); - } - } - } - - // return true if state has been sent to applier - boolean processNextCommittedClusterState(String reason) { - assert Thread.holdsLock(stateMutex); - - final ClusterState newClusterState = pendingStatesQueue.getNextClusterStateToProcess(); - final ClusterState currentState = committedState.get(); - // all pending states have been processed - if (newClusterState == null) { - return false; - } - - assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master"; - assert !newClusterState.blocks().hasGlobalBlock(noMasterBlockService.getNoMasterBlock()) : - "received a cluster state with a master block"; - - if (currentState.nodes().isLocalNodeElectedMaster() && newClusterState.nodes().isLocalNodeElectedMaster() == false) { - handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), - "via a new cluster state"); - return false; - } - - try { - if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) { - String message = String.format( - Locale.ROOT, - "rejecting cluster state version [%d] uuid [%s] received from [%s]", - newClusterState.version(), - newClusterState.stateUUID(), - newClusterState.nodes().getMasterNodeId() - ); - throw new IllegalStateException(message); - } - } catch (Exception e) { - try { - pendingStatesQueue.markAsFailed(newClusterState, e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.error(() -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner); - } - return false; - } - - if (currentState.blocks().hasGlobalBlock(noMasterBlockService.getNoMasterBlock())) { - // its a fresh update from the master as we transition from a start of not having a master to having one - logger.debug("got first state from fresh master [{}]", newClusterState.nodes().getMasterNodeId()); - } - - if (currentState == newClusterState) { - return false; - } - - committedState.set(newClusterState); - - // update failure detection only after the state has been updated to prevent race condition with handleLeaveRequest - // and handleNodeFailure as those check the current state to determine whether the failure is to be handled by this node - if (newClusterState.nodes().isLocalNodeElectedMaster()) { - // update the set of nodes to ping - nodesFD.updateNodesAndPing(newClusterState); - } else { - // check to see that we monitor the correct master of the cluster - if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().getMasterNode())) { - masterFD.restart(newClusterState.nodes().getMasterNode(), - "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]"); - } - } - - clusterApplier.onNewClusterState("apply cluster state (from master [" + reason + "])", - this::clusterState, - new ClusterApplyListener() { - @Override - public void onSuccess(String source) { - try { - pendingStatesQueue.markAsProcessed(newClusterState); - } catch (Exception e) { - onFailure(source, e); - } - } - - @Override - public void onFailure(String source, Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected failure applying [{}]", reason), e); - try { - // TODO: use cluster state uuid instead of full cluster state so that we don't keep reference to CS around - // for too long. - pendingStatesQueue.markAsFailed(newClusterState, e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.error(() -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner); - } - } - }); - - return true; - } - - /** - * In the case we follow an elected master the new cluster state needs to have the same elected master and - * the new cluster state version needs to be equal or higher than our cluster state version. - * If the first condition fails we reject the cluster state and throw an error. - * If the second condition fails we ignore the cluster state. - */ - public static boolean shouldIgnoreOrRejectNewClusterState(Logger logger, ClusterState currentState, ClusterState newClusterState) { - validateStateIsFromCurrentMaster(logger, currentState.nodes(), newClusterState); - - // reject cluster states that are not new from the same master - if (currentState.supersedes(newClusterState) || - (newClusterState.nodes().getMasterNodeId().equals(currentState.nodes().getMasterNodeId()) && - currentState.version() == newClusterState.version())) { - // if the new state has a smaller version, and it has the same master node, then no need to process it - logger.debug("received a cluster state that is not newer than the current one, ignoring (received {}, current {})", - newClusterState.version(), currentState.version()); - return true; - } - - // reject older cluster states if we are following a master - if (currentState.nodes().getMasterNodeId() != null && newClusterState.version() < currentState.version()) { - logger.debug("received a cluster state that has a lower version than the current one, ignoring (received {}, current {})", - newClusterState.version(), currentState.version()); - return true; - } - return false; - } - - /** - * In the case we follow an elected master the new cluster state needs to have the same elected master - * This method checks for this and throws an exception if needed - */ - - public static void validateStateIsFromCurrentMaster(Logger logger, DiscoveryNodes currentNodes, ClusterState newClusterState) { - if (currentNodes.getMasterNodeId() == null) { - return; - } - if (!currentNodes.getMasterNodeId().equals(newClusterState.nodes().getMasterNodeId())) { - logger.warn("received a cluster state from a different master than the current one, rejecting (received {}, current {})", - newClusterState.nodes().getMasterNode(), currentNodes.getMasterNode()); - throw new IllegalStateException("cluster state from a different master than the current one, rejecting (received " + - newClusterState.nodes().getMasterNode() + ", current " + currentNodes.getMasterNode() + ")"); - } - } - - void handleJoinRequest(final DiscoveryNode node, final ClusterState state, final MembershipAction.JoinCallback callback) { - if (nodeJoinController == null) { - throw new IllegalStateException("discovery module is not yet started"); - } else { - // we do this in a couple of places including the cluster update thread. This one here is really just best effort - // to ensure we fail as fast as possible. - onJoinValidators.stream().forEach(a -> a.accept(node, state)); - if (state.getBlocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK) == false) { - JoinTaskExecutor.ensureMajorVersionBarrier(node.getVersion(), state.getNodes().getMinNodeVersion()); - } - // try and connect to the node, if it fails, we can raise an exception back to the client... - transportService.connectToNode(node); - - // validate the join request, will throw a failure if it fails, which will get back to the - // node calling the join request - try { - membership.sendValidateJoinRequestBlocking(node, state, joinTimeout); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to validate incoming join request from node [{}]", node), - e); - callback.onFailure(new IllegalStateException("failure when sending a validation request to node", e)); - return; - } - nodeJoinController.handleJoinRequest(node, callback); - } - } - - private DiscoveryNode findMaster() { - logger.trace("starting to ping"); - List fullPingResponses = pingAndWait(pingTimeout).toList(); - if (fullPingResponses == null) { - logger.trace("No full ping responses"); - return null; - } - if (logger.isTraceEnabled()) { - StringBuilder sb = new StringBuilder(); - if (fullPingResponses.size() == 0) { - sb.append(" {none}"); - } else { - for (ZenPing.PingResponse pingResponse : fullPingResponses) { - sb.append("\n\t--> ").append(pingResponse); - } - } - logger.trace("full ping responses:{}", sb); - } - - final DiscoveryNode localNode = transportService.getLocalNode(); - - // add our selves - assert fullPingResponses.stream().map(ZenPing.PingResponse::node) - .filter(n -> n.equals(localNode)).findAny().isPresent() == false; - - fullPingResponses.add(new ZenPing.PingResponse(localNode, null, this.clusterState())); - - // filter responses - final List pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger); - - List activeMasters = new ArrayList<>(); - for (ZenPing.PingResponse pingResponse : pingResponses) { - // We can't include the local node in pingMasters list, otherwise we may up electing ourselves without - // any check / verifications from other nodes in ZenDiscover#innerJoinCluster() - if (pingResponse.master() != null && !localNode.equals(pingResponse.master())) { - activeMasters.add(pingResponse.master()); - } - } - - // nodes discovered during pinging - List masterCandidates = new ArrayList<>(); - for (ZenPing.PingResponse pingResponse : pingResponses) { - if (pingResponse.node().isMasterNode()) { - masterCandidates.add(new ElectMasterService.MasterCandidate(pingResponse.node(), pingResponse.getClusterStateVersion())); - } - } - - if (activeMasters.isEmpty()) { - if (electMaster.hasEnoughCandidates(masterCandidates)) { - final ElectMasterService.MasterCandidate winner = electMaster.electMaster(masterCandidates); - logger.trace("candidate {} won election", winner); - return winner.getNode(); - } else { - // if we don't have enough master nodes, we bail, because there are not enough master to elect from - logger.warn("not enough master nodes discovered during pinging (found [{}], but needed [{}]), pinging again", - masterCandidates, electMaster.minimumMasterNodes()); - return null; - } - } else { - assert !activeMasters.contains(localNode) : - "local node should never be elected as master when other nodes indicate an active master"; - // lets tie break between discovered nodes - return electMaster.tieBreakActiveMasters(activeMasters); - } - } - - static List filterPingResponses(List fullPingResponses, - boolean masterElectionIgnoreNonMasters, Logger logger) { - List pingResponses; - if (masterElectionIgnoreNonMasters) { - pingResponses = fullPingResponses.stream().filter(ping -> ping.node().isMasterNode()).collect(Collectors.toList()); - } else { - pingResponses = fullPingResponses; - } - - if (logger.isDebugEnabled()) { - StringBuilder sb = new StringBuilder(); - if (pingResponses.isEmpty()) { - sb.append(" {none}"); - } else { - for (ZenPing.PingResponse pingResponse : pingResponses) { - sb.append("\n\t--> ").append(pingResponse); - } - } - logger.debug("filtered ping responses: (ignore_non_masters [{}]){}", masterElectionIgnoreNonMasters, sb); - } - return pingResponses; - } - - protected void rejoin(String reason) { - assert Thread.holdsLock(stateMutex); - ClusterState clusterState = committedState.get(); - - logger.warn("{}, current nodes: {}", reason, clusterState.nodes()); - nodesFD.stop(); - masterFD.stop(reason); - - // TODO: do we want to force a new thread if we actively removed the master? this is to give a full pinging cycle - // before a decision is made. - joinThreadControl.startNewThreadIfNotRunning(); - - if (clusterState.nodes().getMasterNodeId() != null) { - // remove block if it already exists before adding new one - assert clusterState.blocks().hasGlobalBlockWithId(noMasterBlockService.getNoMasterBlock().id()) == false : - "NO_MASTER_BLOCK should only be added by ZenDiscovery"; - ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(clusterState.blocks()) - .addGlobalBlock(noMasterBlockService.getNoMasterBlock()) - .build(); - - DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder(clusterState.nodes()).masterNodeId(null).build(); - clusterState = ClusterState.builder(clusterState) - .blocks(clusterBlocks) - .nodes(discoveryNodes) - .build(); - - committedState.set(clusterState); - clusterApplier.onNewClusterState(reason, this::clusterState, (source, e) -> {}); // don't wait for state to be applied - } - } - - private boolean localNodeMaster() { - return clusterState().nodes().isLocalNodeElectedMaster(); - } - - private void handleAnotherMaster(ClusterState localClusterState, final DiscoveryNode otherMaster, long otherClusterStateVersion, - String reason) { - assert localClusterState.nodes().isLocalNodeElectedMaster() : "handleAnotherMaster called but current node is not a master"; - assert Thread.holdsLock(stateMutex); - - if (otherClusterStateVersion > localClusterState.version()) { - rejoin("zen-disco-discovered another master with a new cluster_state [" + otherMaster + "][" + reason + "]"); - } else { - // TODO: do this outside mutex - logger.warn("discovered [{}] which is also master but with an older cluster_state, telling [{}] to rejoin the cluster ([{}])", - otherMaster, otherMaster, reason); - try { - // make sure we're connected to this node (connect to node does nothing if we're already connected) - // since the network connections are asymmetric, it may be that we received a state but have disconnected from the node - // in the past (after a master failure, for example) - transportService.connectToNode(otherMaster); - transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, - new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), - new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - - @Override - public void handleException(TransportException exp) { - logger.warn(() -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp); - } - }); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e); - } - } - } - - private ZenPing.PingCollection pingAndWait(TimeValue timeout) { - final CompletableFuture response = new CompletableFuture<>(); - try { - zenPing.ping(response::complete, timeout); - } catch (Exception ex) { - // logged later - response.completeExceptionally(ex); - } - - try { - return response.get(); - } catch (InterruptedException e) { - logger.trace("pingAndWait interrupted"); - return new ZenPing.PingCollection(); - } catch (ExecutionException e) { - logger.warn("Ping execution failed", e); - return new ZenPing.PingCollection(); - } - } - - @Override - public void onIncomingClusterState(ClusterState incomingState) { - validateIncomingState(logger, incomingState, committedState.get()); - pendingStatesQueue.addPending(incomingState); - } - - @Override - public void onClusterStateCommitted(String stateUUID, ActionListener processedListener) { - final ClusterState state = pendingStatesQueue.markAsCommitted(stateUUID, - new PendingClusterStatesQueue.StateProcessedListener() { - @Override - public void onNewClusterStateProcessed() { - processedListener.onResponse(null); - } - - @Override - public void onNewClusterStateFailed(Exception e) { - processedListener.onFailure(e); - } - }); - if (state != null) { - synchronized (stateMutex) { - processNextCommittedClusterState("master " + state.nodes().getMasterNode() + - " committed version [" + state.version() + "]"); - } - } - } - - /** - * does simple sanity check of the incoming cluster state. Throws an exception on rejections. - */ - static void validateIncomingState(Logger logger, ClusterState incomingState, ClusterState lastState) { - final ClusterName incomingClusterName = incomingState.getClusterName(); - if (!incomingClusterName.equals(lastState.getClusterName())) { - logger.warn("received cluster state from [{}] which is also master but with a different cluster name [{}]", - incomingState.nodes().getMasterNode(), incomingClusterName); - throw new IllegalStateException("received state from a node that is not part of the cluster"); - } - if (lastState.nodes().getLocalNode().equals(incomingState.nodes().getLocalNode()) == false) { - logger.warn("received a cluster state from [{}] and not part of the cluster, should not happen", - incomingState.nodes().getMasterNode()); - throw new IllegalStateException("received state with a local node that does not match the current local node"); - } - - if (shouldIgnoreOrRejectNewClusterState(logger, lastState, incomingState)) { - String message = String.format( - Locale.ROOT, - "rejecting cluster state version [%d] uuid [%s] received from [%s]", - incomingState.version(), - incomingState.stateUUID(), - incomingState.nodes().getMasterNodeId() - ); - logger.warn(message); - throw new IllegalStateException(message); - } - - } - - private class MembershipListener implements MembershipAction.MembershipListener { - @Override - public void onJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) { - handleJoinRequest(node, ZenDiscovery.this.clusterState(), callback); - } - - @Override - public void onLeave(DiscoveryNode node) { - handleLeaveRequest(node); - } - } - - private class NodeFaultDetectionListener extends NodesFaultDetection.Listener { - - private final AtomicInteger pingsWhileMaster = new AtomicInteger(0); - - @Override - public void onNodeFailure(DiscoveryNode node, String reason) { - handleNodeFailure(node, reason); - } - - @Override - public void onPingReceived(final NodesFaultDetection.PingRequest pingRequest) { - // if we are master, we don't expect any fault detection from another node. If we get it - // means we potentially have two masters in the cluster. - if (!localNodeMaster()) { - pingsWhileMaster.set(0); - return; - } - - if (pingsWhileMaster.incrementAndGet() < maxPingsFromAnotherMaster) { - logger.trace("got a ping from another master {}. current ping count: [{}]", pingRequest.masterNode(), - pingsWhileMaster.get()); - return; - } - logger.debug("got a ping from another master {}. resolving who should rejoin. current ping count: [{}]", - pingRequest.masterNode(), pingsWhileMaster.get()); - synchronized (stateMutex) { - ClusterState currentState = committedState.get(); - if (currentState.nodes().isLocalNodeElectedMaster()) { - pingsWhileMaster.set(0); - handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping"); - } - } - } - } - - private class MasterNodeFailureListener implements MasterFaultDetection.Listener { - - @Override - public void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason) { - handleMasterGone(masterNode, cause, reason); - } - } - - public static class RejoinClusterRequest extends TransportRequest { - - private String fromNodeId; - - RejoinClusterRequest(String fromNodeId) { - this.fromNodeId = fromNodeId; - } - - public RejoinClusterRequest() { - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - fromNodeId = in.readOptionalString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeOptionalString(fromNodeId); - } - } - - class RejoinClusterRequestHandler implements TransportRequestHandler { - @Override - public void messageReceived(final RejoinClusterRequest request, final TransportChannel channel, Task task) throws Exception { - try { - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } catch (Exception e) { - logger.warn("failed to send response on rejoin cluster request handling", e); - } - synchronized (stateMutex) { - rejoin("received a request to rejoin the cluster from [" + request.fromNodeId + "]"); - } - } - } - - /** - * All control of the join thread should happen under the cluster state update task thread. - * This is important to make sure that the background joining process is always in sync with any cluster state updates - * like master loss, failure to join, received cluster state while joining etc. - */ - private class JoinThreadControl { - - private final AtomicBoolean running = new AtomicBoolean(false); - private final AtomicReference currentJoinThread = new AtomicReference<>(); - - /** returns true if join thread control is started and there is currently an active join thread */ - public boolean joinThreadActive() { - Thread currentThread = currentJoinThread.get(); - return running.get() && currentThread != null && currentThread.isAlive(); - } - - /** returns true if join thread control is started and the supplied thread is the currently active joinThread */ - public boolean joinThreadActive(Thread joinThread) { - return running.get() && joinThread.equals(currentJoinThread.get()); - } - - /** cleans any running joining thread and calls {@link #rejoin} */ - public void stopRunningThreadAndRejoin(String reason) { - assert Thread.holdsLock(stateMutex); - currentJoinThread.set(null); - rejoin(reason); - } - - /** starts a new joining thread if there is no currently active one and join thread controlling is started */ - public void startNewThreadIfNotRunning() { - assert Thread.holdsLock(stateMutex); - if (joinThreadActive()) { - return; - } - threadPool.generic().execute(new Runnable() { - @Override - public void run() { - Thread currentThread = Thread.currentThread(); - if (!currentJoinThread.compareAndSet(null, currentThread)) { - return; - } - while (running.get() && joinThreadActive(currentThread)) { - try { - innerJoinCluster(); - return; - } catch (Exception e) { - logger.error("unexpected error while joining cluster, trying again", e); - // Because we catch any exception here, we want to know in - // tests if an uncaught exception got to this point and the test infra uncaught exception - // leak detection can catch this. In practise no uncaught exception should leak - assert ExceptionsHelper.reThrowIfNotNull(e); - } - } - // cleaning the current thread from currentJoinThread is done by explicit calls. - } - }); - } - - /** - * marks the given joinThread as completed and makes sure another thread is running (starting one if needed) - * If the given thread is not the currently running join thread, the command is ignored. - */ - public void markThreadAsDoneAndStartNew(Thread joinThread) { - assert Thread.holdsLock(stateMutex); - if (!markThreadAsDone(joinThread)) { - return; - } - startNewThreadIfNotRunning(); - } - - /** marks the given joinThread as completed. Returns false if the supplied thread is not the currently active join thread */ - public boolean markThreadAsDone(Thread joinThread) { - assert Thread.holdsLock(stateMutex); - return currentJoinThread.compareAndSet(joinThread, null); - } - - public void stop() { - running.set(false); - Thread joinThread = currentJoinThread.getAndSet(null); - if (joinThread != null) { - joinThread.interrupt(); - } - } - - public void start() { - running.set(true); - } - - } - - public final Collection> getOnJoinValidators() { - return onJoinValidators; - } - - static class ZenNodeRemovalClusterStateTaskExecutor extends NodeRemovalClusterStateTaskExecutor { - - private final ElectMasterService electMasterService; - private final Consumer rejoin; - - ZenNodeRemovalClusterStateTaskExecutor( - final AllocationService allocationService, - final ElectMasterService electMasterService, - final Consumer rejoin, - final Logger logger) { - super(allocationService, logger); - this.electMasterService = electMasterService; - this.rejoin = rejoin; - } - - @Override - protected ClusterTasksResult getTaskClusterTasksResult(ClusterState currentState, List tasks, - ClusterState remainingNodesClusterState) { - if (electMasterService.hasEnoughMasterNodes(remainingNodesClusterState.nodes()) == false) { - final ClusterTasksResult.Builder resultBuilder = ClusterTasksResult.builder().successes(tasks); - final int masterNodes = electMasterService.countMasterNodes(remainingNodesClusterState.nodes()); - rejoin.accept(LoggerMessageFormat.format("not enough master nodes (has [{}], but needed [{}])", - masterNodes, electMasterService.minimumMasterNodes())); - return resultBuilder.build(currentState); - } else { - return super.getTaskClusterTasksResult(currentState, tasks, remainingNodesClusterState); - } - } - } -} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java b/server/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java deleted file mode 100644 index eddcf3de6e17c..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java +++ /dev/null @@ -1,196 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.unit.TimeValue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; - -import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; - -public interface ZenPing extends Releasable { - - void start(); - - void ping(Consumer resultsConsumer, TimeValue timeout); - - class PingResponse implements Writeable { - - /** - * An ID of a ping response that was generated on behalf of another node. Needs to be less than all other ping IDs so that fake ping - * responses don't override real ones. - */ - public static long FAKE_PING_ID = -1; - - private static final AtomicLong idGenerator = new AtomicLong(); - - // an always increasing unique identifier for this ping response. - // lower values means older pings. - private final long id; - - private final ClusterName clusterName; - - private final DiscoveryNode node; - - private final DiscoveryNode master; - - private final long clusterStateVersion; - - /** - * @param node the node which this ping describes - * @param master the current master of the node - * @param clusterName the cluster name of the node - * @param clusterStateVersion the current cluster state version of that node - * ({@link ElectMasterService.MasterCandidate#UNRECOVERED_CLUSTER_VERSION} for not recovered) - */ - public PingResponse(DiscoveryNode node, DiscoveryNode master, ClusterName clusterName, long clusterStateVersion) { - this(idGenerator.incrementAndGet(), node, master, clusterName, clusterStateVersion); - } - - /** - * @param id the ping's ID - * @param node the node which this ping describes - * @param master the current master of the node - * @param clusterName the cluster name of the node - * @param clusterStateVersion the current cluster state version of that node -* ({@link ElectMasterService.MasterCandidate#UNRECOVERED_CLUSTER_VERSION} for not recovered) - */ - public PingResponse(long id, DiscoveryNode node, DiscoveryNode master, ClusterName clusterName, long clusterStateVersion) { - this.id = id; - this.node = node; - this.master = master; - this.clusterName = clusterName; - this.clusterStateVersion = clusterStateVersion; - } - - public PingResponse(DiscoveryNode node, DiscoveryNode master, ClusterState state) { - this(node, master, state.getClusterName(), - state.blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK) ? - ElectMasterService.MasterCandidate.UNRECOVERED_CLUSTER_VERSION : state.version()); - } - - PingResponse(StreamInput in) throws IOException { - this.clusterName = new ClusterName(in); - this.node = new DiscoveryNode(in); - this.master = in.readOptionalWriteable(DiscoveryNode::new); - this.clusterStateVersion = in.readLong(); - this.id = in.readLong(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - clusterName.writeTo(out); - node.writeTo(out); - out.writeOptionalWriteable(master); - out.writeLong(clusterStateVersion); - out.writeLong(id); - } - - /** - * an always increasing unique identifier for this ping response. - * lower values means older pings. - */ - public long id() { - return this.id; - } - - /** - * the name of the cluster this node belongs to - */ - public ClusterName clusterName() { - return this.clusterName; - } - - /** the node which this ping describes */ - public DiscoveryNode node() { - return node; - } - - /** the current master of the node */ - public DiscoveryNode master() { - return master; - } - - /** - * the current cluster state version of that node ({@link ElectMasterService.MasterCandidate#UNRECOVERED_CLUSTER_VERSION} - * for not recovered) */ - public long getClusterStateVersion() { - return clusterStateVersion; - } - - @Override - public String toString() { - return "ping_response{node [" + node + "], id[" + id + "], master [" + master + "]," + - "cluster_state_version [" + clusterStateVersion + "], cluster_name[" + clusterName.value() + "]}"; - } - } - - - /** - * a utility collection of pings where only the most recent ping is stored per node - */ - class PingCollection { - - Map pings; - - public PingCollection() { - pings = new HashMap<>(); - } - - /** - * adds a ping if newer than previous pings from the same node - * - * @return true if added, false o.w. - */ - public synchronized boolean addPing(PingResponse ping) { - PingResponse existingResponse = pings.get(ping.node()); - // in case both existing and new ping have the same id (probably because they come - // from nodes from version <1.4.0) we prefer to use the last added one. - if (existingResponse == null || existingResponse.id() <= ping.id()) { - pings.put(ping.node(), ping); - return true; - } - return false; - } - - /** serialize current pings to a list. It is guaranteed that the list contains one ping response per node */ - public synchronized List toList() { - return new ArrayList<>(pings.values()); - } - - /** the number of nodes for which there are known pings */ - public synchronized int size() { - return pings.size(); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/gateway/Gateway.java b/server/src/main/java/org/elasticsearch/gateway/Gateway.java index cffb672f0cfda..62ce2bb6d797a 100644 --- a/server/src/main/java/org/elasticsearch/gateway/Gateway.java +++ b/server/src/main/java/org/elasticsearch/gateway/Gateway.java @@ -29,7 +29,6 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.index.Index; import org.elasticsearch.indices.IndicesService; @@ -44,7 +43,6 @@ public class Gateway { private final TransportNodesListGatewayMetaState listGatewayMetaState; - private final int minimumMasterNodes; private final IndicesService indicesService; public Gateway(final Settings settings, final ClusterService clusterService, @@ -53,7 +51,6 @@ public Gateway(final Settings settings, final ClusterService clusterService, this.indicesService = indicesService; this.clusterService = clusterService; this.listGatewayMetaState = listGatewayMetaState; - this.minimumMasterNodes = ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings); } public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException { @@ -61,7 +58,7 @@ public void performStateRecovery(final GatewayStateRecoveredListener listener) t logger.trace("performing state recovery from {}", Arrays.toString(nodesIds)); final TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds, null).actionGet(); - final int requiredAllocation = Math.max(1, minimumMasterNodes); + final int requiredAllocation = 1; if (nodesState.hasFailures()) { for (final FailedNodeException failedNodeException : nodesState.failures()) { diff --git a/server/src/main/java/org/elasticsearch/gateway/GatewayService.java b/server/src/main/java/org/elasticsearch/gateway/GatewayService.java index 04eb14669e7a2..dd6f031ee15c2 100644 --- a/server/src/main/java/org/elasticsearch/gateway/GatewayService.java +++ b/server/src/main/java/org/elasticsearch/gateway/GatewayService.java @@ -40,7 +40,6 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; @@ -115,8 +114,6 @@ public GatewayService(final Settings settings, final AllocationService allocatio // default the recover after master nodes to the minimum master nodes in the discovery if (RECOVER_AFTER_MASTER_NODES_SETTING.exists(settings)) { recoverAfterMasterNodes = RECOVER_AFTER_MASTER_NODES_SETTING.get(settings); - } else if (discovery instanceof ZenDiscovery) { - recoverAfterMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", -1); } else { recoverAfterMasterNodes = -1; } diff --git a/server/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java b/server/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java deleted file mode 100644 index 450acebc983e9..0000000000000 --- a/server/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.cluster.ack; - -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; -import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; -import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; -import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; -import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; -import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider; -import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; -import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.TransportService; - -import java.util.Arrays; -import java.util.Collection; -import java.util.stream.Stream; - -import static org.elasticsearch.test.ESIntegTestCase.Scope.TEST; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; - -@AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/33673") -@ClusterScope(scope = TEST, minNumDataNodes = 2) -public class AckClusterUpdateSettingsIT extends ESIntegTestCase { - - @Override - protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class); - } - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - //make sure that enough concurrent reroutes can happen at the same time - //we have a minimum of 2 nodes, and a maximum of 10 shards, thus 5 should be enough - .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), 5) - .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), 5) - .put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), 10) - .build(); - } - - @Override - protected int minimumNumberOfShards() { - return cluster().numDataNodes(); - } - - @Override - protected int numberOfReplicas() { - return 0; - } - - - private void removePublishTimeout() { - //to test that the acknowledgement mechanism is working we better disable the wait for publish - //otherwise the operation is most likely acknowledged even if it doesn't support ack - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "0") - .put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "30s") - )); - } - - public void testClusterUpdateSettingsAcknowledgement() { - createIndex("test"); - ensureGreen(); - - // now that the cluster is stable, remove timeout - removePublishTimeout(); - - NodesInfoResponse nodesInfo = client().admin().cluster().prepareNodesInfo().get(); - String excludedNodeId = null; - for (NodeInfo nodeInfo : nodesInfo.getNodes()) { - if (nodeInfo.getNode().isDataNode()) { - excludedNodeId = nodeInfo.getNode().getId(); - break; - } - } - assertNotNull(excludedNodeId); - - ClusterUpdateSettingsResponse clusterUpdateSettingsResponse = client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(Settings.builder().put("cluster.routing.allocation.exclude._id", excludedNodeId)).get(); - assertAcked(clusterUpdateSettingsResponse); - assertThat(clusterUpdateSettingsResponse.getTransientSettings().get("cluster.routing.allocation.exclude._id"), - equalTo(excludedNodeId)); - - for (Client client : clients()) { - ClusterState clusterState = getLocalClusterState(client); - assertThat(clusterState.metaData().transientSettings().get("cluster.routing.allocation.exclude._id"), equalTo(excludedNodeId)); - for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { - for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) { - for (ShardRouting shardRouting : indexShardRoutingTable) { - assert clusterState.nodes() != null; - if (shardRouting.unassigned() == false && clusterState.nodes() - .get(shardRouting.currentNodeId()).getId().equals(excludedNodeId)) { - // if the shard is still there it must be relocating and all nodes need to know, - // since the request was acknowledged reroute happens as part of the update settings - // and we made sure no throttling comes into the picture via settings - assertThat(shardRouting.relocating(), equalTo(true)); - } - } - } - } - } - } - - public void testClusterUpdateSettingsNoAcknowledgement() { - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder() - .put("number_of_shards", between(cluster().numDataNodes(), DEFAULT_MAX_NUM_SHARDS)) - .put("number_of_replicas", 0)).get(); - ensureGreen(); - - // now that the cluster is stable, remove timeout - removePublishTimeout(); - - NodesInfoResponse nodesInfo = client().admin().cluster().prepareNodesInfo().get(); - String excludedNodeId = null; - for (NodeInfo nodeInfo : nodesInfo.getNodes()) { - if (nodeInfo.getNode().isDataNode()) { - excludedNodeId = nodeInfo.getNode().getId(); - break; - } - } - assertNotNull(excludedNodeId); - - ClusterUpdateSettingsResponse clusterUpdateSettingsResponse = client().admin().cluster().prepareUpdateSettings().setTimeout("0s") - .setTransientSettings(Settings.builder().put("cluster.routing.allocation.exclude._id", excludedNodeId)).get(); - assertThat(clusterUpdateSettingsResponse.isAcknowledged(), equalTo(false)); - assertThat(clusterUpdateSettingsResponse.getTransientSettings().get("cluster.routing.allocation.exclude._id"), - equalTo(excludedNodeId)); - } - - private static ClusterState getLocalClusterState(Client client) { - return client.admin().cluster().prepareState().setLocal(true).get().getState(); - } - - public void testOpenIndexNoAcknowledgement() { - createIndex("test"); - ensureGreen(); - removePublishTimeout(); - AcknowledgedResponse closeIndexResponse = client().admin().indices().prepareClose("test").execute().actionGet(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(true)); - - OpenIndexResponse openIndexResponse = client().admin().indices().prepareOpen("test").setTimeout("0s").get(); - assertThat(openIndexResponse.isAcknowledged(), equalTo(false)); - ensureGreen("test"); // make sure that recovery from disk has completed, so that check index doesn't fail. - } - - public void testAckingFailsIfNotPublishedToAllNodes() { - String masterNode = internalCluster().getMasterName(); - String nonMasterNode = Stream.of(internalCluster().getNodeNames()) - .filter(node -> node.equals(masterNode) == false).findFirst().get(); - - MockTransportService masterTransportService = - (MockTransportService) internalCluster().getInstance(TransportService.class, masterNode); - MockTransportService nonMasterTransportService = - (MockTransportService) internalCluster().getInstance(TransportService.class, nonMasterNode); - - logger.info("blocking cluster state publishing from master [{}] to non master [{}]", masterNode, nonMasterNode); - if (randomBoolean() && internalCluster().numMasterNodes() != 2) { - masterTransportService.addFailToSendNoConnectRule(nonMasterTransportService, PublishClusterStateAction.SEND_ACTION_NAME); - } else { - masterTransportService.addFailToSendNoConnectRule(nonMasterTransportService, PublishClusterStateAction.COMMIT_ACTION_NAME); - } - - CreateIndexResponse response = client().admin().indices().prepareCreate("test").get(); - assertFalse(response.isAcknowledged()); - - logger.info("waiting for cluster to reform"); - masterTransportService.clearRule(nonMasterTransportService); - - ensureStableCluster(internalCluster().size()); - - assertAcked(client().admin().indices().prepareDelete("test")); - } -} diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ZenDiscoveryIT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ZenDiscoveryIT.java index d737ef790b5bd..9a17c25f44cce 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ZenDiscoveryIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ZenDiscoveryIT.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoveryStats; -import org.elasticsearch.discovery.zen.FaultDetection; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.TestCustomMetaData; @@ -64,19 +63,12 @@ public class ZenDiscoveryIT extends ESIntegTestCase { public void testNoShardRelocationsOccurWhenElectedMasterNodeFails() throws Exception { - Settings defaultSettings = Settings.builder() - .put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") - .put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") - .build(); - Settings masterNodeSettings = Settings.builder() .put(Node.NODE_DATA_SETTING.getKey(), false) - .put(defaultSettings) .build(); internalCluster().startNodes(2, masterNodeSettings); Settings dateNodeSettings = Settings.builder() .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(defaultSettings) .build(); internalCluster().startNodes(2, dateNodeSettings); ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth() @@ -106,19 +98,12 @@ public void testNoShardRelocationsOccurWhenElectedMasterNodeFails() throws Excep } public void testNodeFailuresAreProcessedOnce() throws IOException { - Settings defaultSettings = Settings.builder() - .put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") - .put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") - .build(); - Settings masterNodeSettings = Settings.builder() .put(Node.NODE_DATA_SETTING.getKey(), false) - .put(defaultSettings) .build(); String master = internalCluster().startNode(masterNodeSettings); Settings dateNodeSettings = Settings.builder() .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(defaultSettings) .build(); internalCluster().startNodes(2, dateNodeSettings); client().admin().cluster().prepareHealth().setWaitForNodes("3").get(); diff --git a/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java index 7f4765df31009..85e14e1decc2a 100644 --- a/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java @@ -26,9 +26,6 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.MembershipAction; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect; @@ -46,7 +43,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING; import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; /** * Tests for discovery during disruptions. @@ -80,18 +76,15 @@ public void testClusterJoinDespiteOfPublishingIssues() throws Exception { TransportService localTransportService = internalCluster().getInstance(TransportService.class, discoveryNodes.getLocalNode().getName()); if (randomBoolean()) { - masterTransportService.addFailToSendNoConnectRule(localTransportService, PublishClusterStateAction.SEND_ACTION_NAME, - PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME); + masterTransportService.addFailToSendNoConnectRule(localTransportService, PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME); } else { - masterTransportService.addFailToSendNoConnectRule(localTransportService, PublishClusterStateAction.COMMIT_ACTION_NAME, - PublicationTransportHandler.COMMIT_STATE_ACTION_NAME); + masterTransportService.addFailToSendNoConnectRule(localTransportService, PublicationTransportHandler.COMMIT_STATE_ACTION_NAME); } logger.info("allowing requests from non master [{}] to master [{}], waiting for two join request", nonMasterNode, masterNode); final CountDownLatch countDownLatch = new CountDownLatch(2); nonMasterTransportService.addSendBehavior(masterTransportService, (connection, requestId, action, request, options) -> { - if (action.equals(MembershipAction.DISCOVERY_JOIN_ACTION_NAME) || - action.equals(JoinHelper.JOIN_ACTION_NAME)) { + if (action.equals(JoinHelper.JOIN_ACTION_NAME)) { countDownLatch.countDown(); } connection.sendRequest(requestId, action, request, options); @@ -143,15 +136,6 @@ public void testElectMasterWithLatestVersion() throws Exception { ensureStableCluster(3); final String preferredMasterName = internalCluster().getMasterName(); final DiscoveryNode preferredMaster = internalCluster().clusterService(preferredMasterName).localNode(); - final Discovery discovery = internalCluster().getInstance(Discovery.class); - // only Zen1 guarantees that node with lowest id is elected - if (discovery instanceof ZenDiscovery) { - for (String node : nodes) { - DiscoveryNode discoveryNode = internalCluster().clusterService(node).localNode(); - assertThat(discoveryNode.getId(), greaterThanOrEqualTo(preferredMaster.getId())); - } - } - logger.info("--> preferred master is {}", preferredMaster); final Set nonPreferredNodes = new HashSet<>(nodes); nonPreferredNodes.remove(preferredMasterName); diff --git a/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java index a345859ad9eb3..60e2fa0b5687e 100644 --- a/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java @@ -35,8 +35,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.discovery.zen.ZenDiscovery; -import org.elasticsearch.monitor.jvm.HotThreads; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.disruption.BlockMasterServiceOnMaster; import org.elasticsearch.test.disruption.IntermittentLongGCDisruption; @@ -126,19 +124,6 @@ public void testStaleMasterNotHijackingMajority() throws Exception { assertDifferentMaster(majoritySide.get(0), oldMasterNode); assertDifferentMaster(majoritySide.get(1), oldMasterNode); - // the test is periodically tripping on the following assertion. To find out which threads are blocking the nodes from making - // progress we print a stack dump - boolean failed = true; - try { - assertDiscoveryCompleted(majoritySide); - failed = false; - } finally { - if (failed) { - logger.error("discovery failed to complete, probably caused by a blocked thread: {}", - new HotThreads().busiestThreads(Integer.MAX_VALUE).ignoreIdleThreads(false).detect()); - } - } - // The old master node is frozen, but here we submit a cluster state update task that doesn't get executed, // but will be queued and once the old master node un-freezes it gets executed. // The old master node will send this update + the cluster state where he is flagged as master to the other @@ -166,7 +151,6 @@ public void onFailure(String source, Exception e) { oldMasterNodeSteppedDown.await(30, TimeUnit.SECONDS); // Make sure that the end state is consistent on all nodes: - assertDiscoveryCompleted(nodes); assertMaster(newMasterNode, nodes); assertThat(masters.size(), equalTo(2)); @@ -211,9 +195,6 @@ public void testMasterNodeGCs() throws Exception { logger.info("waiting for nodes to elect a new master"); ensureStableCluster(2, oldNonMasterNodes.get(0)); - logger.info("waiting for any pinging to stop"); - assertDiscoveryCompleted(oldNonMasterNodes); - // restore GC masterNodeDisruption.stopDisrupting(); final TimeValue waitTime = new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + masterNodeDisruption.expectedTimeToHeal().millis()); @@ -428,23 +409,4 @@ public void testMappingTimeout() throws Exception { }); } - - private void assertDiscoveryCompleted(List nodes) throws InterruptedException { - for (final String node : nodes) { - assertTrue( - "node [" + node + "] is still joining master", - awaitBusy( - () -> { - final Discovery discovery = internalCluster().getInstance(Discovery.class, node); - if (discovery instanceof ZenDiscovery) { - return !((ZenDiscovery) discovery).joiningCluster(); - } - return true; - }, - 30, - TimeUnit.SECONDS - ) - ); - } - } } diff --git a/server/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java b/server/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java deleted file mode 100644 index 4aea0892e8257..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java +++ /dev/null @@ -1,359 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery; - -import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.network.NetworkService; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.PageCacheRecycler; -import org.elasticsearch.discovery.zen.FaultDetection; -import org.elasticsearch.discovery.zen.MasterFaultDetection; -import org.elasticsearch.discovery.zen.NodesFaultDetection; -import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; -import org.elasticsearch.node.Node; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportConnectionListener; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.TransportSettings; -import org.elasticsearch.transport.nio.MockNioTransport; -import org.hamcrest.Matcher; -import org.hamcrest.Matchers; -import org.junit.After; -import org.junit.Before; - -import java.util.Collections; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; - -public class ZenFaultDetectionTests extends ESTestCase { - protected ThreadPool threadPool; - private CircuitBreakerService circuitBreakerService; - - protected static final Version version0 = Version.fromId(/*0*/99); - protected DiscoveryNode nodeA; - protected MockTransportService serviceA; - private Settings settingsA; - - protected static final Version version1 = Version.fromId(199); - protected DiscoveryNode nodeB; - protected MockTransportService serviceB; - private Settings settingsB; - - @Override - @Before - public void setUp() throws Exception { - super.setUp(); - Settings settings = Settings.builder() - .put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), new ByteSizeValue(0)) - .build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool = new TestThreadPool(getClass().getName()); - circuitBreakerService = new HierarchyCircuitBreakerService(settings, clusterSettings); - settingsA = Settings.builder().put("node.name", "TS_A").put(settings).build(); - serviceA = build(settingsA, version0); - nodeA = serviceA.getLocalDiscoNode(); - settingsB = Settings.builder().put("node.name", "TS_B").put(settings).build(); - serviceB = build(settingsB, version1); - nodeB = serviceB.getLocalDiscoNode(); - - // wait till all nodes are properly connected and the event has been sent, so tests in this class - // will not get this callback called on the connections done in this setup - final CountDownLatch latch = new CountDownLatch(2); - TransportConnectionListener waitForConnection = new TransportConnectionListener() { - @Override - public void onNodeConnected(DiscoveryNode node) { - latch.countDown(); - } - - @Override - public void onNodeDisconnected(DiscoveryNode node) { - fail("disconnect should not be called " + node); - } - }; - serviceA.addConnectionListener(waitForConnection); - serviceB.addConnectionListener(waitForConnection); - - serviceA.connectToNode(nodeB); - serviceA.connectToNode(nodeA); - serviceB.connectToNode(nodeA); - serviceB.connectToNode(nodeB); - - assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true)); - serviceA.removeConnectionListener(waitForConnection); - serviceB.removeConnectionListener(waitForConnection); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - serviceA.close(); - serviceB.close(); - terminate(threadPool); - } - - protected MockTransportService build(Settings settings, Version version) { - NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); - MockTransportService transportService = - new MockTransportService( - Settings.builder() - .put(settings) - // trace zenfd actions but keep the default otherwise - .putList(TransportSettings.TRACE_LOG_EXCLUDE_SETTING.getKey(), TransportLivenessAction.NAME) - .build(), - new MockNioTransport(settings, version, threadPool, new NetworkService(Collections.emptyList()), - PageCacheRecycler.NON_RECYCLING_INSTANCE, namedWriteableRegistry, circuitBreakerService), - threadPool, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, - (boundAddress) -> - new DiscoveryNode(Node.NODE_NAME_SETTING.get(settings), boundAddress.publishAddress(), - Node.NODE_ATTRIBUTES.getAsMap(settings), DiscoveryNode.getRolesFromSettings(settings), version), - null, Collections.emptySet()); - transportService.start(); - transportService.acceptIncomingRequests(); - return transportService; - } - - private DiscoveryNodes buildNodesForA(boolean master) { - DiscoveryNodes.Builder builder = DiscoveryNodes.builder(); - builder.add(nodeA); - builder.add(nodeB); - builder.localNodeId(nodeA.getId()); - builder.masterNodeId(master ? nodeA.getId() : nodeB.getId()); - return builder.build(); - } - - private DiscoveryNodes buildNodesForB(boolean master) { - DiscoveryNodes.Builder builder = DiscoveryNodes.builder(); - builder.add(nodeA); - builder.add(nodeB); - builder.localNodeId(nodeB.getId()); - builder.masterNodeId(master ? nodeB.getId() : nodeA.getId()); - return builder.build(); - } - - public void testNodesFaultDetectionConnectOnDisconnect() throws InterruptedException { - boolean shouldRetry = randomBoolean(); - // make sure we don't ping again after the initial ping - final Settings pingSettings = Settings.builder() - .put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry) - .put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m").build(); - ClusterState clusterState = ClusterState.builder(new ClusterName("test")).version(randomNonNegativeLong()) - .nodes(buildNodesForA(true)).build(); - NodesFaultDetection nodesFDA = new NodesFaultDetection(Settings.builder().put(settingsA).put(pingSettings).build(), - threadPool, serviceA, () -> clusterState, clusterState.getClusterName()); - nodesFDA.setLocalNode(nodeA); - NodesFaultDetection nodesFDB = new NodesFaultDetection(Settings.builder().put(settingsB).put(pingSettings).build(), - threadPool, serviceB, () -> clusterState, clusterState.getClusterName()); - nodesFDB.setLocalNode(nodeB); - final CountDownLatch pingSent = new CountDownLatch(1); - nodesFDB.addListener(new NodesFaultDetection.Listener() { - @Override - public void onPingReceived(NodesFaultDetection.PingRequest pingRequest) { - assertThat(pingRequest.clusterStateVersion(), equalTo(clusterState.version())); - pingSent.countDown(); - } - }); - nodesFDA.updateNodesAndPing(clusterState); - - // wait for the first ping to go out, so we will really respond to a disconnect event rather then - // the ping failing - pingSent.await(30, TimeUnit.SECONDS); - - final String[] failureReason = new String[1]; - final DiscoveryNode[] failureNode = new DiscoveryNode[1]; - final CountDownLatch notified = new CountDownLatch(1); - nodesFDA.addListener(new NodesFaultDetection.Listener() { - @Override - public void onNodeFailure(DiscoveryNode node, String reason) { - failureNode[0] = node; - failureReason[0] = reason; - notified.countDown(); - } - }); - // will raise a disconnect on A - serviceB.stop(); - notified.await(30, TimeUnit.SECONDS); - - CircuitBreaker inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); - assertThat(inFlightRequestsBreaker.getTrippedCount(), equalTo(0L)); - - assertEquals(nodeB, failureNode[0]); - Matcher matcher = Matchers.containsString("verified"); - if (!shouldRetry) { - matcher = Matchers.not(matcher); - } - - assertThat(failureReason[0], matcher); - - assertWarnings( - "[discovery.zen.fd.connect_on_network_disconnect] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version.", - "[discovery.zen.fd.ping_interval] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version."); - } - - public void testMasterFaultDetectionConnectOnDisconnect() throws InterruptedException { - Settings.Builder settings = Settings.builder(); - boolean shouldRetry = randomBoolean(); - ClusterName clusterName = new ClusterName(randomAlphaOfLengthBetween(3, 20)); - - // make sure we don't ping - settings.put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry) - .put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m").put("cluster.name", clusterName.value()); - - final ClusterState state = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build(); - AtomicReference clusterStateSupplier = new AtomicReference<>(state); - MasterFaultDetection masterFD = new MasterFaultDetection(settings.build(), threadPool, serviceA, - clusterStateSupplier::get, null, clusterName); - masterFD.restart(nodeB, "test"); - - final String[] failureReason = new String[1]; - final DiscoveryNode[] failureNode = new DiscoveryNode[1]; - final CountDownLatch notified = new CountDownLatch(1); - masterFD.addListener((masterNode, cause, reason) -> { - failureNode[0] = masterNode; - failureReason[0] = reason; - notified.countDown(); - }); - // will raise a disconnect on A - serviceB.stop(); - notified.await(30, TimeUnit.SECONDS); - - CircuitBreaker inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); - assertThat(inFlightRequestsBreaker.getTrippedCount(), equalTo(0L)); - - assertEquals(nodeB, failureNode[0]); - Matcher matcher = Matchers.containsString("verified"); - if (!shouldRetry) { - matcher = Matchers.not(matcher); - } - - assertThat(failureReason[0], matcher); - - assertWarnings( - "[discovery.zen.fd.connect_on_network_disconnect] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version.", - "[discovery.zen.fd.ping_interval] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version."); - } - - public void testMasterFaultDetectionNotSizeLimited() throws InterruptedException { - boolean shouldRetry = randomBoolean(); - ClusterName clusterName = new ClusterName(randomAlphaOfLengthBetween(3, 20)); - final Settings settings = Settings.builder() - .put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry) - .put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "1s") - .put("cluster.name", clusterName.value()).build(); - final ClusterState stateNodeA = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build(); - AtomicReference clusterStateSupplierA = new AtomicReference<>(stateNodeA); - - int minExpectedPings = 2; - - PingProbe pingProbeA = new PingProbe(minExpectedPings); - PingProbe pingProbeB = new PingProbe(minExpectedPings); - - serviceA.addTracer(pingProbeA); - serviceB.addTracer(pingProbeB); - - MasterFaultDetection masterFDNodeA = new MasterFaultDetection(Settings.builder().put(settingsA).put(settings).build(), - threadPool, serviceA, clusterStateSupplierA::get, null, clusterName); - masterFDNodeA.restart(nodeB, "test"); - - final ClusterState stateNodeB = ClusterState.builder(clusterName).nodes(buildNodesForB(true)).build(); - AtomicReference clusterStateSupplierB = new AtomicReference<>(stateNodeB); - - MasterFaultDetection masterFDNodeB = new MasterFaultDetection(Settings.builder().put(settingsB).put(settings).build(), - threadPool, serviceB, clusterStateSupplierB::get, null, clusterName); - masterFDNodeB.restart(nodeB, "test"); - - // let's do a few pings - pingProbeA.awaitMinCompletedPings(); - pingProbeB.awaitMinCompletedPings(); - - CircuitBreaker inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); - assertThat(inFlightRequestsBreaker.getTrippedCount(), equalTo(0L)); - assertThat(pingProbeA.completedPings(), greaterThanOrEqualTo(minExpectedPings)); - assertThat(pingProbeB.completedPings(), greaterThanOrEqualTo(minExpectedPings)); - - assertWarnings( - "[discovery.zen.fd.connect_on_network_disconnect] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version.", - "[discovery.zen.fd.ping_interval] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version."); - } - - private static class PingProbe extends MockTransportService.Tracer { - private final Set> inflightPings = Collections.newSetFromMap(new ConcurrentHashMap<>()); - private final Set> completedPings = Collections.newSetFromMap(new ConcurrentHashMap<>()); - private final CountDownLatch waitForPings; - - PingProbe(int minCompletedPings) { - this.waitForPings = new CountDownLatch(minCompletedPings); - } - - @Override - public void requestSent(DiscoveryNode node, long requestId, String action, TransportRequestOptions options) { - if (MasterFaultDetection.MASTER_PING_ACTION_NAME.equals(action)) { - inflightPings.add(Tuple.tuple(node, requestId)); - } - } - - @Override - public void receivedResponse(long requestId, DiscoveryNode sourceNode, String action) { - if (MasterFaultDetection.MASTER_PING_ACTION_NAME.equals(action)) { - Tuple ping = Tuple.tuple(sourceNode, requestId); - if (inflightPings.remove(ping)) { - completedPings.add(ping); - waitForPings.countDown(); - } - } - } - - public int completedPings() { - return completedPings.size(); - } - - public void awaitMinCompletedPings() throws InterruptedException { - waitForPings.await(); - } - } -} diff --git a/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java b/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java index 13d314c57505b..e16389a38471f 100644 --- a/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java @@ -19,33 +19,18 @@ package org.elasticsearch.discovery.single; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.SeedHostsProvider; -import org.elasticsearch.discovery.zen.UnicastZenPing; -import org.elasticsearch.discovery.zen.ZenPing; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.MockHttpTransport; import org.elasticsearch.test.NodeConfigurationSource; -import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.transport.TransportService; -import java.io.Closeable; import java.io.IOException; import java.nio.file.Path; import java.util.Arrays; -import java.util.Collections; -import java.util.Stack; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; import java.util.function.Function; import static org.hamcrest.Matchers.equalTo; @@ -69,53 +54,6 @@ protected Settings nodeSettings(int nodeOrdinal) { .build(); } - public void testDoesNotRespondToZenPings() throws Exception { - final Settings settings = - Settings.builder().put("cluster.name", internalCluster().getClusterName()).build(); - final Version version = Version.CURRENT; - final Stack closeables = new Stack<>(); - final TestThreadPool threadPool = new TestThreadPool(getClass().getName()); - try { - final MockTransportService pingTransport = - MockTransportService.createNewService(settings, version, threadPool, null); - pingTransport.start(); - closeables.push(pingTransport); - final TransportService nodeTransport = - internalCluster().getInstance(TransportService.class); - // try to ping the single node directly - final SeedHostsProvider provider = - hostsResolver -> Collections.singletonList(nodeTransport.getLocalNode().getAddress()); - final CountDownLatch latch = new CountDownLatch(1); - final DiscoveryNodes nodes = DiscoveryNodes.builder() - .add(nodeTransport.getLocalNode()) - .add(pingTransport.getLocalNode()) - .localNodeId(pingTransport.getLocalNode().getId()) - .build(); - final ClusterName clusterName = new ClusterName(internalCluster().getClusterName()); - final ClusterState state = ClusterState.builder(clusterName).nodes(nodes).build(); - final UnicastZenPing unicastZenPing = - new UnicastZenPing(settings, threadPool, pingTransport, provider, () -> state) { - @Override - protected void finishPingingRound(PingingRound pingingRound) { - latch.countDown(); - super.finishPingingRound(pingingRound); - } - }; - unicastZenPing.start(); - closeables.push(unicastZenPing); - final CompletableFuture responses = new CompletableFuture<>(); - unicastZenPing.ping(responses::complete, TimeValue.timeValueSeconds(3)); - latch.await(); - responses.get(); - assertThat(responses.get().size(), equalTo(0)); - } finally { - while (!closeables.isEmpty()) { - IOUtils.closeWhileHandlingException(closeables.pop()); - } - terminate(threadPool); - } - } - public void testSingleNodesDoNotDiscoverEachOther() throws IOException, InterruptedException { final TransportService service = internalCluster().getInstance(TransportService.class); final int port = service.boundAddress().publishAddress().getPort(); diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java deleted file mode 100644 index 2f3bd44f47586..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.elasticsearch.Version; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.ElectMasterService.MasterCandidate; -import org.elasticsearch.test.ESTestCase; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; - -public class ElectMasterServiceTests extends ESTestCase { - - ElectMasterService electMasterService() { - return new ElectMasterService(Settings.EMPTY); - } - - List generateRandomNodes() { - int count = scaledRandomIntBetween(1, 100); - ArrayList nodes = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - Set roles = new HashSet<>(); - if (randomBoolean()) { - roles.add(DiscoveryNode.Role.MASTER); - } - DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), - roles, Version.CURRENT); - nodes.add(node); - } - - Collections.shuffle(nodes, random()); - return nodes; - } - - List generateRandomCandidates() { - int count = scaledRandomIntBetween(1, 100); - ArrayList candidates = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - Set roles = new HashSet<>(); - roles.add(DiscoveryNode.Role.MASTER); - DiscoveryNode node = new DiscoveryNode("n_" + i, "n_" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), - roles, Version.CURRENT); - candidates.add( - new MasterCandidate(node, randomBoolean() ? MasterCandidate.UNRECOVERED_CLUSTER_VERSION : randomNonNegativeLong())); - } - - Collections.shuffle(candidates, random()); - return candidates; - } - - - public void testSortByMasterLikelihood() { - List nodes = generateRandomNodes(); - List sortedNodes = ElectMasterService.sortByMasterLikelihood(nodes); - assertEquals(nodes.size(), sortedNodes.size()); - DiscoveryNode prevNode = sortedNodes.get(0); - for (int i = 1; i < sortedNodes.size(); i++) { - DiscoveryNode node = sortedNodes.get(i); - if (!prevNode.isMasterNode()) { - assertFalse(node.isMasterNode()); - } else if (node.isMasterNode()) { - assertTrue(prevNode.getId().compareTo(node.getId()) < 0); - } - prevNode = node; - } - } - - public void testTieBreakActiveMasters() { - List nodes = generateRandomCandidates().stream().map(MasterCandidate::getNode).collect(Collectors.toList()); - DiscoveryNode bestMaster = electMasterService().tieBreakActiveMasters(nodes); - for (DiscoveryNode node: nodes) { - if (node.equals(bestMaster) == false) { - assertTrue(bestMaster.getId().compareTo(node.getId()) < 0); - } - } - } - - public void testHasEnoughNodes() { - List nodes = rarely() ? Collections.emptyList() : generateRandomNodes(); - ElectMasterService service = electMasterService(); - int masterNodes = (int) nodes.stream().filter(DiscoveryNode::isMasterNode).count(); - service.minimumMasterNodes(randomIntBetween(-1, masterNodes)); - assertThat(service.hasEnoughMasterNodes(nodes), equalTo(masterNodes > 0)); - service.minimumMasterNodes(masterNodes + 1 + randomIntBetween(0, nodes.size())); - assertFalse(service.hasEnoughMasterNodes(nodes)); - } - - public void testHasEnoughCandidates() { - List candidates = rarely() ? Collections.emptyList() : generateRandomCandidates(); - ElectMasterService service = electMasterService(); - service.minimumMasterNodes(randomIntBetween(-1, candidates.size())); - assertThat(service.hasEnoughCandidates(candidates), equalTo(candidates.size() > 0)); - service.minimumMasterNodes(candidates.size() + 1 + randomIntBetween(0, candidates.size())); - assertFalse(service.hasEnoughCandidates(candidates)); - } - - public void testElectMaster() { - List candidates = generateRandomCandidates(); - ElectMasterService service = electMasterService(); - int minMasterNodes = randomIntBetween(0, candidates.size()); - service.minimumMasterNodes(minMasterNodes); - MasterCandidate master = service.electMaster(candidates); - assertNotNull(master); - for (MasterCandidate candidate : candidates) { - if (candidate.getNode().equals(master.getNode())) { - // nothing much to test here - } else if (candidate.getClusterStateVersion() == master.getClusterStateVersion()) { - assertThat("candidate " + candidate + " has a lower or equal id than master " + master, candidate.getNode().getId(), - greaterThan(master.getNode().getId())); - } else { - assertThat("candidate " + master + " has a higher cluster state version than candidate " + candidate, - master.getClusterStateVersion(), greaterThan(candidate.getClusterStateVersion())); - } - } - } - - public void testCountMasterNodes() { - List nodes = generateRandomNodes(); - ElectMasterService service = electMasterService(); - - int masterNodes = 0; - - for (DiscoveryNode node : nodes) { - if (node.isMasterNode()) { - masterNodes++; - } - } - - assertEquals(masterNodes, service.countMasterNodes(nodes)); - } -} diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java deleted file mode 100644 index 35a2173e0aea0..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ /dev/null @@ -1,868 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateUpdateTask; -import org.elasticsearch.cluster.NotMasterException; -import org.elasticsearch.cluster.block.ClusterBlocks; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; -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.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.TestShardRouting; -import org.elasticsearch.cluster.routing.UnassignedInfo; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.BaseFuture; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.test.ClusterServiceUtils; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; - -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.BrokenBarrierException; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static java.util.Collections.shuffle; -import static org.elasticsearch.cluster.ESAllocationTestCase.createAllocationService; -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.cluster.service.MasterServiceTests.discoveryState; -import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; -import static org.elasticsearch.test.ClusterServiceUtils.setState; -import static org.elasticsearch.test.VersionUtils.allVersions; -import static org.elasticsearch.test.VersionUtils.getPreviousVersion; -import static org.elasticsearch.test.VersionUtils.randomCompatibleVersion; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; - -@TestLogging("org.elasticsearch.discovery.zen:TRACE,org.elasticsearch.cluster.service:TRACE") -public class NodeJoinControllerTests extends ESTestCase { - - private static ThreadPool threadPool; - - private MasterService masterService; - private NodeJoinController nodeJoinController; - - @BeforeClass - public static void beforeClass() { - threadPool = new TestThreadPool("NodeJoinControllerTests"); - } - - @AfterClass - public static void afterClass() { - ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); - threadPool = null; - } - - @Before - public void setUp() throws Exception { - super.setUp(); - } - - @After - public void tearDown() throws Exception { - super.tearDown(); - masterService.close(); - } - - private static ClusterState initialState(boolean withMaster) { - DiscoveryNode localNode = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), - new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())),Version.CURRENT); - ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName())) - .nodes(DiscoveryNodes.builder() - .add(localNode) - .localNodeId(localNode.getId()) - .masterNodeId(withMaster ? localNode.getId() : null)) - .blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).build(); - return initialClusterState; - } - - private void setupMasterServiceAndNodeJoinController(ClusterState initialState) { - if (masterService != null || nodeJoinController != null) { - throw new IllegalStateException("method setupMasterServiceAndNodeJoinController can only be called once"); - } - masterService = ClusterServiceUtils.createMasterService(threadPool, initialState); - nodeJoinController = new NodeJoinController(Settings.EMPTY, masterService, createAllocationService(Settings.EMPTY), - new ElectMasterService(Settings.EMPTY)); - } - - public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException { - setupMasterServiceAndNodeJoinController(initialState(true)); - List nodes = new ArrayList<>(); - nodes.add(discoveryState(masterService).nodes().getLocalNode()); - - int nodeId = 0; - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - joinNode(node); - } - nodeJoinController.startElectionContext(); - ArrayList> pendingJoins = new ArrayList<>(); - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - pendingJoins.add(joinNodeAsync(node)); - } - nodeJoinController.stopElectionContext("test"); - boolean hadSyncJoin = false; - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - joinNode(node); - hadSyncJoin = true; - } - if (hadSyncJoin) { - for (Future joinFuture : pendingJoins) { - assertThat(joinFuture.isDone(), equalTo(true)); - } - } - for (Future joinFuture : pendingJoins) { - joinFuture.get(); - } - } - - public void testFailingJoinsWhenNotMaster() throws ExecutionException, InterruptedException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - - logger.debug("--> testing joins fail post accumulation"); - ArrayList> pendingJoins = new ArrayList<>(); - nodeJoinController.startElectionContext(); - for (int i = 1 + randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - final Future future = joinNodeAsync(node); - pendingJoins.add(future); - assertThat(future.isDone(), equalTo(false)); - } - nodeJoinController.stopElectionContext("test"); - for (Future future : pendingJoins) { - try { - future.get(); - fail("failed to fail accumulated node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - } - - public void testSimpleMasterElectionWithoutRequiredJoins() throws InterruptedException, ExecutionException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - final int requiredJoins = 0; - logger.debug("--> using requiredJoins [{}]", requiredJoins); - // initial (failing) joins shouldn't count - for (int i = randomInt(5); i > 0; i--) { - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - - nodeJoinController.startElectionContext(); - final SimpleFuture electionFuture = new SimpleFuture("master election"); - final Thread masterElection = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error from waitToBeElectedAsMaster", e); - electionFuture.markAsFailed(e); - } - - @Override - protected void doRun() throws Exception { - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), - new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), - equalTo(true)); - electionFuture.markAsDone(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("unexpected error while waiting to be elected as master", t); - electionFuture.markAsFailed(t); - } - }); - } - }); - masterElection.start(); - - logger.debug("--> requiredJoins is set to 0. verifying election finished"); - electionFuture.get(); - } - - public void testSimpleMasterElection() throws InterruptedException, ExecutionException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - final int requiredJoins = 1 + randomInt(5); - logger.debug("--> using requiredJoins [{}]", requiredJoins); - // initial (failing) joins shouldn't count - for (int i = randomInt(5); i > 0; i--) { - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - - nodeJoinController.startElectionContext(); - final SimpleFuture electionFuture = new SimpleFuture("master election"); - final Thread masterElection = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error from waitToBeElectedAsMaster", e); - electionFuture.markAsFailed(e); - } - - @Override - protected void doRun() throws Exception { - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), - new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), - equalTo(true)); - electionFuture.markAsDone(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("unexpected error while waiting to be elected as master", t); - electionFuture.markAsFailed(t); - } - }); - } - }); - masterElection.start(); - assertThat("election finished immediately but required joins is [" + requiredJoins + "]", electionFuture.isDone(), equalTo(false)); - - final int initialJoins = randomIntBetween(0, requiredJoins - 1); - final ArrayList pendingJoins = new ArrayList<>(); - ArrayList nodesToJoin = new ArrayList<>(); - for (int i = 0; i < initialJoins; i++) { - DiscoveryNode node = newNode(nodeId++, true); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - // data nodes shouldn't count - for (int i = 0; i < requiredJoins; i++) { - DiscoveryNode node = newNode(nodeId++, false); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - // add - - shuffle(nodesToJoin, random()); - logger.debug("--> joining [{}] unique master nodes. Total of [{}] join requests", initialJoins, nodesToJoin.size()); - for (DiscoveryNode node : nodesToJoin) { - pendingJoins.add(joinNodeAsync(node)); - } - - logger.debug("--> asserting master election didn't finish yet"); - assertThat("election finished after [" + initialJoins + "] master nodes but required joins is [" + requiredJoins + "]", - electionFuture.isDone(), equalTo(false)); - - final int finalJoins = requiredJoins - initialJoins + randomInt(5); - nodesToJoin.clear(); - for (int i = 0; i < finalJoins; i++) { - DiscoveryNode node = newNode(nodeId++, true); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - for (int i = 0; i < requiredJoins; i++) { - DiscoveryNode node = newNode(nodeId++, false); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - shuffle(nodesToJoin, random()); - logger.debug("--> joining [{}] nodes, with repetition a total of [{}]", finalJoins, nodesToJoin.size()); - for (DiscoveryNode node : nodesToJoin) { - pendingJoins.add(joinNodeAsync(node)); - } - logger.debug("--> waiting for master election to with no exception"); - electionFuture.get(); - - logger.debug("--> waiting on all joins to be processed"); - for (SimpleFuture future : pendingJoins) { - logger.debug("waiting on {}", future); - future.get(); // throw any exception - } - - logger.debug("--> testing accumulation stopped"); - nodeJoinController.startElectionContext(); - nodeJoinController.stopElectionContext("test"); - - } - - public void testMasterElectionTimeout() throws InterruptedException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - final int requiredJoins = 1 + randomInt(5); - logger.debug("--> using requiredJoins [{}]", requiredJoins); - // initial (failing) joins shouldn't count - for (int i = randomInt(5); i > 0; i--) { - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - - nodeJoinController.startElectionContext(); - final int initialJoins = randomIntBetween(0, requiredJoins - 1); - final ArrayList pendingJoins = new ArrayList<>(); - ArrayList nodesToJoin = new ArrayList<>(); - for (int i = 0; i < initialJoins; i++) { - DiscoveryNode node = newNode(nodeId++); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - shuffle(nodesToJoin, random()); - logger.debug("--> joining [{}] nodes, with repetition a total of [{}]", initialJoins, nodesToJoin.size()); - for (DiscoveryNode node : nodesToJoin) { - pendingJoins.add(joinNodeAsync(node)); - } - - final AtomicReference failure = new AtomicReference<>(); - final CountDownLatch latch = new CountDownLatch(1); - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueMillis(1), new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), - equalTo(true)); - latch.countDown(); - } - - @Override - public void onFailure(Throwable t) { - failure.set(t); - latch.countDown(); - } - }); - latch.await(); - logger.debug("--> verifying election timed out"); - assertThat(failure.get(), instanceOf(NotMasterException.class)); - - logger.debug("--> verifying all joins are failed"); - for (SimpleFuture future : pendingJoins) { - logger.debug("waiting on {}", future); - try { - future.get(); // throw any exception - fail("failed to fail node join [" + future + "]"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - } - - public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException { - ClusterState state = initialState(true); - final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); - final DiscoveryNode other_node = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), - emptyMap(), emptySet(), Version.CURRENT); - nodesBuilder.add(other_node); - setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build()); - - state = discoveryState(masterService); - joinNode(other_node); - assertTrue("failed to publish a new state upon existing join", discoveryState(masterService) != state); - } - - public void testNormalConcurrentJoins() throws InterruptedException { - setupMasterServiceAndNodeJoinController(initialState(true)); - Thread[] threads = new Thread[3 + randomInt(5)]; - ArrayList nodes = new ArrayList<>(); - nodes.add(discoveryState(masterService).nodes().getLocalNode()); - final CyclicBarrier barrier = new CyclicBarrier(threads.length); - final List backgroundExceptions = new CopyOnWriteArrayList<>(); - for (int i = 0; i < threads.length; i++) { - final DiscoveryNode node = newNode(i); - final int iterations = rarely() ? randomIntBetween(1, 4) : 1; - nodes.add(node); - threads[i] = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error in join thread", e); - backgroundExceptions.add(e); - } - - @Override - protected void doRun() throws Exception { - barrier.await(); - for (int i = 0; i < iterations; i++) { - logger.debug("{} joining", node); - joinNode(node); - } - } - }, "t_" + i); - threads[i].start(); - } - - logger.info("--> waiting for joins to complete"); - for (Thread thread : threads) { - thread.join(); - } - - assertNodesInCurrentState(nodes); - } - - public void testElectionWithConcurrentJoins() throws InterruptedException, BrokenBarrierException { - setupMasterServiceAndNodeJoinController(initialState(false)); - - nodeJoinController.startElectionContext(); - - Thread[] threads = new Thread[3 + randomInt(5)]; - final int requiredJoins = randomInt(threads.length); - ArrayList nodes = new ArrayList<>(); - nodes.add(discoveryState(masterService).nodes().getLocalNode()); - final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1); - final List backgroundExceptions = new CopyOnWriteArrayList<>(); - for (int i = 0; i < threads.length; i++) { - final DiscoveryNode node = newNode(i, true); - final int iterations = rarely() ? randomIntBetween(1, 4) : 1; - nodes.add(node); - threads[i] = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error in join thread", e); - backgroundExceptions.add(e); - } - - @Override - protected void doRun() throws Exception { - barrier.await(); - for (int i = 0; i < iterations; i++) { - logger.debug("{} joining", node); - joinNode(node); - } - } - }, "t_" + i); - threads[i].start(); - } - - barrier.await(); - logger.info("--> waiting to be elected as master (required joins [{}])", requiredJoins); - final AtomicReference failure = new AtomicReference<>(); - final CountDownLatch latch = new CountDownLatch(1); - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), - equalTo(true)); - latch.countDown(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("unexpected error while waiting to be elected as master", t); - failure.set(t); - latch.countDown(); - } - }); - latch.await(); - ExceptionsHelper.reThrowIfNotNull(failure.get()); - - - logger.info("--> waiting for joins to complete"); - for (Thread thread : threads) { - thread.join(); - } - - assertNodesInCurrentState(nodes); - } - - public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState state = discoveryState(masterService); - final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); - final DiscoveryNode other_node = new DiscoveryNode("other_node", existing.getAddress(), emptyMap(), emptySet(), Version.CURRENT); - - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); - } - - public void testRejectingJoinWithSameIdButDifferentNode() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState state = discoveryState(masterService); - final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); - final DiscoveryNode other_node = new DiscoveryNode( - randomBoolean() ? existing.getName() : "other_name", - existing.getId(), - randomBoolean() ? existing.getAddress() : buildNewFakeTransportAddress(), - randomBoolean() ? existing.getAttributes() : Collections.singletonMap("attr", "other"), - randomBoolean() ? existing.getRoles() : new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), - existing.getVersion()); - - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); - } - - public void testRejectingRestartedNodeJoinsBeforeProcessingNodeLeft() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState state = discoveryState(masterService); - final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); - joinNode(existing); // OK - - final DiscoveryNode other_node = new DiscoveryNode(existing.getId(), existing.getAddress(), existing.getAttributes(), - existing.getRoles(), Version.CURRENT); - - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); - } - - public void testRejectingJoinWithIncompatibleVersion() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - final Version badVersion; - if (randomBoolean()) { - badVersion = getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion()); - } else { - badVersion = randomFrom(allVersions().stream().filter(v -> v.major < Version.CURRENT.major).collect(Collectors.toList())); - } - final DiscoveryNode badNode = new DiscoveryNode("badNode", buildNewFakeTransportAddress(), emptyMap(), - new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), badVersion); - - final Version goodVersion = - randomFrom(allVersions().stream().filter(v -> v.major >= Version.CURRENT.major).collect(Collectors.toList())); - final DiscoveryNode goodNode = new DiscoveryNode("goodNode", buildNewFakeTransportAddress(), emptyMap(), - new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), goodVersion); - - CountDownLatch latch = new CountDownLatch(1); - // block cluster state - masterService.submitStateUpdateTask("test", new ClusterStateUpdateTask(Priority.IMMEDIATE) { - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - latch.await(); - return currentState; - } - - @Override - public void onFailure(String source, Exception e) { - throw new AssertionError(e); - } - }); - - final SimpleFuture badJoin; - final SimpleFuture goodJoin; - if (randomBoolean()) { - badJoin = joinNodeAsync(badNode); - goodJoin = joinNodeAsync(goodNode); - } else { - goodJoin = joinNodeAsync(goodNode); - badJoin = joinNodeAsync(badNode); - } - assert goodJoin.isDone() == false; - assert badJoin.isDone() == false; - latch.countDown(); - goodJoin.get(); - ExecutionException e = expectThrows(ExecutionException.class, badJoin::get); - assertThat(e.getCause(), instanceOf(IllegalStateException.class)); - assertThat(e.getCause().getMessage(), allOf(containsString("node version"), containsString("not supported"))); - } - - public void testRejectingJoinWithIncompatibleVersionWithUnrecoveredState() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState.Builder builder = ClusterState.builder(discoveryState(masterService)); - builder.blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)); - setState(masterService, builder.build()); - final Version badVersion = getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion()); - final DiscoveryNode badNode = new DiscoveryNode("badNode", buildNewFakeTransportAddress(), emptyMap(), - new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), badVersion); - - final Version goodVersion = randomFrom(randomCompatibleVersion(random(), Version.CURRENT)); - final DiscoveryNode goodNode = new DiscoveryNode("goodNode", buildNewFakeTransportAddress(), emptyMap(), - new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), goodVersion); - - CountDownLatch latch = new CountDownLatch(1); - // block cluster state - masterService.submitStateUpdateTask("test", new ClusterStateUpdateTask(Priority.IMMEDIATE) { - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - latch.await(); - return currentState; - } - - @Override - public void onFailure(String source, Exception e) { - throw new AssertionError(e); - } - }); - - final SimpleFuture badJoin; - final SimpleFuture goodJoin; - if (randomBoolean()) { - badJoin = joinNodeAsync(badNode); - goodJoin = joinNodeAsync(goodNode); - } else { - goodJoin = joinNodeAsync(goodNode); - badJoin = joinNodeAsync(badNode); - } - assert goodJoin.isDone() == false; - assert badJoin.isDone() == false; - latch.countDown(); - goodJoin.get(); - ExecutionException e = expectThrows(ExecutionException.class, badJoin::get); - assertThat(e.getCause(), instanceOf(IllegalStateException.class)); - assertThat(e.getCause().getMessage(), allOf(containsString("node version"), containsString("not supported"))); - } - - /** - * Tests tha node can become a master, even though the last cluster state it knows contains - * nodes that conflict with the joins it got and needs to become a master - */ - public void testElectionBasedOnConflictingNodes() throws InterruptedException, ExecutionException { - ClusterState initialState = initialState(true); - final DiscoveryNode masterNode = initialState.nodes().getLocalNode(); - final DiscoveryNode otherNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), - EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); - // simulate master going down with stale nodes in it's cluster state (for example when min master nodes is set to 2) - // also add some shards to that node - DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(initialState.nodes()); - discoBuilder.masterNodeId(null); - discoBuilder.add(otherNode); - ClusterState.Builder stateBuilder = ClusterState.builder(initialState).nodes(discoBuilder); - if (randomBoolean()) { - IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder() - .put(SETTING_VERSION_CREATED, Version.CURRENT) - .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 1) - .put(SETTING_CREATION_DATE, System.currentTimeMillis())).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 = randomBoolean() ? masterNode : otherNode; - final DiscoveryNode replicaNode = primaryNode.equals(masterNode) ? otherNode : masterNode; - final boolean primaryStarted = randomBoolean(); - indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting("test", 0, primaryNode.getId(), null, true, - primaryStarted ? ShardRoutingState.STARTED : ShardRoutingState.INITIALIZING, - primaryStarted ? null : new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "getting there"))); - if (primaryStarted) { - boolean replicaStared = randomBoolean(); - indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting("test", 0, replicaNode.getId(), null, false, - replicaStared ? ShardRoutingState.STARTED : ShardRoutingState.INITIALIZING, - replicaStared ? null : new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "getting there"))); - } else { - indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting("test", 0, null, null, false, - ShardRoutingState.UNASSIGNED, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "life sucks"))); - } - 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()); - } - - setupMasterServiceAndNodeJoinController(stateBuilder.build()); - - // conflict on node id or address - final DiscoveryNode conflictingNode = randomBoolean() ? - new DiscoveryNode(otherNode.getId(), randomBoolean() ? otherNode.getAddress() : buildNewFakeTransportAddress(), - otherNode.getAttributes(), otherNode.getRoles(), Version.CURRENT) : - new DiscoveryNode("conflicting_address_node", otherNode.getAddress(), otherNode.getAttributes(), otherNode.getRoles(), - Version.CURRENT); - - nodeJoinController.startElectionContext(); - final SimpleFuture joinFuture = joinNodeAsync(conflictingNode); - final CountDownLatch elected = new CountDownLatch(1); - nodeJoinController.waitToBeElectedAsMaster(1, TimeValue.timeValueHours(5), new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - elected.countDown(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("failed to be elected as master", t); - throw new AssertionError("failed to be elected as master", t); - } - }); - - elected.await(); - - joinFuture.get(); // throw any exception - - final ClusterState finalState = discoveryState(masterService); - final DiscoveryNodes finalNodes = finalState.nodes(); - assertTrue(finalNodes.isLocalNodeElectedMaster()); - assertThat(finalNodes.getLocalNode(), equalTo(masterNode)); - assertThat(finalNodes.getSize(), equalTo(2)); - assertThat(finalNodes.get(conflictingNode.getId()), equalTo(conflictingNode)); - List activeShardsOnRestartedNode = - StreamSupport.stream(finalState.getRoutingNodes().node(conflictingNode.getId()).spliterator(), false) - .filter(ShardRouting::active).collect(Collectors.toList()); - assertThat(activeShardsOnRestartedNode, empty()); - } - - - private void addNodes(int count) { - ClusterState state = initialState(true); - final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); - for (int i = 0;i< count;i++) { - final DiscoveryNode node = new DiscoveryNode("node_" + state.nodes().getSize() + i, buildNewFakeTransportAddress(), - emptyMap(), new HashSet<>(randomSubsetOf(Arrays.asList(DiscoveryNode.Role.values()))), Version.CURRENT); - nodesBuilder.add(node); - } - setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build()); - } - - protected void assertNodesInCurrentState(List expectedNodes) { - final ClusterState state = discoveryState(masterService); - logger.info("assert for [{}] in:\n{}", expectedNodes, state); - DiscoveryNodes discoveryNodes = state.nodes(); - for (DiscoveryNode node : expectedNodes) { - assertThat("missing " + node + "\n" + discoveryNodes, discoveryNodes.get(node.getId()), equalTo(node)); - } - assertThat(discoveryNodes.getSize(), equalTo(expectedNodes.size())); - } - - static class SimpleFuture extends BaseFuture { - final String description; - - SimpleFuture(String description) { - this.description = description; - } - - public void markAsDone() { - set(null); - } - - public void markAsFailed(Throwable t) { - setException(t); - } - - @Override - public String toString() { - return "future [" + description + "]"; - } - } - - static final AtomicInteger joinId = new AtomicInteger(); - - private SimpleFuture joinNodeAsync(final DiscoveryNode node) throws InterruptedException { - final SimpleFuture future = new SimpleFuture("join of " + node + " (id [" + joinId.incrementAndGet() + "]"); - logger.debug("starting {}", future); - // clone the node before submitting to simulate an incoming join, which is guaranteed to have a new - // disco node object serialized off the network - nodeJoinController.handleJoinRequest(cloneNode(node), new MembershipAction.JoinCallback() { - @Override - public void onSuccess() { - logger.debug("{} completed", future); - future.markAsDone(); - } - - @Override - public void onFailure(Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e); - future.markAsFailed(e); - } - }); - return future; - } - - /** - * creates an object clone of node, so it will be a different object instance - */ - private DiscoveryNode cloneNode(DiscoveryNode node) { - return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node.getHostAddress(), - node.getAddress(), node.getAttributes(), node.getRoles(), node.getVersion()); - } - - private void joinNode(final DiscoveryNode node) throws InterruptedException, ExecutionException { - joinNodeAsync(node).get(); - } - - protected DiscoveryNode newNode(int i) { - return newNode(i, randomBoolean()); - } - - protected DiscoveryNode newNode(int i, boolean master) { - Set roles = new HashSet<>(); - if (master) { - roles.add(DiscoveryNode.Role.MASTER); - } - final String prefix = master ? "master_" : "data_"; - return new DiscoveryNode(prefix + i, i + "", buildNewFakeTransportAddress(), emptyMap(), roles, Version.CURRENT); - } -} diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueueTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueueTests.java deleted file mode 100644 index b1faaba576ba5..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueueTests.java +++ /dev/null @@ -1,268 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.zen.PendingClusterStatesQueue; -import org.elasticsearch.discovery.zen.PendingClusterStatesQueue.ClusterStateContext; -import org.elasticsearch.test.ESTestCase; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; -import static org.hamcrest.Matchers.sameInstance; - -public class PendingClusterStatesQueueTests extends ESTestCase { - - public void testSelectNextStateToProcess_empty() { - PendingClusterStatesQueue queue = new PendingClusterStatesQueue(logger, randomIntBetween(1, 200)); - assertThat(queue.getNextClusterStateToProcess(), nullValue()); - } - - public void testDroppingStatesAtCapacity() { - List states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4"); - Collections.shuffle(states, random()); - // insert half of the states - final int numberOfStateToDrop = states.size() / 2; - List stateToDrop = states.subList(0, numberOfStateToDrop); - final int queueSize = states.size() - numberOfStateToDrop; - PendingClusterStatesQueue queue = createQueueWithStates(stateToDrop, queueSize); - List committedContexts = randomCommitStates(queue); - for (ClusterState state : states.subList(numberOfStateToDrop, states.size())) { - queue.addPending(state); - } - - assertThat(queue.pendingClusterStates().length, equalTo(queueSize)); - // check all committed states got a failure due to the drop - for (ClusterStateContext context : committedContexts) { - assertThat(((MockListener) context.listener).failure, notNullValue()); - } - - // all states that should have dropped are indeed dropped. - for (ClusterState state : stateToDrop) { - assertThat(queue.findState(state.stateUUID()), nullValue()); - } - - } - - public void testSimpleQueueSameMaster() { - final int numUpdates = scaledRandomIntBetween(50, 100); - List states = randomStates(numUpdates, "master"); - Collections.shuffle(states, random()); - PendingClusterStatesQueue queue; - queue = createQueueWithStates(states); - - // no state is committed yet - assertThat(queue.getNextClusterStateToProcess(), nullValue()); - - ClusterState highestCommitted = null; - for (ClusterStateContext context : randomCommitStates(queue)) { - if (highestCommitted == null || context.state.supersedes(highestCommitted)) { - highestCommitted = context.state; - } - } - - assertThat(queue.getNextClusterStateToProcess(), sameInstance(highestCommitted)); - - queue.markAsProcessed(highestCommitted); - - // now there is nothing more to process - assertThat(queue.getNextClusterStateToProcess(), nullValue()); - } - - public void testProcessedStateCleansStatesFromOtherMasters() { - List states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - List committedContexts = randomCommitStates(queue); - ClusterState randomCommitted = randomFrom(committedContexts).state; - queue.markAsProcessed(randomCommitted); - final String processedMaster = randomCommitted.nodes().getMasterNodeId(); - - // now check that queue doesn't contain anything pending from another master - for (ClusterStateContext context : queue.pendingStates) { - final String pendingMaster = context.state.nodes().getMasterNodeId(); - assertThat("found a cluster state from [" + pendingMaster - + "], after a state from [" + processedMaster + "] was processed", - pendingMaster, equalTo(processedMaster)); - } - // and check all committed contexts from another master were failed - for (ClusterStateContext context : committedContexts) { - if (context.state.nodes().getMasterNodeId().equals(processedMaster) == false) { - assertThat(((MockListener) context.listener).failure, notNullValue()); - } - } - } - - public void testFailedStateCleansSupersededStatesOnly() { - List states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - List committedContexts = randomCommitStates(queue); - ClusterState toFail = randomFrom(committedContexts).state; - queue.markAsFailed(toFail, new ElasticsearchException("boo!")); - final Map committedContextsById = new HashMap<>(); - for (ClusterStateContext context : committedContexts) { - committedContextsById.put(context.stateUUID(), context); - } - - // now check that queue doesn't contain superseded states - for (ClusterStateContext context : queue.pendingStates) { - if (context.committed()) { - assertFalse("found a committed cluster state, which is superseded by a failed state.\nFound:" + - context.state + "\nfailed:" + toFail, - toFail.supersedes(context.state)); - } - } - // check no state has been erroneously removed - for (ClusterState state : states) { - ClusterStateContext pendingContext = queue.findState(state.stateUUID()); - if (pendingContext != null) { - continue; - } - if (state.equals(toFail)) { - continue; - } - assertThat("non-committed states should never be removed", committedContextsById, hasKey(state.stateUUID())); - final ClusterStateContext context = committedContextsById.get(state.stateUUID()); - assertThat("removed state is not superseded by failed state. \nRemoved state:" + context + "\nfailed: " + toFail, - toFail.supersedes(context.state), equalTo(true)); - assertThat("removed state was failed with wrong exception", ((MockListener) context.listener).failure, notNullValue()); - assertThat("removed state was failed with wrong exception", ((MockListener) context.listener).failure.getMessage(), - containsString("boo")); - } - } - - public void testFailAllAndClear() { - List states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - List committedContexts = randomCommitStates(queue); - queue.failAllStatesAndClear(new ElasticsearchException("boo!")); - assertThat(queue.pendingStates, empty()); - assertThat(queue.getNextClusterStateToProcess(), nullValue()); - for (ClusterStateContext context : committedContexts) { - assertThat("state was failed with wrong exception", ((MockListener) context.listener).failure, notNullValue()); - assertThat("state was failed with wrong exception", ((MockListener) context.listener).failure.getMessage(), - containsString("boo")); - } - } - - public void testQueueStats() { - List states = randomStates(scaledRandomIntBetween(10, 100), "master"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - assertThat(queue.stats().getTotal(), equalTo(states.size())); - assertThat(queue.stats().getPending(), equalTo(states.size())); - assertThat(queue.stats().getCommitted(), equalTo(0)); - - List committedContexts = randomCommitStates(queue); - assertThat(queue.stats().getTotal(), equalTo(states.size())); - assertThat(queue.stats().getPending(), equalTo(states.size() - committedContexts.size())); - assertThat(queue.stats().getCommitted(), equalTo(committedContexts.size())); - - ClusterState highestCommitted = null; - for (ClusterStateContext context : committedContexts) { - if (highestCommitted == null || context.state.supersedes(highestCommitted)) { - highestCommitted = context.state; - } - } - assert highestCommitted != null; - - queue.markAsProcessed(highestCommitted); - assertThat((long)queue.stats().getTotal(), equalTo(states.size() - (1 + highestCommitted.version()))); - assertThat((long)queue.stats().getPending(), equalTo(states.size() - (1 + highestCommitted.version()))); - assertThat(queue.stats().getCommitted(), equalTo(0)); - } - - protected List randomCommitStates(PendingClusterStatesQueue queue) { - List committedContexts = new ArrayList<>(); - for (int iter = randomInt(queue.pendingStates.size() - 1); iter >= 0; iter--) { - ClusterState state = queue.markAsCommitted(randomFrom(queue.pendingStates).stateUUID(), new MockListener()); - if (state != null) { - // null cluster state means we committed twice - committedContexts.add(queue.findState(state.stateUUID())); - } - } - return committedContexts; - } - - PendingClusterStatesQueue createQueueWithStates(List states) { - return createQueueWithStates(states, states.size() * 2); // we don't care about limits (there are dedicated tests for that) - } - - PendingClusterStatesQueue createQueueWithStates(List states, int maxQueueSize) { - PendingClusterStatesQueue queue; - queue = new PendingClusterStatesQueue(logger, maxQueueSize); - for (ClusterState state : states) { - queue.addPending(state); - } - return queue; - } - - List randomStates(int count, String... masters) { - ArrayList states = new ArrayList<>(count); - ClusterState[] lastClusterStatePerMaster = new ClusterState[masters.length]; - for (; count > 0; count--) { - int masterIndex = randomInt(masters.length - 1); - ClusterState state = lastClusterStatePerMaster[masterIndex]; - if (state == null) { - state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(DiscoveryNodes.builder() - .add(new DiscoveryNode(masters[masterIndex], buildNewFakeTransportAddress(), - emptyMap(), emptySet(),Version.CURRENT)).masterNodeId(masters[masterIndex]).build() - ).build(); - } else { - state = ClusterState.builder(state).incrementVersion().build(); - } - states.add(state); - lastClusterStatePerMaster[masterIndex] = state; - } - return states; - } - - static class MockListener implements PendingClusterStatesQueue.StateProcessedListener { - volatile boolean processed; - volatile Throwable failure; - - @Override - public void onNewClusterStateProcessed() { - processed = true; - } - - @Override - public void onNewClusterStateFailed(Exception e) { - failure = e; - } - } - -} diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java deleted file mode 100644 index 3a8880da68d31..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java +++ /dev/null @@ -1,962 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterModule; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; -import org.elasticsearch.cluster.Diff; -import org.elasticsearch.cluster.block.ClusterBlocks; -import org.elasticsearch.cluster.coordination.PublishClusterStateStats; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; -import org.elasticsearch.node.Node; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.BytesTransportRequest; -import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportConnectionListener; -import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.transport.TransportSettings; -import org.junit.After; -import org.junit.Before; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.emptyIterable; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; - -@TestLogging("org.elasticsearch.discovery.zen.publish:TRACE") -public class PublishClusterStateActionTests extends ESTestCase { - - private static final ClusterName CLUSTER_NAME = ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY); - - protected ThreadPool threadPool; - protected Map nodes = new HashMap<>(); - - public static class MockNode implements PublishClusterStateAction.IncomingClusterStateListener { - public final DiscoveryNode discoveryNode; - public final MockTransportService service; - public MockPublishAction action; - public final ClusterStateListener listener; - private final PendingClusterStatesQueue pendingStatesQueue; - - public volatile ClusterState clusterState; - - private final Logger logger; - - public MockNode(DiscoveryNode discoveryNode, MockTransportService service, - @Nullable ClusterStateListener listener, Logger logger) { - this.discoveryNode = discoveryNode; - this.service = service; - this.listener = listener; - this.logger = logger; - this.clusterState = ClusterState.builder(CLUSTER_NAME).nodes(DiscoveryNodes.builder() - .add(discoveryNode).localNodeId(discoveryNode.getId()).build()).build(); - this.pendingStatesQueue = new PendingClusterStatesQueue(logger, 25); - } - - public MockNode setAsMaster() { - this.clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .masterNodeId(discoveryNode.getId())).build(); - return this; - } - - public MockNode resetMasterId() { - this.clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .masterNodeId(null)).build(); - return this; - } - - - public void connectTo(DiscoveryNode node) { - service.connectToNode(node); - } - - @Override - public void onIncomingClusterState(ClusterState incomingState) { - ZenDiscovery.validateIncomingState(logger, incomingState, clusterState); - pendingStatesQueue.addPending(incomingState); - } - - public void onClusterStateCommitted(String stateUUID, ActionListener processedListener) { - final ClusterState state = pendingStatesQueue.markAsCommitted(stateUUID, - new PendingClusterStatesQueue.StateProcessedListener() { - @Override - public void onNewClusterStateProcessed() { - processedListener.onResponse(null); - } - - @Override - public void onNewClusterStateFailed(Exception e) { - processedListener.onFailure(e); - } - }); - if (state != null) { - ClusterState newClusterState = pendingStatesQueue.getNextClusterStateToProcess(); - logger.debug("[{}] received version [{}], uuid [{}]", - discoveryNode.getName(), newClusterState.version(), newClusterState.stateUUID()); - if (listener != null) { - ClusterChangedEvent event = new ClusterChangedEvent("", newClusterState, clusterState); - listener.clusterChanged(event); - } - if (clusterState.nodes().getMasterNode() == null || newClusterState.supersedes(clusterState)) { - clusterState = newClusterState; - } - pendingStatesQueue.markAsProcessed(newClusterState); - } - } - - public DiscoveryNodes nodes() { - return clusterState.nodes(); - } - - } - - public MockNode createMockNode(final String name) throws Exception { - return createMockNode(name, Settings.EMPTY, null); - } - - public MockNode createMockNode(String name, final Settings basSettings, @Nullable ClusterStateListener listener) throws Exception { - return createMockNode(name, basSettings, listener, threadPool, logger, nodes); - } - - public static MockNode createMockNode(String name, final Settings basSettings, @Nullable ClusterStateListener listener, - ThreadPool threadPool, Logger logger, Map nodes) throws Exception { - final Settings settings = Settings.builder() - .put("name", name) - .put(TransportSettings.TRACE_LOG_INCLUDE_SETTING.getKey(), "").put( - TransportSettings.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING") - .put(basSettings) - .build(); - - MockTransportService service = buildTransportService(settings, threadPool); - DiscoveryNode discoveryNode = service.getLocalDiscoNode(); - MockNode node = new MockNode(discoveryNode, service, listener, logger); - node.action = buildPublishClusterStateAction(settings, service, node); - final CountDownLatch latch = new CountDownLatch(nodes.size() * 2); - TransportConnectionListener waitForConnection = new TransportConnectionListener() { - @Override - public void onNodeConnected(DiscoveryNode node) { - latch.countDown(); - } - - @Override - public void onNodeDisconnected(DiscoveryNode node) { - fail("disconnect should not be called " + node); - } - }; - node.service.addConnectionListener(waitForConnection); - for (MockNode curNode : nodes.values()) { - curNode.service.addConnectionListener(waitForConnection); - curNode.connectTo(node.discoveryNode); - node.connectTo(curNode.discoveryNode); - } - assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true)); - for (MockNode curNode : nodes.values()) { - curNode.service.removeConnectionListener(waitForConnection); - } - node.service.removeConnectionListener(waitForConnection); - if (nodes.put(name, node) != null) { - fail("Node with the name " + name + " already exist"); - } - return node; - } - - public MockTransportService service(String name) { - MockNode node = nodes.get(name); - if (node != null) { - return node.service; - } - return null; - } - - public PublishClusterStateAction action(String name) { - MockNode node = nodes.get(name); - if (node != null) { - return node.action; - } - return null; - } - - @Override - @Before - public void setUp() throws Exception { - super.setUp(); - threadPool = new TestThreadPool(getClass().getName()); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - for (MockNode curNode : nodes.values()) { - curNode.service.close(); - } - terminate(threadPool); - } - - private static MockTransportService buildTransportService(Settings settings, ThreadPool threadPool) { - MockTransportService transportService = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); - transportService.start(); - transportService.acceptIncomingRequests(); - return transportService; - } - - private static MockPublishAction buildPublishClusterStateAction( - Settings settings, - MockTransportService transportService, - PublishClusterStateAction.IncomingClusterStateListener listener - ) { - DiscoverySettings discoverySettings = - new DiscoverySettings(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); - NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(ClusterModule.getNamedWriteables()); - return new MockPublishAction( - transportService, - namedWriteableRegistry, - listener, - discoverySettings); - } - - public void testSimpleClusterStatePublishing() throws Exception { - MockNode nodeA = createMockNode("nodeA").setAsMaster(); - MockNode nodeB = createMockNode("nodeB"); - - // Initial cluster state - ClusterState clusterState = nodeA.clusterState; - - // cluster state update - add nodeB - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(clusterState.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder() - .addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - assertThat(nodeB.clusterState.blocks().global().size(), equalTo(1)); - - // cluster state update - remove block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - assertTrue(nodeB.clusterState.wasReadFromDiff()); - - // Adding new node - this node should get full cluster state while nodeB should still be getting diffs - - MockNode nodeC = createMockNode("nodeC"); - - // cluster state update 3 - register node C - previousClusterState = clusterState; - discoveryNodes = DiscoveryNodes.builder(discoveryNodes).add(nodeC.discoveryNode).build(); - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - // First state - assertSameStateFromFull(nodeC.clusterState, clusterState); - - // cluster state update 4 - update settings - previousClusterState = clusterState; - MetaData metaData = MetaData.builder(clusterState.metaData()) - .transientSettings(Settings.builder().put("foo", "bar").build()).build(); - clusterState = ClusterState.builder(clusterState).metaData(metaData).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - assertThat(nodeB.clusterState.blocks().global().size(), equalTo(0)); - assertSameStateFromDiff(nodeC.clusterState, clusterState); - assertThat(nodeC.clusterState.blocks().global().size(), equalTo(0)); - - // cluster state update - skipping one version change - should request full cluster state - previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); - clusterState = ClusterState.builder(clusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - assertSameStateFromFull(nodeC.clusterState, clusterState); - assertFalse(nodeC.clusterState.wasReadFromDiff()); - - // node A steps down from being master - nodeA.resetMasterId(); - nodeB.resetMasterId(); - nodeC.resetMasterId(); - - // node B becomes the master and sends a version of the cluster state that goes back - discoveryNodes = DiscoveryNodes.builder(discoveryNodes) - .add(nodeA.discoveryNode) - .add(nodeB.discoveryNode) - .add(nodeC.discoveryNode) - .masterNodeId(nodeB.discoveryNode.getId()) - .localNodeId(nodeB.discoveryNode.getId()) - .build(); - previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeB.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeA.clusterState, clusterState); - assertSameStateFromFull(nodeC.clusterState, clusterState); - } - - public void testUnexpectedDiffPublishing() throws Exception { - MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, event -> { - fail("Shouldn't send cluster state to myself"); - }).setAsMaster(); - - MockNode nodeB = createMockNode("nodeB"); - - // Initial cluster state with both states - the second node still shouldn't - // get diff even though it's present in the previous cluster state - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(nodeA.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = ClusterState.builder(CLUSTER_NAME).nodes(discoveryNodes).build(); - ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder() - .addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - } - - public void testDisablingDiffPublishing() throws Exception { - Settings noDiffPublishingSettings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), false).build(); - - MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - fail("Shouldn't send cluster state to myself"); - } - }); - - MockNode nodeB = createMockNode("nodeB", noDiffPublishingSettings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - assertFalse(event.state().wasReadFromDiff()); - } - }); - - // Initial cluster state - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() - .add(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.getId()).masterNodeId(nodeA.discoveryNode.getId()).build(); - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).nodes(discoveryNodes).build(); - - // cluster state update - add nodeB - discoveryNodes = DiscoveryNodes.builder(discoveryNodes).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder() - .addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - assertWarnings( - "[discovery.zen.publish_diff.enable] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version."); - } - - - /** - * Test not waiting on publishing works correctly (i.e., publishing times out) - */ - public void testSimultaneousClusterStatePublishing() throws Exception { - int numberOfNodes = randomIntBetween(2, 10); - int numberOfIterations = scaledRandomIntBetween(5, 50); - Settings settings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), randomBoolean()).build(); - MockNode master = createMockNode("node0", settings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - assertProperMetaDataForVersion(event.state().metaData(), event.state().version()); - } - }).setAsMaster(); - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(master.nodes()); - for (int i = 1; i < numberOfNodes; i++) { - final String name = "node" + i; - final MockNode node = createMockNode(name, settings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - assertProperMetaDataForVersion(event.state().metaData(), event.state().version()); - } - }); - discoveryNodesBuilder.add(node.discoveryNode); - } - - AssertingAckListener[] listeners = new AssertingAckListener[numberOfIterations]; - DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); - MetaData metaData = MetaData.EMPTY_META_DATA; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).metaData(metaData).build(); - ClusterState previousState; - for (int i = 0; i < numberOfIterations; i++) { - previousState = clusterState; - metaData = buildMetaDataForVersion(metaData, i + 1); - clusterState = ClusterState.builder(clusterState).incrementVersion().metaData(metaData).nodes(discoveryNodes).build(); - listeners[i] = publishState(master.action, clusterState, previousState); - } - - for (int i = 0; i < numberOfIterations; i++) { - listeners[i].await(1, TimeUnit.SECONDS); - } - - // set the master cs - master.clusterState = clusterState; - - for (MockNode node : nodes.values()) { - assertSameState(node.clusterState, clusterState); - assertThat(node.clusterState.nodes().getLocalNode(), equalTo(node.discoveryNode)); - } - - assertWarnings( - "[discovery.zen.publish_diff.enable] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version."); - } - - public void testSerializationFailureDuringDiffPublishing() throws Exception { - MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - fail("Shouldn't send cluster state to myself"); - } - }).setAsMaster(); - - MockNode nodeB = createMockNode("nodeB"); - - // Initial cluster state with both states - the second node still shouldn't get - // diff even though it's present in the previous cluster state - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(nodeA.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = ClusterState.builder(CLUSTER_NAME).nodes(discoveryNodes).build(); - ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder() - .addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); - - ClusterState unserializableClusterState = new ClusterState(clusterState.version(), clusterState.stateUUID(), - clusterState) { - @Override - public Diff diff(ClusterState previousState) { - return new Diff() { - @Override - public ClusterState apply(ClusterState part) { - fail("this diff shouldn't be applied"); - return part; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - throw new IOException("Simulated failure of diff serialization"); - } - }; - } - }; - try { - publishStateAndWait(nodeA.action, unserializableClusterState, previousClusterState); - fail("cluster state published despite of diff errors"); - } catch (FailedToCommitClusterStateException e) { - assertThat(e.getCause(), notNullValue()); - assertThat(e.getCause().getMessage(), containsString("failed to serialize")); - } - } - - - public void testFailToPublishWithLessThanMinMasterNodes() throws Exception { - final int masterNodes = randomIntBetween(1, 10); - - MockNode master = createMockNode("master"); - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder().add(master.discoveryNode); - for (int i = 1; i < masterNodes; i++) { - discoveryNodesBuilder.add(createMockNode("node" + i).discoveryNode); - } - final int dataNodes = randomIntBetween(0, 5); - final Settings dataSettings = Settings.builder().put(Node.NODE_MASTER_SETTING.getKey(), false).build(); - for (int i = 0; i < dataNodes; i++) { - discoveryNodesBuilder.add(createMockNode("data_" + i, dataSettings, null).discoveryNode); - } - discoveryNodesBuilder.localNodeId(master.discoveryNode.getId()).masterNodeId(master.discoveryNode.getId()); - DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); - MetaData metaData = MetaData.EMPTY_META_DATA; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).metaData(metaData).nodes(discoveryNodes).build(); - ClusterState previousState = master.clusterState; - try { - publishState(master.action, clusterState, previousState, masterNodes + randomIntBetween(1, 5)); - fail("cluster state publishing didn't fail despite of not having enough nodes"); - } catch (FailedToCommitClusterStateException expected) { - logger.debug("failed to publish as expected", expected); - } - } - - public void testPublishingWithSendingErrors() throws Exception { - int goodNodes = randomIntBetween(2, 5); - int errorNodes = randomIntBetween(1, 5); - int timeOutNodes = randomBoolean() ? 0 : randomIntBetween(1, 5); // adding timeout nodes will force timeout errors - final int numberOfMasterNodes = goodNodes + errorNodes + timeOutNodes + 1; // master - final boolean expectingToCommit = randomBoolean(); - Settings.Builder settings = Settings.builder(); - // make sure we have a reasonable timeout if we expect to timeout, o.w. one that will make the test "hang" - settings.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), expectingToCommit == false && timeOutNodes > 0 ? "100ms" : "1h") - .put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "5ms"); // test is about committing - - MockNode master = createMockNode("master", settings.build(), null); - - // randomize things a bit - int[] nodeTypes = new int[goodNodes + errorNodes + timeOutNodes]; - for (int i = 0; i < goodNodes; i++) { - nodeTypes[i] = 0; - } - for (int i = goodNodes; i < goodNodes + errorNodes; i++) { - nodeTypes[i] = 1; - } - for (int i = goodNodes + errorNodes; i < nodeTypes.length; i++) { - nodeTypes[i] = 2; - } - Collections.shuffle(Arrays.asList(nodeTypes), random()); - - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder().add(master.discoveryNode); - for (int i = 0; i < nodeTypes.length; i++) { - final MockNode mockNode = createMockNode("node" + i); - discoveryNodesBuilder.add(mockNode.discoveryNode); - switch (nodeTypes[i]) { - case 1: - mockNode.action.errorOnSend.set(true); - break; - case 2: - mockNode.action.timeoutOnSend.set(true); - break; - } - } - final int dataNodes = randomIntBetween(0, 3); // data nodes don't matter - for (int i = 0; i < dataNodes; i++) { - final MockNode mockNode = createMockNode("data_" + i, - Settings.builder().put(Node.NODE_MASTER_SETTING.getKey(), false).build(), null); - discoveryNodesBuilder.add(mockNode.discoveryNode); - if (randomBoolean()) { - // we really don't care - just chaos monkey - mockNode.action.errorOnCommit.set(randomBoolean()); - mockNode.action.errorOnSend.set(randomBoolean()); - mockNode.action.timeoutOnCommit.set(randomBoolean()); - mockNode.action.timeoutOnSend.set(randomBoolean()); - } - } - - final int minMasterNodes; - final String expectedBehavior; - if (expectingToCommit) { - minMasterNodes = randomIntBetween(0, goodNodes + 1); // count master - expectedBehavior = "succeed"; - } else { - minMasterNodes = randomIntBetween(goodNodes + 2, numberOfMasterNodes); // +2 because of master - expectedBehavior = timeOutNodes > 0 ? "timeout" : "fail"; - } - logger.info("--> expecting commit to {}. good nodes [{}], errors [{}], timeouts [{}]. min_master_nodes [{}]", - expectedBehavior, goodNodes + 1, errorNodes, timeOutNodes, minMasterNodes); - - discoveryNodesBuilder.localNodeId(master.discoveryNode.getId()).masterNodeId(master.discoveryNode.getId()); - DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); - MetaData metaData = MetaData.EMPTY_META_DATA; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).metaData(metaData).nodes(discoveryNodes).build(); - ClusterState previousState = master.clusterState; - try { - publishState(master.action, clusterState, previousState, minMasterNodes); - if (expectingToCommit == false) { - fail("cluster state publishing didn't fail despite of not have enough nodes"); - } - } catch (FailedToCommitClusterStateException exception) { - logger.debug("failed to publish as expected", exception); - if (expectingToCommit) { - throw exception; - } - assertThat(exception.getMessage(), containsString(timeOutNodes > 0 ? "timed out" : "failed")); - } - - assertWarnings( - "[discovery.zen.publish_timeout] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version.", - "[discovery.zen.commit_timeout] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version."); - } - - public void testOutOfOrderCommitMessages() throws Throwable { - MockNode node = createMockNode("node").setAsMaster(); - final CapturingTransportChannel channel = new CapturingTransportChannel(); - - List states = new ArrayList<>(); - final int numOfStates = scaledRandomIntBetween(3, 25); - for (int i = 1; i <= numOfStates; i++) { - states.add(ClusterState.builder(node.clusterState).version(i).stateUUID(ClusterState.UNKNOWN_UUID).build()); - } - - final ClusterState finalState = states.get(numOfStates - 1); - - logger.info("--> publishing states"); - for (ClusterState state : states) { - node.action.handleIncomingClusterStateRequest( - new BytesTransportRequest(PublishClusterStateAction.serializeFullClusterState(state, Version.CURRENT), Version.CURRENT), - channel); - assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE)); - assertThat(channel.error.get(), nullValue()); - channel.clear(); - - } - - logger.info("--> committing states"); - - long largestVersionSeen = Long.MIN_VALUE; - Randomness.shuffle(states); - for (ClusterState state : states) { - node.action.handleCommitRequest(new PublishClusterStateAction.CommitClusterStateRequest(state.stateUUID()), channel); - if (largestVersionSeen < state.getVersion()) { - assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE)); - if (channel.error.get() != null) { - throw channel.error.get(); - } - largestVersionSeen = state.getVersion(); - } else { - // older cluster states will be rejected - assertNotNull(channel.error.get()); - assertThat(channel.error.get(), instanceOf(IllegalStateException.class)); - } - channel.clear(); - } - - //now check the last state held - assertSameState(node.clusterState, finalState); - } - - /** - * Tests that cluster is committed or times out. It should never be the case that we fail - * an update due to a commit timeout, but it ends up being committed anyway - */ - public void testTimeoutOrCommit() throws Exception { - Settings settings = Settings.builder() - // short but so we will sometime commit sometime timeout - .put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "1ms").build(); - - MockNode master = createMockNode("master", settings, null); - MockNode node = createMockNode("node", settings, null); - ClusterState state = ClusterState.builder(master.clusterState) - .nodes(DiscoveryNodes.builder(master.clusterState.nodes()) - .add(node.discoveryNode).masterNodeId(master.discoveryNode.getId())).build(); - - for (int i = 0; i < 10; i++) { - state = ClusterState.builder(state).incrementVersion().build(); - logger.debug("--> publishing version [{}], UUID [{}]", state.version(), state.stateUUID()); - boolean success; - try { - publishState(master.action, state, master.clusterState, 2).await(1, TimeUnit.HOURS); - success = true; - } catch (FailedToCommitClusterStateException OK) { - success = false; - } - logger.debug("--> publishing [{}], verifying...", success ? "succeeded" : "failed"); - - if (success) { - assertSameState(node.clusterState, state); - } else { - assertThat(node.clusterState.stateUUID(), not(equalTo(state.stateUUID()))); - } - } - - assertWarnings( - "[discovery.zen.commit_timeout] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version."); - } - - private void assertPublishClusterStateStats(String description, MockNode node, long expectedFull, long expectedIncompatibleDiffs, - long expectedCompatibleDiffs) { - PublishClusterStateStats stats = node.action.stats(); - assertThat(description + ": full cluster states", stats.getFullClusterStateReceivedCount(), equalTo(expectedFull)); - assertThat(description + ": incompatible cluster state diffs", stats.getIncompatibleClusterStateDiffReceivedCount(), - equalTo(expectedIncompatibleDiffs)); - assertThat(description + ": compatible cluster state diffs", stats.getCompatibleClusterStateDiffReceivedCount(), - equalTo(expectedCompatibleDiffs)); - } - - public void testPublishClusterStateStats() throws Exception { - MockNode nodeA = createMockNode("nodeA").setAsMaster(); - MockNode nodeB = createMockNode("nodeB"); - - assertPublishClusterStateStats("nodeA: initial state", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: initial state", nodeB, 0, 0, 0); - - // Initial cluster state - ClusterState clusterState = nodeA.clusterState; - - // cluster state update - add nodeB - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(clusterState.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // Sent as a full cluster state update - assertPublishClusterStateStats("nodeA: after full update", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: after full update", nodeB, 1, 0, 0); - - // Increment cluster state version - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // Sent, successfully, as a cluster state diff - assertPublishClusterStateStats("nodeA: after successful diff update", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: after successful diff update", nodeB, 1, 0, 1); - - // Increment cluster state version twice - previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); - clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // Sent, unsuccessfully, as a diff and then retried as a full update - assertPublishClusterStateStats("nodeA: after unsuccessful diff update", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: after unsuccessful diff update", nodeB, 2, 1, 1); - - // node A steps down from being master - nodeA.resetMasterId(); - nodeB.resetMasterId(); - - // node B becomes the master and sends a version of the cluster state that goes back - discoveryNodes = DiscoveryNodes.builder(discoveryNodes) - .add(nodeA.discoveryNode) - .add(nodeB.discoveryNode) - .masterNodeId(nodeB.discoveryNode.getId()) - .localNodeId(nodeB.discoveryNode.getId()) - .build(); - previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeB.action, clusterState, previousClusterState); - - // Sent, unsuccessfully, as a diff, and then retried as a full update - assertPublishClusterStateStats("nodeA: B became master", nodeA, 1, 1, 0); - assertPublishClusterStateStats("nodeB: B became master", nodeB, 2, 1, 1); - } - - private MetaData buildMetaDataForVersion(MetaData metaData, long version) { - ImmutableOpenMap.Builder indices = ImmutableOpenMap.builder(metaData.indices()); - indices.put("test" + version, IndexMetaData.builder("test" + version) - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards((int) version).numberOfReplicas(0).build()); - return MetaData.builder(metaData) - .transientSettings(Settings.builder().put("test", version).build()) - .indices(indices.build()) - .build(); - } - - private void assertProperMetaDataForVersion(MetaData metaData, long version) { - for (long i = 1; i <= version; i++) { - assertThat(metaData.index("test" + i), notNullValue()); - assertThat(metaData.index("test" + i).getNumberOfShards(), equalTo((int) i)); - } - assertThat(metaData.index("test" + (version + 1)), nullValue()); - assertThat(metaData.transientSettings().get("test"), equalTo(Long.toString(version))); - } - - public void publishStateAndWait(PublishClusterStateAction action, ClusterState state, - ClusterState previousState) throws InterruptedException { - publishState(action, state, previousState).await(1, TimeUnit.SECONDS); - } - - public AssertingAckListener publishState(PublishClusterStateAction action, ClusterState state, - ClusterState previousState) throws InterruptedException { - final int minimumMasterNodes = randomIntBetween(-1, state.nodes().getMasterNodes().size()); - return publishState(action, state, previousState, minimumMasterNodes); - } - - public AssertingAckListener publishState(PublishClusterStateAction action, ClusterState state, - ClusterState previousState, int minMasterNodes) throws InterruptedException { - AssertingAckListener assertingAckListener = new AssertingAckListener(state.nodes().getSize() - 1); - ClusterChangedEvent changedEvent = new ClusterChangedEvent("test update", state, previousState); - action.publish(changedEvent, minMasterNodes, assertingAckListener); - return assertingAckListener; - } - - public static class AssertingAckListener implements Discovery.AckListener { - private final List> errors = new CopyOnWriteArrayList<>(); - private final Set successfulAcks = Collections.synchronizedSet(new HashSet<>()); - private final CountDownLatch countDown; - private final CountDownLatch commitCountDown; - - public AssertingAckListener(int nodeCount) { - countDown = new CountDownLatch(nodeCount); - commitCountDown = new CountDownLatch(1); - } - - @Override - public void onCommit(TimeValue commitTime) { - commitCountDown.countDown(); - } - - @Override - public void onNodeAck(DiscoveryNode node, @Nullable Exception e) { - if (e != null) { - errors.add(new Tuple<>(node, e)); - } else { - successfulAcks.add(node); - } - countDown.countDown(); - } - - public Set await(long timeout, TimeUnit unit) throws InterruptedException { - assertThat(awaitErrors(timeout, unit), emptyIterable()); - assertTrue(commitCountDown.await(timeout, unit)); - return new HashSet<>(successfulAcks); - } - - public List> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException { - countDown.await(timeout, unit); - return errors; - } - - } - - void assertSameState(ClusterState actual, ClusterState expected) { - assertThat(actual, notNullValue()); - final String reason = "\n--> actual ClusterState: " + actual + "\n" + - "--> expected ClusterState:" + expected; - assertThat("unequal UUIDs" + reason, actual.stateUUID(), equalTo(expected.stateUUID())); - assertThat("unequal versions" + reason, actual.version(), equalTo(expected.version())); - } - - void assertSameStateFromDiff(ClusterState actual, ClusterState expected) { - assertSameState(actual, expected); - assertTrue(actual.wasReadFromDiff()); - } - - void assertSameStateFromFull(ClusterState actual, ClusterState expected) { - assertSameState(actual, expected); - assertFalse(actual.wasReadFromDiff()); - } - - public static class MockPublishAction extends PublishClusterStateAction { - - AtomicBoolean timeoutOnSend = new AtomicBoolean(); - AtomicBoolean errorOnSend = new AtomicBoolean(); - AtomicBoolean timeoutOnCommit = new AtomicBoolean(); - AtomicBoolean errorOnCommit = new AtomicBoolean(); - - public MockPublishAction(TransportService transportService, NamedWriteableRegistry namedWriteableRegistry, - IncomingClusterStateListener listener, DiscoverySettings discoverySettings) { - super(transportService, namedWriteableRegistry, listener, discoverySettings); - } - - @Override - protected void handleIncomingClusterStateRequest(BytesTransportRequest request, TransportChannel channel) throws IOException { - if (errorOnSend.get()) { - throw new ElasticsearchException("forced error on incoming cluster state"); - } - if (timeoutOnSend.get()) { - return; - } - super.handleIncomingClusterStateRequest(request, channel); - } - - @Override - protected void handleCommitRequest(PublishClusterStateAction.CommitClusterStateRequest request, TransportChannel channel) { - if (errorOnCommit.get()) { - throw new ElasticsearchException("forced error on incoming commit"); - } - if (timeoutOnCommit.get()) { - return; - } - super.handleCommitRequest(request, channel); - } - } - - static class CapturingTransportChannel implements TransportChannel { - - AtomicReference response = new AtomicReference<>(); - AtomicReference error = new AtomicReference<>(); - - public void clear() { - response.set(null); - error.set(null); - } - - @Override - public String getProfileName() { - return "_noop_"; - } - - @Override - public void sendResponse(TransportResponse response) throws IOException { - this.response.set(response); - assertThat(error.get(), nullValue()); - } - - @Override - public void sendResponse(Exception exception) throws IOException { - this.error.set(exception); - assertThat(response.get(), nullValue()); - } - - @Override - public String getChannelType() { - return "capturing"; - } - } -} diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java deleted file mode 100644 index 49163f9aa1fb1..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ /dev/null @@ -1,602 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.replication.ClusterStateCreationUtils; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.ClusterModule; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskExecutor; -import org.elasticsearch.cluster.ESAllocationTestCase; -import org.elasticsearch.cluster.coordination.JoinTaskExecutor; -import org.elasticsearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor; -import org.elasticsearch.cluster.coordination.ValidateJoinRequest; -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.routing.allocation.AllocationService; -import org.elasticsearch.cluster.service.ClusterApplier; -import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; -import org.elasticsearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener; -import org.elasticsearch.discovery.zen.ZenDiscovery.ZenNodeRemovalClusterStateTaskExecutor; -import org.elasticsearch.gateway.GatewayMetaState; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.test.ClusterServiceUtils; -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.TransportService; -import org.hamcrest.CoreMatchers; -import org.hamcrest.core.IsInstanceOf; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayDeque; -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.Locale; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -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.cluster.service.MasterServiceTests.discoveryState; -import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; -import static org.elasticsearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.emptyArray; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasToString; -import static org.hamcrest.Matchers.is; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -public class ZenDiscoveryUnitTests extends ESTestCase { - - public void testShouldIgnoreNewClusterState() { - ClusterName clusterName = new ClusterName("abc"); - - DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("a").add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); - DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder(); - newNodes.masterNodeId("a").add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); - - ClusterState.Builder currentState = ClusterState.builder(clusterName); - currentState.nodes(currentNodes); - ClusterState.Builder newState = ClusterState.builder(clusterName); - newState.nodes(newNodes); - - currentState.version(2); - newState.version(1); - assertTrue("should ignore, because new state's version is lower to current state's version", - shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); - currentState.version(1); - newState.version(1); - assertTrue("should ignore, because new state's version is equal to current state's version", - shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); - currentState.version(1); - newState.version(2); - assertFalse("should not ignore, because new state's version is higher to current state's version", - shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); - - currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId("b").add(new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)); - - // version isn't taken into account, so randomize it to ensure this. - if (randomBoolean()) { - currentState.version(2); - newState.version(1); - } else { - currentState.version(1); - newState.version(2); - } - currentState.nodes(currentNodes); - try { - shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build()); - fail("should ignore, because current state's master is not equal to new state's master"); - } catch (IllegalStateException e) { - assertThat(e.getMessage(), containsString("cluster state from a different master than the current one, rejecting")); - } - - currentNodes = DiscoveryNodes.builder(); - currentNodes.masterNodeId(null); - currentState.nodes(currentNodes); - // version isn't taken into account, so randomize it to ensure this. - if (randomBoolean()) { - currentState.version(2); - newState.version(1); - } else { - currentState.version(1); - newState.version(2); - } - assertFalse("should not ignore, because current state doesn't have a master", - shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())); - } - - public void testFilterNonMasterPingResponse() { - ArrayList responses = new ArrayList<>(); - ArrayList masterNodes = new ArrayList<>(); - ArrayList allNodes = new ArrayList<>(); - for (int i = randomIntBetween(10, 20); i >= 0; i--) { - Set roles = new HashSet<>(randomSubsetOf(Arrays.asList(Role.values()))); - DiscoveryNode node = new DiscoveryNode("node_" + i, "id_" + i, buildNewFakeTransportAddress(), Collections.emptyMap(), - roles, Version.CURRENT); - responses.add(new ZenPing.PingResponse(node, randomBoolean() ? null : node, new ClusterName("test"), randomLong())); - allNodes.add(node); - if (node.isMasterNode()) { - masterNodes.add(node); - } - } - - boolean ignore = randomBoolean(); - List filtered = ZenDiscovery.filterPingResponses(responses, ignore, logger); - final List filteredNodes = filtered.stream().map(ZenPing.PingResponse::node).collect(Collectors.toList()); - if (ignore) { - assertThat(filteredNodes, equalTo(masterNodes)); - } else { - assertThat(filteredNodes, equalTo(allNodes)); - } - } - - public void testNodesUpdatedAfterClusterStatePublished() throws Exception { - ThreadPool threadPool = new TestThreadPool(getClass().getName()); - // randomly make minimum_master_nodes a value higher than we have nodes for, so it will force failure - int minMasterNodes = randomBoolean() ? 3 : 1; - Settings settings = Settings.builder() - .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.toString(minMasterNodes)).build(); - - ArrayDeque toClose = new ArrayDeque<>(); - try { - Set expectedFDNodes = null; - - final MockTransportService masterTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); - masterTransport.start(); - DiscoveryNode masterNode = masterTransport.getLocalNode(); - toClose.addFirst(masterTransport); - ClusterState state = ClusterStateCreationUtils.state(masterNode, masterNode, masterNode); - // build the zen discovery and discovery service - MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode); - toClose.addFirst(masterMasterService); - // TODO: clustername shouldn't be stored twice in cluster service, but for now, work around it - state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build(); - Settings settingsWithClusterName = Settings.builder().put(settings).put( - ClusterName.CLUSTER_NAME_SETTING.getKey(), discoveryState(masterMasterService).getClusterName().value()).build(); - ZenDiscovery masterZen = buildZenDiscovery( - settingsWithClusterName, - masterTransport, masterMasterService, threadPool); - masterZen.setCommittedState(state); - toClose.addFirst(masterZen); - masterTransport.acceptIncomingRequests(); - - final MockTransportService otherTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); - otherTransport.start(); - toClose.addFirst(otherTransport); - - DiscoveryNode otherNode = otherTransport.getLocalNode(); - final ClusterState otherState = ClusterState.builder(discoveryState(masterMasterService).getClusterName()) - .nodes(DiscoveryNodes.builder().add(otherNode).localNodeId(otherNode.getId())).build(); - MasterService otherMasterService = ClusterServiceUtils.createMasterService(threadPool, otherNode); - toClose.addFirst(otherMasterService); - ZenDiscovery otherZen = buildZenDiscovery(settingsWithClusterName, otherTransport, otherMasterService, threadPool); - otherZen.setCommittedState(otherState); - toClose.addFirst(otherZen); - otherTransport.acceptIncomingRequests(); - - masterTransport.connectToNode(otherNode); - otherTransport.connectToNode(masterNode); - - // a new cluster state with a new discovery node (we will test if the cluster state - // was updated by the presence of this node in NodesFaultDetection) - ClusterState newState = ClusterState.builder(discoveryState(masterMasterService)).incrementVersion().nodes( - DiscoveryNodes.builder(state.nodes()).add(otherNode).masterNodeId(masterNode.getId()) - ).build(); - - // publishing a new cluster state - ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent("testing", newState, state); - AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1); - expectedFDNodes = masterZen.getFaultDetectionNodes(); - AwaitingPublishListener awaitingPublishListener = new AwaitingPublishListener(); - masterZen.publish(clusterChangedEvent, awaitingPublishListener, listener); - awaitingPublishListener.await(); - if (awaitingPublishListener.getException() == null) { - // publication succeeded, wait for acks - listener.await(10, TimeUnit.SECONDS); - // publish was a success, update expected FD nodes based on new cluster state - expectedFDNodes = fdNodesForState(newState, masterNode); - } else { - // not successful, so expectedFDNodes above should remain what it was originally assigned - assertEquals(3, minMasterNodes); // ensure min master nodes is the higher value, otherwise we shouldn't fail - } - - assertEquals(expectedFDNodes, masterZen.getFaultDetectionNodes()); - } finally { - IOUtils.close(toClose); - terminate(threadPool); - } - - assertWarnings("[discovery.zen.minimum_master_nodes] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version."); - } - - public void testPendingCSQueueIsClearedWhenClusterStatePublished() throws Exception { - ThreadPool threadPool = new TestThreadPool(getClass().getName()); - // randomly make minimum_master_nodes a value higher than we have nodes for, so it will force failure - int minMasterNodes = randomBoolean() ? 3 : 1; - Settings settings = Settings.builder() - .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.toString(minMasterNodes)).build(); - - ArrayDeque toClose = new ArrayDeque<>(); - try { - final MockTransportService masterTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); - masterTransport.start(); - DiscoveryNode masterNode = masterTransport.getLocalNode(); - toClose.addFirst(masterTransport); - ClusterState state = ClusterStateCreationUtils.state(masterNode, null, masterNode); - // build the zen discovery and master service for the master node - MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode); - toClose.addFirst(masterMasterService); - state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build(); - ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterMasterService, threadPool); - masterZen.setCommittedState(state); - toClose.addFirst(masterZen); - masterTransport.acceptIncomingRequests(); - - // inject a pending cluster state - masterZen.pendingClusterStatesQueue().addPending(ClusterState.builder(new ClusterName("foreign")).build()); - - // a new cluster state with a new discovery node (we will test if the cluster state - // was updated by the presence of this node in NodesFaultDetection) - ClusterState newState = ClusterState.builder(discoveryState(masterMasterService)).incrementVersion().nodes( - DiscoveryNodes.builder(discoveryState(masterMasterService).nodes()).masterNodeId(masterNode.getId()) - ).build(); - - // publishing a new cluster state - ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent("testing", newState, state); - AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1); - AwaitingPublishListener awaitingPublishListener = new AwaitingPublishListener(); - masterZen.publish(clusterChangedEvent, awaitingPublishListener, listener); - awaitingPublishListener.await(); - if (awaitingPublishListener.getException() == null) { - // publication succeeded, wait for acks - listener.await(1, TimeUnit.HOURS); - } - // queue should be cleared whether successful or not - assertThat(Arrays.toString(masterZen.pendingClusterStates()), masterZen.pendingClusterStates(), emptyArray()); - } finally { - IOUtils.close(toClose); - terminate(threadPool); - } - - assertWarnings("[discovery.zen.minimum_master_nodes] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version."); - } - - private class AwaitingPublishListener implements ActionListener { - private final CountDownLatch countDownLatch = new CountDownLatch(1); - private FailedToCommitClusterStateException exception; - - @Override - public synchronized void onResponse(Void aVoid) { - assertThat(countDownLatch.getCount(), is(1L)); - countDownLatch.countDown(); - } - - @Override - public synchronized void onFailure(Exception e) { - assertThat(e, IsInstanceOf.instanceOf(FailedToCommitClusterStateException.class)); - exception = (FailedToCommitClusterStateException) e; - onResponse(null); - } - - public void await() throws InterruptedException { - countDownLatch.await(); - } - - public synchronized FailedToCommitClusterStateException getException() { - return exception; - } - } - - private ZenDiscovery buildZenDiscovery(Settings settings, TransportService service, MasterService masterService, - ThreadPool threadPool) { - ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - ClusterApplier clusterApplier = new ClusterApplier() { - @Override - public void setInitialState(ClusterState initialState) { - - } - - @Override - public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener) { - listener.onSuccess(source); - } - }; - ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, - new NamedWriteableRegistry(ClusterModule.getNamedWriteables()), - masterService, clusterApplier, clusterSettings, hostsResolver -> Collections.emptyList(), - ESAllocationTestCase.createAllocationService(), - Collections.emptyList(), mock(GatewayMetaState.class)); - zenDiscovery.start(); - return zenDiscovery; - } - - private Set fdNodesForState(ClusterState clusterState, DiscoveryNode localNode) { - final Set discoveryNodes = new HashSet<>(); - clusterState.getNodes().getNodes().valuesIt().forEachRemaining(discoveryNode -> { - // the local node isn't part of the nodes that are pinged (don't ping ourselves) - if (discoveryNode.getId().equals(localNode.getId()) == false) { - discoveryNodes.add(discoveryNode); - } - }); - return discoveryNodes; - } - - public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { - final int iters = randomIntBetween(3, 10); - for (int i = 0; i < iters; i++) { - ClusterState.Builder stateBuilder = ClusterState.builder(ClusterName.DEFAULT); - final DiscoveryNode otherNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), - EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); - final DiscoveryNode localNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), - EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); - MembershipAction.ValidateJoinRequestRequestHandler request = new MembershipAction.ValidateJoinRequestRequestHandler - (() -> localNode, JoinTaskExecutor.addBuiltInJoinValidators(Collections.emptyList())); - 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 ValidateJoinRequest(stateBuilder.build()), null, null)); - assertEquals("index [test] version not supported: " - + VersionUtils.getPreviousVersion(Version.CURRENT.minimumIndexCompatibilityVersion()) - + " minimum compatible index version is: " + Version.CURRENT.minimumIndexCompatibilityVersion(), ex.getMessage()); - } else { - AtomicBoolean sendResponse = new AtomicBoolean(false); - request.messageReceived(new ValidateJoinRequest(stateBuilder.build()), new TransportChannel() { - - @Override - public String getProfileName() { - return null; - } - - @Override - public String getChannelType() { - return null; - } - - @Override - public void sendResponse(TransportResponse response) throws IOException { - sendResponse.set(true); - } - - @Override - public void sendResponse(Exception exception) throws IOException { - - } - }, null); - assertTrue(sendResponse.get()); - } - } - } - - public void testIncomingClusterStateValidation() throws Exception { - ClusterName clusterName = new ClusterName("abc"); - - DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder().add( - new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT)).localNodeId("a"); - - ClusterState previousState = ClusterState.builder(clusterName).nodes(currentNodes).build(); - - logger.info("--> testing acceptances of any master when having no master"); - ClusterState state = ClusterState.builder(previousState) - .nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId(randomAlphaOfLength(10))).incrementVersion().build(); - ZenDiscovery.validateIncomingState(logger, state, previousState); - - // now set a master node - previousState = state; - state = ClusterState.builder(previousState) - .nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId("master")).build(); - logger.info("--> testing rejection of another master"); - try { - ZenDiscovery.validateIncomingState(logger, state, previousState); - fail("node accepted state from another master"); - } catch (IllegalStateException OK) { - assertThat(OK.toString(), containsString("cluster state from a different master than the current one, rejecting")); - } - - logger.info("--> test state from the current master is accepted"); - previousState = state; - ZenDiscovery.validateIncomingState(logger, ClusterState.builder(previousState) - .nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId("master")).incrementVersion().build(), previousState); - - - logger.info("--> testing rejection of another cluster name"); - try { - ZenDiscovery.validateIncomingState(logger, ClusterState.builder(new ClusterName(randomAlphaOfLength(10))) - .nodes(previousState.nodes()).build(), previousState); - fail("node accepted state with another cluster name"); - } catch (IllegalStateException OK) { - assertThat(OK.toString(), containsString("received state from a node that is not part of the cluster")); - } - - logger.info("--> testing rejection of a cluster state with wrong local node"); - try { - state = ClusterState.builder(previousState) - .nodes(DiscoveryNodes.builder(previousState.nodes()).localNodeId("_non_existing_").build()) - .incrementVersion().build(); - ZenDiscovery.validateIncomingState(logger, state, previousState); - fail("node accepted state with non-existence local node"); - } catch (IllegalStateException OK) { - assertThat(OK.toString(), containsString("received state with a local node that does not match the current local node")); - } - - try { - DiscoveryNode otherNode = new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); - state = ClusterState.builder(previousState).nodes( - DiscoveryNodes.builder(previousState.nodes()).add(otherNode) - .localNodeId(otherNode.getId()).build() - ).incrementVersion().build(); - ZenDiscovery.validateIncomingState(logger, state, previousState); - fail("node accepted state with existent but wrong local node"); - } catch (IllegalStateException OK) { - assertThat(OK.toString(), containsString("received state with a local node that does not match the current local node")); - } - - logger.info("--> testing acceptance of an old cluster state"); - final ClusterState incomingState = previousState; - previousState = ClusterState.builder(previousState).incrementVersion().build(); - final ClusterState finalPreviousState = previousState; - final IllegalStateException e = - expectThrows(IllegalStateException.class, () -> ZenDiscovery.validateIncomingState(logger, incomingState, finalPreviousState)); - final String message = String.format( - Locale.ROOT, - "rejecting cluster state version [%d] uuid [%s] received from [%s]", - incomingState.version(), - incomingState.stateUUID(), - incomingState.nodes().getMasterNodeId() - ); - assertThat(e, hasToString("java.lang.IllegalStateException: " + message)); - - ClusterState higherVersionState = ClusterState.builder(previousState).incrementVersion().build(); - // remove the master of the node (but still have a previous cluster state with it)! - higherVersionState = ClusterState.builder(higherVersionState) - .nodes(DiscoveryNodes.builder(higherVersionState.nodes()).masterNodeId(null)).build(); - // an older version from a *new* master is also OK! - state = ClusterState.builder(previousState) - .nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId("_new_master_").build()) - .build(); - - ZenDiscovery.validateIncomingState(logger, state, higherVersionState); - } - - public void testNotEnoughMasterNodesAfterRemove() throws Exception { - final ElectMasterService electMasterService = mock(ElectMasterService.class); - when(electMasterService.hasEnoughMasterNodes(any(Iterable.class))).thenReturn(false); - - final AllocationService allocationService = mock(AllocationService.class); - - final AtomicBoolean rejoinCalled = new AtomicBoolean(); - final Consumer submitRejoin = source -> rejoinCalled.set(true); - - final AtomicReference remainingNodesClusterState = new AtomicReference<>(); - final ZenNodeRemovalClusterStateTaskExecutor executor = - new ZenNodeRemovalClusterStateTaskExecutor(allocationService, electMasterService, submitRejoin, logger) { - @Override - protected ClusterState remainingNodesClusterState(ClusterState currentState, DiscoveryNodes.Builder remainingNodesBuilder) { - remainingNodesClusterState.set(super.remainingNodesClusterState(currentState, remainingNodesBuilder)); - return remainingNodesClusterState.get(); - } - }; - - final DiscoveryNodes.Builder builder = DiscoveryNodes.builder(); - final int nodes = randomIntBetween(2, 16); - final List tasks = new ArrayList<>(); - // to ensure there is at least one removal - boolean first = true; - for (int i = 0; i < nodes; i++) { - final DiscoveryNode node = node(i); - builder.add(node); - if (first || randomBoolean()) { - tasks.add(new NodeRemovalClusterStateTaskExecutor.Task(node, randomBoolean() ? "left" : "failed")); - } - first = false; - } - final ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(builder).build(); - - final ClusterStateTaskExecutor.ClusterTasksResult result = - executor.execute(clusterState, tasks); - verify(electMasterService).hasEnoughMasterNodes(eq(remainingNodesClusterState.get().nodes())); - verify(electMasterService).countMasterNodes(eq(remainingNodesClusterState.get().nodes())); - verify(electMasterService).minimumMasterNodes(); - verifyNoMoreInteractions(electMasterService); - - // ensure that we did not reroute - verifyNoMoreInteractions(allocationService); - assertTrue(rejoinCalled.get()); - assertThat(result.resultingState, CoreMatchers.equalTo(clusterState)); - - for (final NodeRemovalClusterStateTaskExecutor.Task task : tasks) { - assertNotNull(result.resultingState.nodes().get(task.node().getId())); - } - } - - private DiscoveryNode node(final int id) { - return new DiscoveryNode(Integer.toString(id), buildNewFakeTransportAddress(), Version.CURRENT); - } -} diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java deleted file mode 100644 index 9fa680fc200d1..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery.zen; - -import org.elasticsearch.Version; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.ESTestCase; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static org.hamcrest.Matchers.equalTo; - -public class ZenPingTests extends ESTestCase { - public void testPingCollection() { - DiscoveryNode[] nodes = new DiscoveryNode[randomIntBetween(1, 30)]; - long maxIdPerNode[] = new long[nodes.length]; - DiscoveryNode masterPerNode[] = new DiscoveryNode[nodes.length]; - long clusterStateVersionPerNode[] = new long[nodes.length]; - ArrayList pings = new ArrayList<>(); - for (int i = 0; i < nodes.length; i++) { - nodes[i] = new DiscoveryNode("" + i, buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); - } - - for (int pingCount = scaledRandomIntBetween(10, nodes.length * 10); pingCount > 0; pingCount--) { - int node = randomInt(nodes.length - 1); - DiscoveryNode masterNode = null; - if (randomBoolean()) { - masterNode = nodes[randomInt(nodes.length - 1)]; - } - long clusterStateVersion = randomLong(); - ZenPing.PingResponse ping = new ZenPing.PingResponse(nodes[node], masterNode, ClusterName.CLUSTER_NAME_SETTING. - getDefault(Settings.EMPTY), clusterStateVersion); - if (rarely()) { - // ignore some pings - continue; - } - // update max ping info - maxIdPerNode[node] = ping.id(); - masterPerNode[node] = masterNode; - clusterStateVersionPerNode[node] = clusterStateVersion; - pings.add(ping); - } - - // shuffle - Collections.shuffle(pings, random()); - - ZenPing.PingCollection collection = new ZenPing.PingCollection(); - pings.forEach(collection::addPing); - - List aggregate = collection.toList(); - - for (ZenPing.PingResponse ping : aggregate) { - int nodeId = Integer.parseInt(ping.node().getId()); - assertThat(maxIdPerNode[nodeId], equalTo(ping.id())); - assertThat(masterPerNode[nodeId], equalTo(ping.master())); - assertThat(clusterStateVersionPerNode[nodeId], equalTo(ping.getClusterStateVersion())); - - maxIdPerNode[nodeId] = -1; // mark as seen - } - - for (int i = 0; i < maxIdPerNode.length; i++) { - assertTrue("node " + i + " had pings but it was not found in collection", maxIdPerNode[i] <= 0); - } - - - } -} diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 4582e27d027da..39c74900de435 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -106,9 +106,6 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.Discovery; -import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.TestEnvironment; @@ -187,8 +184,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.elasticsearch.client.Requests.syncedFlushRequest; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; @@ -203,7 +198,6 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -562,29 +556,10 @@ protected final void afterInternal(boolean afterClass) throws Exception { assertThat("test leaves persistent cluster metadata behind", persistentKeys, empty()); final Set transientKeys = new HashSet<>(metaData.transientSettings().keySet()); - if (isInternalCluster() && internalCluster().getAutoManageMinMasterNode()) { - // this is set by the test infra - transientKeys.remove(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()); - } assertThat("test leaves transient cluster metadata behind", transientKeys, empty()); } ensureClusterSizeConsistency(); ensureClusterStateConsistency(); - if (isInternalCluster()) { - // check no pending cluster states are leaked - for (Discovery discovery : internalCluster().getInstances(Discovery.class)) { - if (discovery instanceof ZenDiscovery) { - final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery; - assertBusy(() -> { - final ClusterState[] states = zenDiscovery.pendingClusterStates(); - assertThat(zenDiscovery.clusterState().nodes().getLocalNode().getName() + - " still having pending states:\n" + - Stream.of(states).map(ClusterState::toString).collect(Collectors.joining("\n")), - states, emptyArray()); - }); - } - } - } beforeIndexDeletion(); cluster().wipe(excludeTemplates()); // wipe after to make sure we fail in the test that didn't ack the delete if (afterClass || currentClusterScope == Scope.TEST) { @@ -1667,8 +1642,7 @@ public enum Scope { boolean supportsDedicatedMasters() default true; /** - * The cluster automatically manages the {@link ElectMasterService#DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING} by default - * as nodes are started and stopped. Set this to false to manage the setting manually. + * The cluster automatically manages the bootstrap voting configuration. Set this to false to manage the setting manually. */ boolean autoMinMasterNodes() default true; diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index a8a4f11a82e3e..8d92f80f62e4a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -30,7 +30,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.store.AlreadyClosedException; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; @@ -76,8 +75,6 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLockObtainFailedException; @@ -153,13 +150,11 @@ import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; import static org.elasticsearch.discovery.DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING; -import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.discovery.FileBasedSeedHostsProvider.UNICAST_HOSTS_FILE; import static org.elasticsearch.test.ESTestCase.assertBusy; import static org.elasticsearch.test.ESTestCase.awaitBusy; import static org.elasticsearch.test.ESTestCase.getTestTransportType; import static org.elasticsearch.test.ESTestCase.randomFrom; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -433,11 +428,6 @@ public String getClusterName() { return clusterName; } - /** returns true if the {@link ElectMasterService#DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING} setting is auto managed by this cluster */ - public boolean getAutoManageMinMasterNode() { - return autoManageMinMasterNodes; - } - public String[] getNodeNames() { return nodes.keySet().toArray(Strings.EMPTY_ARRAY); } @@ -651,22 +641,8 @@ private Settings getNodeSettings(final int nodeId, final long seed, final Settin final boolean usingZen1 = usingZen1(updatedSettings.build()); if (usingSingleNodeDiscovery == false) { if (autoManageMinMasterNodes) { - assertThat("min master nodes may not be set when auto managed", - updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()), nullValue()); assertThat("automatically managing min master nodes require nodes to complete a join cycle when starting", updatedSettings.get(INITIAL_STATE_TIMEOUT_SETTING.getKey()), nullValue()); - - if (usingZen1) { - updatedSettings - // don't wait too long not to slow down tests - .put(ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.getKey(), "5s") - .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), defaultMinMasterNodes); - } - } else { - if (usingZen1) { - assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " must be configured", - updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()), not(nullValue())); - } } } @@ -975,10 +951,6 @@ Settings closeForRestart(RestartCallback callback, int minMasterNodes) throws Ex Settings.Builder newSettings = Settings.builder(); newSettings.put(callbackSettings); if (minMasterNodes >= 0) { - if (usingZen1(newSettings.build())) { - assertFalse("min master nodes is auto managed", DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(newSettings.build())); - newSettings.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes); - } if (INITIAL_MASTER_NODES_SETTING.exists(callbackSettings) == false) { newSettings.putList(INITIAL_MASTER_NODES_SETTING.getKey()); } @@ -1010,24 +982,6 @@ private void recreateNode(final Settings newSettings, final Runnable onTransport .put(newSettings) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newIdSeed) .build(); - if (usingZen1(finalSettings)) { - if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings) == false) { - throw new IllegalStateException(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + - " is not configured after restart of [" + name + "]"); - } - } else { - if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings)) { - // simulating an upgrade from Zen1 to Zen2, but there's no way to remove a setting when restarting a node, so - // you have to set it to REMOVED_MINIMUM_MASTER_NODES (== Integer.MAX_VALUE) to indicate its removal: - assertTrue(DISCOVERY_TYPE_SETTING.exists(finalSettings)); - assertThat(DISCOVERY_TYPE_SETTING.get(finalSettings), equalTo(ZEN2_DISCOVERY_TYPE)); - assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(finalSettings), equalTo(REMOVED_MINIMUM_MASTER_NODES)); - - final Builder builder = Settings.builder().put(finalSettings); - builder.remove(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()); - finalSettings = builder.build(); - } - } Collection> plugins = node.getClasspathPlugins(); node = new MockNode(finalSettings, plugins); node.injector().getInstance(TransportService.class).addLifecycleListener(new LifecycleListener() { @@ -1126,9 +1080,6 @@ private synchronized void reset(boolean wipeData) throws IOException { if (wipeData) { wipePendingDataDirectories(); } - if (nodes.size() > 0 && autoManageMinMasterNodes) { - updateMinMasterNodes(getMasterNodesCount()); - } logger.debug("Cluster hasn't changed - moving out - nodes: [{}] nextNodeId: [{}] numSharedNodes: [{}]", nodes.keySet(), nextNodeId.get(), newSize); return; @@ -1655,11 +1606,6 @@ private synchronized void startAndPublishNodesAndClients(List nod .filter(nac -> nodes.containsKey(nac.name) == false) // filter out old masters .count(); final int currentMasters = getMasterNodesCount(); - if (autoManageMinMasterNodes && currentMasters > 0 && newMasters > 0 && - getMinMasterNodes(currentMasters + newMasters) <= currentMasters) { - // if we're adding too many master-eligible nodes at once, we can't update the min master setting before adding the nodes. - updateMinMasterNodes(currentMasters + newMasters); - } rebuildUnicastHostFiles(nodeAndClients); // ensure that new nodes can find the existing nodes when they start List> futures = nodeAndClients.stream().map(node -> executor.submit(node::startNode)).collect(Collectors.toList()); @@ -1678,7 +1624,6 @@ private synchronized void startAndPublishNodesAndClients(List nod getMinMasterNodes(currentMasters + newMasters) > currentMasters) { // update once masters have joined validateClusterFormed(); - updateMinMasterNodes(currentMasters + newMasters); } } } @@ -1813,10 +1758,6 @@ private void restartNode(NodeAndClient nodeAndClient, RestartCallback callback) // the fact it left validateClusterFormed(nodeAndClient.name); } - - if (excludedNodeIds.isEmpty() == false) { - updateMinMasterNodes(getMasterNodesCount()); - } } private NodeAndClient removeNode(NodeAndClient nodeAndClient) { @@ -1852,10 +1793,6 @@ private Set excludeMasters(Collection nodeAndClients) { throw new AssertionError("unexpected", e); } } - - if (stoppingMasters > 0) { - updateMinMasterNodes(getMasterNodesCount() - Math.toIntExact(stoppingMasters)); - } } return excludedNodeIds; } @@ -2146,28 +2083,6 @@ public List startDataOnlyNodes(int numNodes) { .put(Node.NODE_DATA_SETTING.getKey(), true).build()); } - /** - * updates the min master nodes setting in the current running cluster. - * - * @param eligibleMasterNodeCount the number of master eligible nodes to use as basis for the min master node setting - */ - private void updateMinMasterNodes(int eligibleMasterNodeCount) { - assert autoManageMinMasterNodes; - final int minMasterNodes = getMinMasterNodes(eligibleMasterNodeCount); - if (getMasterNodesCount() > 0) { - // there should be at least one master to update - logger.debug("updating min_master_nodes to [{}]", minMasterNodes); - try { - assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings( - Settings.builder().put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes) - )); - } catch (Exception e) { - throw new ElasticsearchException("failed to update minimum master node to [{}] (current masters [{}])", e, - minMasterNodes, getMasterNodesCount()); - } - } - } - /** calculates a min master nodes value based on the given number of master nodes */ private static int getMinMasterNodes(int eligibleMasterNodes) { return eligibleMasterNodes / 2 + 1; From b0e51703c5dea0ac30aaf27d9114a5fedf51d647 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 14:51:27 +0100 Subject: [PATCH 13/30] checkstyle for SeedHostsResolver --- .../java/org/elasticsearch/discovery/SeedHostsResolver.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java index c13697965ede5..926216b9b686e 100644 --- a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java +++ b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java @@ -53,7 +53,8 @@ public class SeedHostsResolver extends AbstractLifecycleComponent implements ConfiguredHostsResolver { public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING = - Setting.intSetting("discovery.zen.ping.unicast.concurrent_connects", 10, 0, Setting.Property.NodeScope, Setting.Property.Deprecated); + Setting.intSetting("discovery.zen.ping.unicast.concurrent_connects", 10, 0, Setting.Property.NodeScope, + Setting.Property.Deprecated); public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT = Setting.positiveTimeSetting("discovery.zen.ping.unicast.hosts.resolve_timeout", TimeValue.timeValueSeconds(5), Setting.Property.NodeScope, Setting.Property.Deprecated); From cda20f5f37ead5bc81e1a145c147bb06ee54706a Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 16:36:08 +0100 Subject: [PATCH 14/30] remove zen1 from build --- .../gradle/test/ClusterConfiguration.groovy | 21 ------- .../gradle/test/ClusterFormationTasks.groovy | 58 +++++-------------- qa/logging-config/build.gradle | 1 - qa/rolling-upgrade/build.gradle | 1 - qa/unconfigured-node-name/build.gradle | 1 - x-pack/qa/rolling-upgrade-basic/build.gradle | 2 - .../build.gradle | 2 - x-pack/qa/rolling-upgrade/build.gradle | 2 - 8 files changed, 15 insertions(+), 73 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy index e64f30d48bd3f..1e3a200dba874 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy @@ -63,27 +63,6 @@ class ClusterConfiguration { @Input boolean debug = false - /** - * Configuration of the setting {@code discovery.zen.minimum_master_nodes} on the nodes. - * In case of more than one node, this defaults to the number of nodes - */ - @Input - Closure minimumMasterNodes = { - if (bwcVersion != null && bwcVersion.before("6.5.0")) { - return numNodes > 1 ? numNodes : -1 - } else { - return numNodes > 1 ? numNodes.intdiv(2) + 1 : -1 - } - } - - /** - * Whether the initial_master_nodes setting should be automatically derived from the nodes - * in the cluster. Only takes effect if all nodes in the cluster understand this setting - * and the discovery type is not explicitly set. - */ - @Input - boolean autoSetInitialMasterNodes = true - /** * Whether the file-based discovery provider should be automatically setup based on * the nodes in the cluster. Only takes effect if no other hosts provider is already diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index d1bb5c467564c..0bf3744412c8b 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -127,42 +127,25 @@ class ClusterFormationTasks { nodes.add(node) Closure writeConfigSetup Object dependsOn - if (node.nodeVersion.onOrAfter("6.5.0")) { - writeConfigSetup = { Map esConfig -> - if (config.getAutoSetHostsProvider()) { - // Don't force discovery provider if one is set by the test cluster specs already - final String seedProvidersSettingName = - node.nodeVersion.onOrAfter("7.0.0") ? "discovery.seed_providers" : "discovery.zen.hosts_provider"; - if (esConfig.containsKey(seedProvidersSettingName) == false) { - esConfig[seedProvidersSettingName] = 'file' - } - esConfig[node.nodeVersion.onOrAfter("7.0.0") ? "discovery.seed_hosts" : "discovery.zen.ping.unicast.hosts"] = [] - } - boolean supportsInitialMasterNodes = hasBwcNodes == false || config.bwcVersion.onOrAfter("7.0.0") - if (esConfig['discovery.type'] == null && config.getAutoSetInitialMasterNodes() && supportsInitialMasterNodes) { - esConfig['cluster.initial_master_nodes'] = nodes.stream().map({ n -> - if (n.config.settings['node.name'] == null) { - return "node-" + n.nodeNum - } else { - return n.config.settings['node.name'] - } - }).collect(Collectors.toList()) + writeConfigSetup = { Map esConfig -> + if (config.getAutoSetHostsProvider()) { + if (esConfig.containsKey("discovery.seed_providers") == false) { + esConfig["discovery.seed_providers"] = 'file' } - esConfig + esConfig["discovery.seed_hosts"] = [] } - dependsOn = startDependencies - } else { - dependsOn = startTasks.empty ? startDependencies : startTasks.get(0) - writeConfigSetup = { Map esConfig -> - String unicastTransportUri = node.config.unicastTransportUri(nodes.get(0), node, project.createAntBuilder()) - if (unicastTransportUri == null) { - esConfig['discovery.zen.ping.unicast.hosts'] = [] - } else { - esConfig['discovery.zen.ping.unicast.hosts'] = "\"${unicastTransportUri}\"" - } - esConfig + if (esConfig['discovery.type'] == null && config.getAutoSetInitialMasterNodes()) { + esConfig['cluster.initial_master_nodes'] = nodes.stream().map({ n -> + if (n.config.settings['node.name'] == null) { + return "node-" + n.nodeNum + } else { + return n.config.settings['node.name'] + } + }).collect(Collectors.toList()) } + esConfig } + dependsOn = startDependencies startTasks.add(configureNode(project, prefix, runner, dependsOn, node, config, distro, writeConfigSetup)) } @@ -380,17 +363,6 @@ class ClusterFormationTasks { // Don't wait for state, just start up quickly. This will also allow new and old nodes in the BWC case to become the master 'discovery.initial_state_timeout' : '0s' ] - int minimumMasterNodes = node.config.minimumMasterNodes.call() - if (node.nodeVersion.before("7.0.0") && minimumMasterNodes > 0) { - esConfig['discovery.zen.minimum_master_nodes'] = minimumMasterNodes - } - if (node.nodeVersion.before("7.0.0") && esConfig.containsKey('discovery.zen.master_election.wait_for_joins_timeout') == false) { - // If a node decides to become master based on partial information from the pinging, don't let it hang for 30 seconds to correct - // its mistake. Instead, only wait 5s to do another round of pinging. - // This is necessary since we use 30s as the default timeout in REST requests waiting for cluster formation - // so we need to bail quicker than the default 30s for the cluster to form in time. - esConfig['discovery.zen.master_election.wait_for_joins_timeout'] = '5s' - } esConfig['node.max_local_storage_nodes'] = node.config.numNodes esConfig['http.port'] = node.config.httpPort esConfig['transport.tcp.port'] = node.config.transportPort diff --git a/qa/logging-config/build.gradle b/qa/logging-config/build.gradle index 0abdc1247514a..3f6217ae717e9 100644 --- a/qa/logging-config/build.gradle +++ b/qa/logging-config/build.gradle @@ -23,7 +23,6 @@ apply plugin: 'elasticsearch.rest-test' apply plugin: 'elasticsearch.standalone-test' integTestCluster { - autoSetInitialMasterNodes = false autoSetHostsProvider = false /** * Provide a custom log4j configuration where layout is an old style pattern and confirm that Elasticsearch diff --git a/qa/rolling-upgrade/build.gradle b/qa/rolling-upgrade/build.gradle index 0fa245959a8de..89d9fd1eacaf1 100644 --- a/qa/rolling-upgrade/build.gradle +++ b/qa/rolling-upgrade/build.gradle @@ -77,7 +77,6 @@ for (Version version : bwcVersions.wireCompatible) { dependsOn lastRunner, "${baseName}#oldClusterTestCluster#node${stopNode}.stop" clusterName = 'rolling-upgrade' otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - autoSetInitialMasterNodes = false /* Override the data directory so the new node always gets the node we * just stopped's data directory. */ dataDir = { nodeNumber -> oldClusterTest.nodes[stopNode].dataDir } diff --git a/qa/unconfigured-node-name/build.gradle b/qa/unconfigured-node-name/build.gradle index 5aba0562e03f6..45d8a15f46744 100644 --- a/qa/unconfigured-node-name/build.gradle +++ b/qa/unconfigured-node-name/build.gradle @@ -24,7 +24,6 @@ integTestCluster { setting 'node.name', null // Run with no discovery configuration at all, demonstrating that a node in its // "out-of-the-box" configuration can automatically bootstrap a cluster - autoSetInitialMasterNodes = false autoSetHostsProvider = false } diff --git a/x-pack/qa/rolling-upgrade-basic/build.gradle b/x-pack/qa/rolling-upgrade-basic/build.gradle index 715d7d0854bbc..5b7cad500361c 100644 --- a/x-pack/qa/rolling-upgrade-basic/build.gradle +++ b/x-pack/qa/rolling-upgrade-basic/build.gradle @@ -55,8 +55,6 @@ for (Version version : bwcVersions.wireCompatible) { dependsOn lastRunner, "${baseName}#oldClusterTestCluster#node${stopNode}.stop" clusterName = 'rolling-upgrade-basic' otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - minimumMasterNodes = { 2 } - autoSetInitialMasterNodes = false /* Override the data directory so the new node always gets the node we * just stopped's data directory. */ dataDir = { nodeNumber -> oldClusterTest.nodes[stopNode].dataDir } diff --git a/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle b/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle index 0df8740424def..144b410391867 100644 --- a/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle +++ b/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle @@ -93,8 +93,6 @@ for (Version version : bwcVersions.wireCompatible) { dependsOn lastRunner, "${prefix}#${cluster}#clusterTestCluster#node${stopNode}.stop" clusterName = cluster otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - minimumMasterNodes = { 2 } - autoSetInitialMasterNodes = false /* Override the data directory so the new node always gets the node we * just stopped's data directory. */ dataDir = { nodeNumber -> clusterTest.nodes[stopNode].dataDir } diff --git a/x-pack/qa/rolling-upgrade/build.gradle b/x-pack/qa/rolling-upgrade/build.gradle index 71a2156e3df91..8f03bdf67095b 100644 --- a/x-pack/qa/rolling-upgrade/build.gradle +++ b/x-pack/qa/rolling-upgrade/build.gradle @@ -211,8 +211,6 @@ subprojects { setupCommand 'setupTestUser', 'bin/elasticsearch-users', 'useradd', 'test_user', '-p', 'x-pack-test-password', '-r', 'superuser' clusterName = 'rolling-upgrade' otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - minimumMasterNodes = { 2 } - autoSetInitialMasterNodes = false /* Override the data directory so the new node always gets the node we * just stopped's data directory. */ dataDir = { nodeNumber -> oldClusterTest.nodes[stopNode].dataDir } From f4967bdc08989a1ebe267fdce10bcbe54021cd85 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 16:36:36 +0100 Subject: [PATCH 15/30] Add migration docs for 8.0 --- docs/reference/migration/migrate_8_0.asciidoc | 2 ++ .../migration/migrate_8_0/discovery.asciidoc | 29 +++++++++++++++++++ .../discovery/discovery-settings.asciidoc | 27 +++-------------- 3 files changed, 35 insertions(+), 23 deletions(-) create mode 100644 docs/reference/migration/migrate_8_0/discovery.asciidoc diff --git a/docs/reference/migration/migrate_8_0.asciidoc b/docs/reference/migration/migrate_8_0.asciidoc index 58c2b1218231a..d6571112c57d7 100644 --- a/docs/reference/migration/migrate_8_0.asciidoc +++ b/docs/reference/migration/migrate_8_0.asciidoc @@ -12,6 +12,7 @@ See also <> and <>. coming[8.0.0] * <> +* <> * <> [float] @@ -31,4 +32,5 @@ Elasticsearch 7.x in order to be readable by Elasticsearch 8.x. ========================================= include::migrate_8_0/analysis.asciidoc[] +include::migrate_8_0/discovery.asciidoc[] include::migrate_8_0/mappings.asciidoc[] diff --git a/docs/reference/migration/migrate_8_0/discovery.asciidoc b/docs/reference/migration/migrate_8_0/discovery.asciidoc new file mode 100644 index 0000000000000..a078370b8509d --- /dev/null +++ b/docs/reference/migration/migrate_8_0/discovery.asciidoc @@ -0,0 +1,29 @@ +[float] +[[breaking_80_discovery_changes]] +=== Discovery changes + +[float] +==== Removal of old discovery settings + +All settings under the `discovery.zen` namespace, which existed only for BWC reasons in 7.x, +will no longer be supported. In particular, this includes: + +- `discovery.zen.no_master_block` +- `discovery.zen.hosts_provider` +- `discovery.zen.ping.unicast.concurrent_connects` +- `discovery.zen.ping.unicast.hosts.resolve_timeout` +- `discovery.zen.ping.unicast.hosts` +- `discovery.zen.unsafe_rolling_upgrades_enabled` +- `discovery.zen.commit_timeout` +- `discovery.zen.fd.connect_on_network_disconnect` +- `discovery.zen.fd.ping_interval` +- `discovery.zen.fd.ping_timeout` +- `discovery.zen.fd.ping_retries` +- `discovery.zen.fd.register_connection_listener` +- `discovery.zen.join_retry_attempts` +- `discovery.zen.join_retry_delay` +- `discovery.zen.max_pings_from_another_master` +- `discovery.zen.send_leave_request` +- `discovery.zen.master_election.wait_for_joins_timeout` +- `discovery.zen.master_election.ignore_non_master_pings` +- `discovery.zen.publish.max_pending_cluster_states` \ No newline at end of file diff --git a/docs/reference/modules/discovery/discovery-settings.asciidoc b/docs/reference/modules/discovery/discovery-settings.asciidoc index 3386fd66b499f..b5b1b4799e87b 100644 --- a/docs/reference/modules/discovery/discovery-settings.asciidoc +++ b/docs/reference/modules/discovery/discovery-settings.asciidoc @@ -8,22 +8,14 @@ Discovery and cluster formation are affected by the following settings: Provides a list of master-eligible nodes in the cluster. Each value has the format `host:port` or `host`, where `port` defaults to the setting `transport.profiles.default.port`. Note that IPv6 hosts must be bracketed. - The default value is `["127.0.0.1", "[::1]"]`. See <>. This - setting was previously known as `discovery.zen.ping.unicast.hosts`. Its old - name is deprecated but continues to work in order to preserve backwards - compatibility. Support for the old name will be removed in a future - version. + The default value is `["127.0.0.1", "[::1]"]`. See <>. `discovery.seed_providers`:: Specifies which types of <> to use to obtain the addresses of the seed nodes used to start the discovery process. By default, it is the - <>. This - setting was previously known as `discovery.zen.hosts_provider`. Its old - name is deprecated but continues to work in order to preserve backwards - compatibility. Support for the old name will be removed in a future - version. + <>. `discovery.cluster_formation_warning_timeout`:: @@ -55,20 +47,12 @@ Discovery and cluster formation are affected by the following settings: `discovery.seed_resolver.max_concurrent_resolvers`:: Specifies how many concurrent DNS lookups to perform when resolving the - addresses of seed nodes. Defaults to `10`. This setting was previously - known as `discovery.zen.ping.unicast.concurrent_connects`. Its old name is - deprecated but continues to work in order to preserve backwards - compatibility. Support for the old name will be removed in a future - version. + addresses of seed nodes. Defaults to `10`. `discovery.seed_resolver.timeout`:: Specifies how long to wait for each DNS lookup performed when resolving the - addresses of seed nodes. Defaults to `5s`. This setting was previously - known as `discovery.zen.ping.unicast.hosts.resolve_timeout`. Its old name - is deprecated but continues to work in order to preserve backwards - compatibility. Support for the old name will be removed in a future - version. + addresses of seed nodes. Defaults to `5s`. `cluster.auto_shrink_voting_configuration`:: @@ -192,7 +176,4 @@ APIs are not be blocked and can run on any available node. * For the cluster to be fully operational, it must have an active master. =============================== -WARNING: This setting replaces the `discovery.zen.no_master_block` setting in -earlier versions. The `discovery.zen.no_master_block` setting is ignored. - -- From 22438b72f65c729e3c2ff6addfff9763a583d88d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 16:37:19 +0100 Subject: [PATCH 16/30] remove more legacy settings --- .../coordination/ClusterBootstrapService.java | 5 +-- .../coordination/NoMasterBlockService.java | 6 ---- .../common/settings/ClusterSettings.java | 5 --- .../discovery/DiscoveryModule.java | 16 +-------- .../discovery/SeedHostsResolver.java | 22 ------------- .../SettingsBasedSeedHostsProvider.java | 14 +------- .../bootstrap/BootstrapChecksTests.java | 3 -- .../NoMasterBlockServiceTests.java | 33 ------------------- .../discovery/DiscoveryModuleTests.java | 21 ------------ .../SettingsBasedSeedHostsProviderTests.java | 18 ---------- .../elasticsearch/license/LicensingTests.java | 2 +- 11 files changed, 4 insertions(+), 141 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java index fa58dd240fa82..526503aefd552 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java @@ -49,9 +49,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.unmodifiableSet; import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING; -import static org.elasticsearch.discovery.DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING; import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; -import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING; public class ClusterBootstrapService { @@ -93,8 +91,7 @@ public ClusterBootstrapService(Settings settings, TransportService transportServ } public static boolean discoveryIsConfigured(Settings settings) { - return Stream.of(DISCOVERY_SEED_PROVIDERS_SETTING, LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING, - DISCOVERY_SEED_HOSTS_SETTING, LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING, + return Stream.of(DISCOVERY_SEED_PROVIDERS_SETTING, DISCOVERY_SEED_HOSTS_SETTING, INITIAL_MASTER_NODES_SETTING).anyMatch(s -> s.exists(settings)); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java b/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java index 2944c3bb23283..294b3b9f47ccc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java @@ -35,9 +35,6 @@ public class NoMasterBlockService { public static final ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, false, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL); - public static final Setting LEGACY_NO_MASTER_BLOCK_SETTING = - new Setting<>("discovery.zen.no_master_block", "write", NoMasterBlockService::parseNoMasterBlock, - Property.Dynamic, Property.NodeScope, Property.Deprecated); public static final Setting NO_MASTER_BLOCK_SETTING = new Setting<>("cluster.no_master_block", "write", NoMasterBlockService::parseNoMasterBlock, Property.Dynamic, Property.NodeScope); @@ -47,9 +44,6 @@ public class NoMasterBlockService { public NoMasterBlockService(Settings settings, ClusterSettings clusterSettings) { this.noMasterBlock = NO_MASTER_BLOCK_SETTING.get(settings); clusterSettings.addSettingsUpdateConsumer(NO_MASTER_BLOCK_SETTING, this::setNoMasterBlock); - - LEGACY_NO_MASTER_BLOCK_SETTING.get(settings); // for deprecation warnings - clusterSettings.addSettingsUpdateConsumer(LEGACY_NO_MASTER_BLOCK_SETTING, b -> {}); // for deprecation warnings } private static ClusterBlock parseNoMasterBlock(String value) { diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index a60648f773c37..5359d11cdbc60 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -230,7 +230,6 @@ public void apply(Settings value, Settings current, Settings previous) { DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING, DiscoverySettings.COMMIT_TIMEOUT_SETTING, NoMasterBlockService.NO_MASTER_BLOCK_SETTING, - NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING, GatewayService.EXPECTED_DATA_NODES_SETTING, GatewayService.EXPECTED_MASTER_NODES_SETTING, GatewayService.EXPECTED_NODES_SETTING, @@ -385,13 +384,9 @@ public void apply(Settings value, Settings current, Settings previous) { DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, - DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, - SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, - SeedHostsResolver.LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING, - SeedHostsResolver.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT, SearchService.DEFAULT_KEEPALIVE_SETTING, SearchService.KEEPALIVE_INTERVAL_SETTING, SearchService.MAX_KEEPALIVE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 8f8189676ff0f..d680837bf1b8c 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -71,9 +71,6 @@ public class DiscoveryModule { public static final Setting DISCOVERY_TYPE_SETTING = new Setting<>("discovery.type", ZEN2_DISCOVERY_TYPE, Function.identity(), Property.NodeScope); - public static final Setting> LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING = - Setting.listSetting("discovery.zen.hosts_provider", Collections.emptyList(), Function.identity(), - Property.NodeScope, Property.Deprecated); public static final Setting> DISCOVERY_SEED_PROVIDERS_SETTING = Setting.listSetting("discovery.seed_providers", Collections.emptyList(), Function.identity(), Property.NodeScope); @@ -100,7 +97,7 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic } } - List seedProviderNames = getSeedProviderNames(settings); + List seedProviderNames = DISCOVERY_SEED_PROVIDERS_SETTING.get(settings); // for bwc purposes, add settings provider even if not explicitly specified if (seedProviderNames.contains("settings") == false) { List extendedSeedProviderNames = new ArrayList<>(); @@ -142,17 +139,6 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic discovery = Objects.requireNonNull(discoverySupplier.get()); } - private List getSeedProviderNames(Settings settings) { - if (LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.exists(settings)) { - if (DISCOVERY_SEED_PROVIDERS_SETTING.exists(settings)) { - throw new IllegalArgumentException("it is forbidden to set both [" + DISCOVERY_SEED_PROVIDERS_SETTING.getKey() + "] and [" - + LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.getKey() + "]"); - } - return LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.get(settings); - } - return DISCOVERY_SEED_PROVIDERS_SETTING.get(settings); - } - public Discovery getDiscovery() { return discovery; } diff --git a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java index 926216b9b686e..769c573c60793 100644 --- a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java +++ b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java @@ -52,12 +52,6 @@ import java.util.stream.Collectors; public class SeedHostsResolver extends AbstractLifecycleComponent implements ConfiguredHostsResolver { - public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING = - Setting.intSetting("discovery.zen.ping.unicast.concurrent_connects", 10, 0, Setting.Property.NodeScope, - Setting.Property.Deprecated); - public static final Setting LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT = - Setting.positiveTimeSetting("discovery.zen.ping.unicast.hosts.resolve_timeout", TimeValue.timeValueSeconds(5), - Setting.Property.NodeScope, Setting.Property.Deprecated); public static final Setting DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING = Setting.intSetting("discovery.seed_resolver.max_concurrent_resolvers", 10, 0, Setting.Property.NodeScope); public static final Setting DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING = @@ -85,26 +79,10 @@ public SeedHostsResolver(String nodeName, Settings settings, TransportService tr } public static int getMaxConcurrentResolvers(Settings settings) { - if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.exists(settings)) { - if (DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.exists(settings)) { - throw new IllegalArgumentException("it is forbidden to set both [" - + DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.getKey() + "] and [" - + LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.getKey() + "]"); - } - return LEGACY_DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings); - } return DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.get(settings); } public static TimeValue getResolveTimeout(Settings settings) { - if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.exists(settings)) { - if (DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.exists(settings)) { - throw new IllegalArgumentException("it is forbidden to set both [" - + DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.getKey() + "] and [" - + LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.getKey() + "]"); - } - return LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_RESOLVE_TIMEOUT.get(settings); - } return DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.get(settings); } diff --git a/server/src/main/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProvider.java b/server/src/main/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProvider.java index d0c8a5c65dff0..b3b3ca27894a5 100644 --- a/server/src/main/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProvider.java +++ b/server/src/main/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProvider.java @@ -44,9 +44,6 @@ public class SettingsBasedSeedHostsProvider implements SeedHostsProvider { private static final Logger logger = LogManager.getLogger(SettingsBasedSeedHostsProvider.class); - public static final Setting> LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING = - Setting.listSetting("discovery.zen.ping.unicast.hosts", emptyList(), Function.identity(), Property.NodeScope, Property.Deprecated); - public static final Setting> DISCOVERY_SEED_HOSTS_SETTING = Setting.listSetting("discovery.seed_hosts", emptyList(), Function.identity(), Property.NodeScope); @@ -58,16 +55,7 @@ public class SettingsBasedSeedHostsProvider implements SeedHostsProvider { private final int limitPortCounts; public SettingsBasedSeedHostsProvider(Settings settings, TransportService transportService) { - if (LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.exists(settings)) { - if (DISCOVERY_SEED_HOSTS_SETTING.exists(settings)) { - throw new IllegalArgumentException("it is forbidden to set both [" - + DISCOVERY_SEED_HOSTS_SETTING.getKey() + "] and [" - + LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey() + "]"); - } - configuredHosts = LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.get(settings); - // we only limit to 1 address, makes no sense to ping 100 ports - limitPortCounts = LIMIT_FOREIGN_PORTS_COUNT; - } else if (DISCOVERY_SEED_HOSTS_SETTING.exists(settings)) { + if (DISCOVERY_SEED_HOSTS_SETTING.exists(settings)) { configuredHosts = DISCOVERY_SEED_HOSTS_SETTING.get(settings); // we only limit to 1 address, makes no sense to ping 100 ports limitPortCounts = LIMIT_FOREIGN_PORTS_COUNT; diff --git a/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java b/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java index 5d8cebc319623..e6ecb54df9f07 100644 --- a/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java +++ b/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java @@ -730,8 +730,5 @@ public void testDiscoveryConfiguredCheck() throws NodeValidationException { ensureChecksPass.accept(Settings.builder().putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey())); ensureChecksPass.accept(Settings.builder().putList(DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING.getKey())); ensureChecksPass.accept(Settings.builder().putList(SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING.getKey())); - ensureChecksPass.accept(Settings.builder().putList(DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.getKey())); - ensureChecksPass.accept(Settings.builder().putList(SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING - .getKey())); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java index 990fd8a01469d..df73a2542e6c2 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java @@ -25,7 +25,6 @@ import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ALL; import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_SETTING; import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_WRITES; -import static org.elasticsearch.cluster.coordination.NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING; import static org.elasticsearch.common.settings.ClusterSettings.BUILT_IN_CLUSTER_SETTINGS; import static org.hamcrest.Matchers.sameInstance; @@ -39,33 +38,16 @@ private void createService(Settings settings) { noMasterBlockService = new NoMasterBlockService(settings, clusterSettings); } - private void assertDeprecatedWarningEmitted() { - assertWarnings("[discovery.zen.no_master_block] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version."); - } - public void testBlocksWritesByDefault() { createService(Settings.EMPTY); assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); } - public void testIgnoresLegacySettingBlockingWrites() { - createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); - assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); - assertDeprecatedWarningEmitted(); - } - public void testBlocksWritesIfConfiguredBySetting() { createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); } - public void testIgnoresLegacySettingBlockingAll() { - createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); - assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); - assertDeprecatedWarningEmitted(); - } - public void testBlocksAllIfConfiguredBySetting() { createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); @@ -76,12 +58,6 @@ public void testRejectsInvalidSetting() { createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "unknown").build())); } - public void testRejectsInvalidLegacySetting() { - expectThrows(IllegalArgumentException.class, () -> - createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "unknown").build())); - assertDeprecatedWarningEmitted(); - } - public void testSettingCanBeUpdated() { createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); @@ -89,13 +65,4 @@ public void testSettingCanBeUpdated() { clusterSettings.applySettings(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); } - - public void testIgnoresUpdatesToLegacySetting() { - createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); - assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); - - clusterSettings.applySettings(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); - assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); - assertDeprecatedWarningEmitted(); - } } diff --git a/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java b/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java index fe74a736fe3ba..97e966533122d 100644 --- a/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java @@ -116,27 +116,6 @@ public void testSeedProviders() { assertTrue(created.get()); } - public void testLegacyHostsProvider() { - Settings settings = Settings.builder().put(DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.getKey(), "custom").build(); - AtomicBoolean created = new AtomicBoolean(false); - DummyHostsProviderPlugin plugin = () -> Collections.singletonMap("custom", () -> { - created.set(true); - return hostsResolver -> Collections.emptyList(); - }); - newModule(settings, Collections.singletonList(plugin)); - assertTrue(created.get()); - assertWarnings("[discovery.zen.hosts_provider] setting was deprecated in Elasticsearch and will be removed in a future release! " + - "See the breaking changes documentation for the next major version."); - } - - public void testLegacyAndNonLegacyProvidersRejected() { - Settings settings = Settings.builder().putList(DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING.getKey()) - .putList(DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.getKey()).build(); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> - newModule(settings, Collections.emptyList())); - assertEquals("it is forbidden to set both [discovery.seed_providers] and [discovery.zen.hosts_provider]", e.getMessage()); - } - public void testUnknownSeedsProvider() { Settings settings = Settings.builder().put(DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING.getKey(), "dne").build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> diff --git a/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderTests.java b/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderTests.java index f001d69ecc831..d98e152149382 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.discovery.SeedHostsProvider.HostsResolver; -import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.TransportService; @@ -75,21 +74,4 @@ public void testGetsHostsFromSetting() { .build(), null).getSeedAddresses(hostsResolver); assertTrue(hostsResolver.getResolvedHosts()); } - - public void testGetsHostsFromLegacySetting() { - final AssertingHostsResolver hostsResolver = new AssertingHostsResolver(1, "bar", "foo"); - new SettingsBasedSeedHostsProvider(Settings.builder() - .putList(SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey(), "foo", "bar") - .build(), null).getSeedAddresses(hostsResolver); - assertTrue(hostsResolver.getResolvedHosts()); - assertWarnings("[discovery.zen.ping.unicast.hosts] setting was deprecated in Elasticsearch and will be removed in a future " + - "release! See the breaking changes documentation for the next major version."); - } - - public void testForbidsBothSettingsAtTheSameTime() { - expectThrows(IllegalArgumentException.class, () -> new SettingsBasedSeedHostsProvider(Settings.builder() - .putList(SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey()) - .putList(SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING.getKey()) - .build(), null)); - } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java index 23bcef624ac8d..9fb044d6202ed 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java @@ -66,7 +66,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; -@TestLogging("org.elasticsearch.cluster.service:TRACE,org.elasticsearch.discovery.zen:TRACE,org.elasticsearch.action.search:TRACE," + +@TestLogging("org.elasticsearch.cluster.service:TRACE,org.elasticsearch.action.search:TRACE," + "org.elasticsearch.search:TRACE") public class LicensingTests extends SecurityIntegTestCase { private static final String ROLES = From 6a4e69d8656b8ecbd6fadd636a5679e55af05cf6 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 16:42:06 +0100 Subject: [PATCH 17/30] remove DiscoverySettings --- .../common/settings/ClusterSettings.java | 6 +- .../discovery/DiscoverySettings.java | 93 ----- .../java/org/elasticsearch/node/Node.java | 6 +- .../org/elasticsearch/cluster/ack/AckIT.java | 317 ------------------ .../ElasticsearchNodeCommandIT.java | 11 +- .../discovery/DiscoveryDisruptionIT.java | 2 +- .../test/InternalTestCluster.java | 2 +- .../test/test/InternalTestClusterTests.java | 4 +- 8 files changed, 14 insertions(+), 427 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java delete mode 100644 server/src/test/java/org/elasticsearch/cluster/ack/AckIT.java diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 5359d11cdbc60..3b4dd5cac88e4 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -64,7 +64,6 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.PeerFinder; import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; @@ -226,9 +225,6 @@ public void apply(Settings value, Settings current, Settings previous) { InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, DestructiveOperations.REQUIRES_NAME_SETTING, - DiscoverySettings.PUBLISH_TIMEOUT_SETTING, - DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING, - DiscoverySettings.COMMIT_TIMEOUT_SETTING, NoMasterBlockService.NO_MASTER_BLOCK_SETTING, GatewayService.EXPECTED_DATA_NODES_SETTING, GatewayService.EXPECTED_MASTER_NODES_SETTING, @@ -381,7 +377,7 @@ public void apply(Settings value, Settings current, Settings previous) { Environment.PATH_SHARED_DATA_SETTING, Environment.PIDFILE_SETTING, NodeEnvironment.NODE_ID_SEED_SETTING, - DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING, + Node.INITIAL_STATE_TIMEOUT_SETTING, DiscoveryModule.DISCOVERY_TYPE_SETTING, DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java b/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java deleted file mode 100644 index e1a0c20864ea3..0000000000000 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.discovery; - -import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; - -/** - * Exposes common discovery settings that may be supported by all the different discovery implementations - */ -public class DiscoverySettings { - - /** - * sets the timeout for a complete publishing cycle, including both sending and committing. the master - * will continue to process the next cluster state update after this time has elapsed - **/ - public static final Setting PUBLISH_TIMEOUT_SETTING = - Setting.positiveTimeSetting("discovery.zen.publish_timeout", TimeValue.timeValueSeconds(30), - Property.Dynamic, Property.NodeScope, Property.Deprecated); - - /** - * sets the timeout for receiving enough acks for a specific cluster state and committing it. failing - * to receive responses within this window will cause the cluster state change to be rejected. - */ - public static final Setting COMMIT_TIMEOUT_SETTING = - new Setting<>("discovery.zen.commit_timeout", PUBLISH_TIMEOUT_SETTING::getRaw, - (s) -> TimeValue.parseTimeValue(s, TimeValue.timeValueSeconds(30), "discovery.zen.commit_timeout"), - Property.Dynamic, Property.NodeScope, Property.Deprecated); - public static final Setting PUBLISH_DIFF_ENABLE_SETTING = - Setting.boolSetting("discovery.zen.publish_diff.enable", true, Property.Dynamic, Property.NodeScope, Property.Deprecated); - public static final Setting INITIAL_STATE_TIMEOUT_SETTING = - Setting.positiveTimeSetting("discovery.initial_state_timeout", TimeValue.timeValueSeconds(30), Property.NodeScope); - - private volatile TimeValue publishTimeout; - - private volatile TimeValue commitTimeout; - private volatile boolean publishDiff; - - public DiscoverySettings(Settings settings, ClusterSettings clusterSettings) { - clusterSettings.addSettingsUpdateConsumer(PUBLISH_DIFF_ENABLE_SETTING, this::setPublishDiff); - clusterSettings.addSettingsUpdateConsumer(COMMIT_TIMEOUT_SETTING, this::setCommitTimeout); - clusterSettings.addSettingsUpdateConsumer(PUBLISH_TIMEOUT_SETTING, this::setPublishTimeout); - this.publishTimeout = PUBLISH_TIMEOUT_SETTING.get(settings); - this.commitTimeout = COMMIT_TIMEOUT_SETTING.get(settings); - this.publishDiff = PUBLISH_DIFF_ENABLE_SETTING.get(settings); - } - - /** - * Returns the current publish timeout - */ - public TimeValue getPublishTimeout() { - return publishTimeout; - } - - public TimeValue getCommitTimeout() { - return commitTimeout; - } - - private void setPublishDiff(boolean publishDiff) { - this.publishDiff = publishDiff; - } - - private void setPublishTimeout(TimeValue publishTimeout) { - this.publishTimeout = publishTimeout; - } - - private void setCommitTimeout(TimeValue commitTimeout) { - this.commitTimeout = commitTimeout; - } - - public boolean getPublishDiff() { return publishDiff;} - -} diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 19af7a467a789..5d69ba505ee9d 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -86,7 +86,6 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.GatewayAllocator; @@ -228,6 +227,9 @@ public class Node implements Closeable { } }, Setting.Property.NodeScope); + public static final Setting INITIAL_STATE_TIMEOUT_SETTING = + Setting.positiveTimeSetting("discovery.initial_state_timeout", TimeValue.timeValueSeconds(30), Property.NodeScope); + private static final String CLIENT_TYPE = "node"; private final Lifecycle lifecycle = new Lifecycle(); @@ -683,7 +685,7 @@ public Node start() throws NodeValidationException { : "clusterService has a different local node than the factory provided"; transportService.acceptIncomingRequests(); discovery.startInitialJoin(); - final TimeValue initialStateTimeout = DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings); + final TimeValue initialStateTimeout = INITIAL_STATE_TIMEOUT_SETTING.get(settings); configureNodeAndClusterIdStateListener(clusterService); if (initialStateTimeout.millis() > 0) { diff --git a/server/src/test/java/org/elasticsearch/cluster/ack/AckIT.java b/server/src/test/java/org/elasticsearch/cluster/ack/AckIT.java deleted file mode 100644 index edad8494f54de..0000000000000 --- a/server/src/test/java/org/elasticsearch/cluster/ack/AckIT.java +++ /dev/null @@ -1,317 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.cluster.ack; - -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; -import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.AliasMetaData; -import org.elasticsearch.cluster.metadata.AliasOrIndex; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.IndexMetaData.State; -import org.elasticsearch.cluster.routing.RoutingNode; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ClusterScope; - -import java.util.concurrent.TimeUnit; - -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.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.anyOf; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; - -@ClusterScope(minNumDataNodes = 2) -@AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/32767") -public class AckIT extends ESIntegTestCase { - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - //to test that the acknowledgement mechanism is working we better disable the wait for publish - //otherwise the operation is most likely acknowledged even if it doesn't support ack - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "30s") // explicitly set so it won't default to publish timeout - .put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "0s") // don't wait post commit to check acking - .build(); - } - - public void testUpdateSettingsAcknowledgement() { - createIndex("test"); - - assertAcked(client().admin().indices().prepareUpdateSettings("test") - .setSettings(Settings.builder().put("refresh_interval", 9999, TimeUnit.MILLISECONDS))); - - for (Client client : clients()) { - String refreshInterval = getLocalClusterState(client).metaData().index("test").getSettings().get("index.refresh_interval"); - assertThat(refreshInterval, equalTo("9999ms")); - } - } - - public void testUpdateSettingsNoAcknowledgement() { - createIndex("test"); - AcknowledgedResponse updateSettingsResponse = client().admin().indices().prepareUpdateSettings("test").setTimeout("0s") - .setSettings(Settings.builder().put("refresh_interval", 9999, TimeUnit.MILLISECONDS)).get(); - assertThat(updateSettingsResponse.isAcknowledged(), equalTo(false)); - } - - public void testClusterRerouteAcknowledgement() throws InterruptedException { - assertAcked(prepareCreate("test").setSettings(Settings.builder() - .put(indexSettings()) - .put(SETTING_NUMBER_OF_SHARDS, between(cluster().numDataNodes(), DEFAULT_MAX_NUM_SHARDS)) - .put(SETTING_NUMBER_OF_REPLICAS, 0) - )); - ensureGreen(); - - MoveAllocationCommand moveAllocationCommand = getAllocationCommand(); - final Index index = client().admin().cluster().prepareState().get().getState().metaData().index("test").getIndex(); - final ShardId commandShard = new ShardId(index, moveAllocationCommand.shardId()); - - assertAcked(client().admin().cluster().prepareReroute().add(moveAllocationCommand)); - - for (Client client : clients()) { - ClusterState clusterState = getLocalClusterState(client); - for (ShardRouting shardRouting : clusterState.getRoutingNodes().node(moveAllocationCommand.fromNode())) { - //if the shard that we wanted to move is still on the same node, it must be relocating - if (shardRouting.shardId().equals(commandShard)) { - assertThat(shardRouting.relocating(), equalTo(true)); - } - - } - - boolean found = false; - for (ShardRouting shardRouting : clusterState.getRoutingNodes().node(moveAllocationCommand.toNode())) { - if (shardRouting.shardId().equals(commandShard)) { - assertThat(shardRouting.state(), anyOf(equalTo(ShardRoutingState.INITIALIZING), equalTo(ShardRoutingState.STARTED))); - found = true; - break; - } - } - assertThat(found, equalTo(true)); - } - } - - public void testClusterRerouteNoAcknowledgement() throws InterruptedException { - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, between(cluster().numDataNodes(), DEFAULT_MAX_NUM_SHARDS)) - .put(SETTING_NUMBER_OF_REPLICAS, 0)).get(); - ensureGreen(); - - MoveAllocationCommand moveAllocationCommand = getAllocationCommand(); - - ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute() - .setTimeout("0s").add(moveAllocationCommand).get(); - assertThat(clusterRerouteResponse.isAcknowledged(), equalTo(false)); - } - - public void testClusterRerouteAcknowledgementDryRun() throws InterruptedException { - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, between(cluster().numDataNodes(), DEFAULT_MAX_NUM_SHARDS)) - .put(SETTING_NUMBER_OF_REPLICAS, 0)).get(); - ensureGreen(); - - MoveAllocationCommand moveAllocationCommand = getAllocationCommand(); - - final Index index = client().admin().cluster().prepareState().get().getState().metaData().index("test").getIndex(); - final ShardId commandShard = new ShardId(index, moveAllocationCommand.shardId()); - - assertAcked(client().admin().cluster().prepareReroute().setDryRun(true).add(moveAllocationCommand)); - - // testing only on master with the latest cluster state as we didn't make any change thus - // we cannot guarantee that all nodes hold the same cluster state version. We only know there - // was no need to change anything, thus no need for ack on this update. - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get(); - boolean found = false; - for (ShardRouting shardRouting : clusterStateResponse.getState().getRoutingNodes().node(moveAllocationCommand.fromNode())) { - //the shard that we wanted to move is still on the same node, as we had dryRun flag - if (shardRouting.shardId().equals(commandShard)) { - assertThat(shardRouting.started(), equalTo(true)); - found = true; - break; - } - } - assertThat(found, equalTo(true)); - - for (ShardRouting shardRouting : clusterStateResponse.getState().getRoutingNodes().node(moveAllocationCommand.toNode())) { - if (shardRouting.shardId().equals(commandShard)) { - fail("shard [" + shardRouting + "] shouldn't be on node [" + moveAllocationCommand.toString() + "]"); - } - } - } - - public void testClusterRerouteNoAcknowledgementDryRun() throws InterruptedException { - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, between(cluster().numDataNodes(), DEFAULT_MAX_NUM_SHARDS)) - .put(SETTING_NUMBER_OF_REPLICAS, 0)).get(); - ensureGreen(); - - MoveAllocationCommand moveAllocationCommand = getAllocationCommand(); - - ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setTimeout("0s") - .setDryRun(true).add(moveAllocationCommand).get(); - //acknowledged anyway as no changes were made - assertThat(clusterRerouteResponse.isAcknowledged(), equalTo(true)); - } - - private MoveAllocationCommand getAllocationCommand() { - String fromNodeId = null; - String toNodeId = null; - ShardRouting shardToBeMoved = null; - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get(); - for (RoutingNode routingNode : clusterStateResponse.getState().getRoutingNodes()) { - if (routingNode.node().isDataNode()) { - if (fromNodeId == null && routingNode.numberOfOwningShards() > 0) { - fromNodeId = routingNode.nodeId(); - shardToBeMoved = routingNode.copyShards().get(randomInt(routingNode.size() - 1)); - } else { - toNodeId = routingNode.nodeId(); - } - - if (toNodeId != null && fromNodeId != null) { - break; - } - } - } - - assertNotNull(fromNodeId); - assertNotNull(toNodeId); - assertNotNull(shardToBeMoved); - - logger.info("==> going to move shard [{}] from [{}] to [{}]", shardToBeMoved, fromNodeId, toNodeId); - return new MoveAllocationCommand(shardToBeMoved.getIndexName(), shardToBeMoved.id(), fromNodeId, toNodeId); - } - - public void testIndicesAliasesAcknowledgement() { - createIndex("test"); - - //testing acknowledgement when trying to submit an existing alias too - //in that case it would not make any change, but we are sure about the cluster state - //as the previous operation was acknowledged - for (int i = 0; i < 2; i++) { - assertAcked(client().admin().indices().prepareAliases().addAlias("test", "alias")); - - for (Client client : clients()) { - AliasMetaData aliasMetaData = ((AliasOrIndex.Alias) getLocalClusterState(client) - .metaData().getAliasAndIndexLookup().get("alias")).getFirstAliasMetaData(); - assertThat(aliasMetaData.alias(), equalTo("alias")); - } - } - } - - public void testIndicesAliasesNoAcknowledgement() { - createIndex("test"); - - AcknowledgedResponse indicesAliasesResponse = client().admin().indices().prepareAliases() - .addAlias("test", "alias").setTimeout("0s").get(); - assertThat(indicesAliasesResponse.isAcknowledged(), equalTo(false)); - } - - public void testCloseIndexAcknowledgement() { - createIndex("test"); - ensureGreen(); - - assertAcked(client().admin().indices().prepareClose("test")); - - for (Client client : clients()) { - IndexMetaData indexMetaData = getLocalClusterState(client).metaData().indices().get("test"); - assertThat(indexMetaData.getState(), equalTo(State.CLOSE)); - } - } - - public void testCloseIndexNoAcknowledgement() { - createIndex("test"); - ensureGreen(); - - AcknowledgedResponse closeIndexResponse = client().admin().indices().prepareClose("test").setTimeout("0s").get(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(false)); - } - - public void testOpenIndexAcknowledgement() { - createIndex("test"); - ensureGreen(); - - assertAcked(client().admin().indices().prepareClose("test")); - - assertAcked(client().admin().indices().prepareOpen("test")); - - for (Client client : clients()) { - IndexMetaData indexMetaData = getLocalClusterState(client).metaData().indices().get("test"); - assertThat(indexMetaData.getState(), equalTo(State.OPEN)); - } - } - - public void testPutMappingAcknowledgement() { - createIndex("test"); - ensureGreen(); - - assertAcked(client().admin().indices().preparePutMapping("test").setType("test").setSource("field", "type=keyword")); - - for (Client client : clients()) { - assertThat(getLocalClusterState(client).metaData().indices().get("test").getMappings().get("test"), notNullValue()); - } - } - - public void testPutMappingNoAcknowledgement() { - createIndex("test"); - ensureGreen(); - - AcknowledgedResponse putMappingResponse = client().admin().indices().preparePutMapping("test").setType("test") - .setSource("field", "type=keyword").setTimeout("0s").get(); - assertThat(putMappingResponse.isAcknowledged(), equalTo(false)); - } - - public void testCreateIndexAcknowledgement() { - createIndex("test"); - - for (Client client : clients()) { - assertThat(getLocalClusterState(client).metaData().indices().containsKey("test"), equalTo(true)); - } - - //let's wait for green, otherwise there can be issues with after test checks (mock directory wrapper etc.) - //but we do want to check that the new index is on all nodes cluster state even before green - ensureGreen(); - } - - public void testCreateIndexNoAcknowledgement() { - CreateIndexResponse createIndexResponse = client().admin().indices().prepareCreate("test").setTimeout("0s").get(); - assertThat(createIndexResponse.isAcknowledged(), equalTo(false)); - - //let's wait for green, otherwise there can be issues with after test checks (mock directory wrapper etc.) - ensureGreen(); - } - - private static ClusterState getLocalClusterState(Client client) { - return client.admin().cluster().prepareState().setLocal(true).get().getState(); - } -} diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java index 6fe7b65fad798..6003e949dfb5e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java @@ -26,7 +26,6 @@ import org.elasticsearch.cluster.metadata.Manifest; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeMetaData; @@ -149,7 +148,7 @@ public void testBootstrapNoNodeMetaData() throws IOException { public void testBootstrapNotBootstrappedCluster() throws Exception { internalCluster().startNode( Settings.builder() - .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup + .put(Node.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup .build()); assertBusy(() -> { ClusterState state = client().admin().cluster().prepareState().setLocal(true) @@ -166,7 +165,7 @@ public void testBootstrapNotBootstrappedCluster() throws Exception { public void testDetachNotBootstrappedCluster() throws Exception { internalCluster().startNode( Settings.builder() - .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup + .put(Node.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup .build()); assertBusy(() -> { ClusterState state = client().admin().cluster().prepareState().setLocal(true) @@ -256,12 +255,12 @@ public void test3MasterNodes2Failed() throws Exception { logger.info("--> start 1st master-eligible node"); masterNodes.add(internalCluster().startMasterOnlyNode(Settings.builder() - .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") + .put(Node.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") .build())); // node ordinal 0 logger.info("--> start one data-only node"); String dataNode = internalCluster().startDataOnlyNode(Settings.builder() - .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") + .put(Node.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") .build()); // node ordinal 1 logger.info("--> start 2nd and 3rd master-eligible nodes and bootstrap"); @@ -385,7 +384,7 @@ public void testNoInitialBootstrapAfterDetach() throws Exception { String node = internalCluster().startMasterOnlyNode(Settings.builder() // give the cluster 2 seconds to elect the master (it should not) - .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "2s") + .put(Node.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "2s") .build()); ClusterState state = internalCluster().client().admin().cluster().prepareState().setLocal(true) diff --git a/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java index 85e14e1decc2a..923af536cbb58 100644 --- a/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java @@ -110,7 +110,7 @@ public void testClusterFormingWithASlowNode() { SlowClusterStateProcessing disruption = new SlowClusterStateProcessing(random(), 0, 0, 1000, 2000); // don't wait for initial state, we want to add the disruption while the cluster is forming - internalCluster().startNodes(3, Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "3s").build()); + internalCluster().startNodes(3); logger.info("applying disruption while cluster is forming ..."); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 8d92f80f62e4a..cae76718378aa 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -149,7 +149,7 @@ import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_TYPE_SETTING; import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; -import static org.elasticsearch.discovery.DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING; +import static org.elasticsearch.node.Node.INITIAL_STATE_TIMEOUT_SETTING; import static org.elasticsearch.discovery.FileBasedSeedHostsProvider.UNICAST_HOSTS_FILE; import static org.elasticsearch.test.ESTestCase.assertBusy; import static org.elasticsearch.test.ESTestCase.awaitBusy; diff --git a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java index 1a5f7edf4227c..8461e6ade09ac 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java @@ -27,9 +27,9 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.discovery.DiscoveryModule; -import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.SettingsBasedSeedHostsProvider; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalTestCluster; @@ -335,7 +335,7 @@ public Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), numNodes) .put(NetworkModule.TRANSPORT_TYPE_KEY, getTestTransportType()) - .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0) + .put(Node.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0) .putList(DISCOVERY_SEED_PROVIDERS_SETTING.getKey(), "file") .putList(SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING.getKey()) .build(); From e5abf4aa361a070212ebc6d5c480a3469c608bd2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 16:54:09 +0100 Subject: [PATCH 18/30] add extra settings --- docs/reference/migration/migrate_8_0/discovery.asciidoc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/reference/migration/migrate_8_0/discovery.asciidoc b/docs/reference/migration/migrate_8_0/discovery.asciidoc index a078370b8509d..d616c2d50284a 100644 --- a/docs/reference/migration/migrate_8_0/discovery.asciidoc +++ b/docs/reference/migration/migrate_8_0/discovery.asciidoc @@ -10,6 +10,9 @@ will no longer be supported. In particular, this includes: - `discovery.zen.no_master_block` - `discovery.zen.hosts_provider` +- `discovery.zen.publish_timeout` +- `discovery.zen.commit_timeout` +- `discovery.zen.publish_diff.enable` - `discovery.zen.ping.unicast.concurrent_connects` - `discovery.zen.ping.unicast.hosts.resolve_timeout` - `discovery.zen.ping.unicast.hosts` From e16720f1ed49f889dd3fe26eb3fad7aa7a048df2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 17:05:52 +0100 Subject: [PATCH 19/30] oops delete frenzy --- .../elasticsearch/gradle/test/ClusterConfiguration.groovy | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy index 1e3a200dba874..4afb2b03afa16 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy @@ -63,6 +63,14 @@ class ClusterConfiguration { @Input boolean debug = false + /** + * Whether the initial_master_nodes setting should be automatically derived from the nodes + * in the cluster. Only takes effect if all nodes in the cluster understand this setting + * and the discovery type is not explicitly set. + */ + @Input + boolean autoSetInitialMasterNodes = true + /** * Whether the file-based discovery provider should be automatically setup based on * the nodes in the cluster. Only takes effect if no other hosts provider is already From f7cd1b814141027cc1c614d11c65158a31a2e04b Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 27 Feb 2019 19:04:59 +0100 Subject: [PATCH 20/30] too much delete --- qa/unconfigured-node-name/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/qa/unconfigured-node-name/build.gradle b/qa/unconfigured-node-name/build.gradle index 45d8a15f46744..5aba0562e03f6 100644 --- a/qa/unconfigured-node-name/build.gradle +++ b/qa/unconfigured-node-name/build.gradle @@ -24,6 +24,7 @@ integTestCluster { setting 'node.name', null // Run with no discovery configuration at all, demonstrating that a node in its // "out-of-the-box" configuration can automatically bootstrap a cluster + autoSetInitialMasterNodes = false autoSetHostsProvider = false } From 799c335154814e5d2c427fe6222c1d75b8fe67b2 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 1 Mar 2019 11:37:24 +0100 Subject: [PATCH 21/30] remove more mentions of zen --- .../java/org/elasticsearch/cluster/MinimumMasterNodesIT.java | 2 +- .../cluster/coordination/ElasticsearchNodeCommandIT.java | 2 +- .../org/elasticsearch/test/transport/MockTransportService.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java b/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java index 4df8ecaf77b12..8f395c2d137a0 100644 --- a/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java @@ -60,7 +60,7 @@ import static org.hamcrest.Matchers.nullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0, autoMinMasterNodes = false) -@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.discovery.zen:TRACE") +@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE") public class MinimumMasterNodesIT extends ESIntegTestCase { @Override diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java index 6003e949dfb5e..d93133ef78b6f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java @@ -48,7 +48,7 @@ import static org.hamcrest.Matchers.equalTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, autoMinMasterNodes = false) -@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.discovery.zen:TRACE") +@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.cluster.coordination:TRACE") public class ElasticsearchNodeCommandIT extends ESIntegTestCase { private MockTerminal executeCommand(ElasticsearchNodeCommand command, Environment environment, int nodeOrdinal, boolean abort) diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index e72bbf3a72897..d34a78ebb9e47 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -81,7 +81,7 @@ * Matching requests to rules is based on the delegate address associated with the * discovery node of the request, namely by DiscoveryNode.getAddress(). * This address is usually the publish address of the node but can also be a different one - * (for example, @see org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing, which constructs + * (for example, @see org.elasticsearch.discovery.HandshakingTransportAddressConnector, which constructs * fake DiscoveryNode instances where the publish address is one of the bound addresses). */ public final class MockTransportService extends TransportService { From ed74cd2c597b4575a1effb1a72208e0a95066683 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 1 Mar 2019 11:48:54 +0100 Subject: [PATCH 22/30] remove minimumMasterNodesOnPublishingMaster --- .../state/TransportClusterStateAction.java | 1 - .../elasticsearch/cluster/ClusterState.java | 53 +++++-------------- .../ClusterSerializationTests.java | 6 +-- 3 files changed, 14 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java index 3eb3f1746b4b6..d35a41105aa0d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -126,7 +126,6 @@ private void buildResponse(final ClusterStateRequest request, ClusterState.Builder builder = ClusterState.builder(currentState.getClusterName()); builder.version(currentState.version()); builder.stateUUID(currentState.stateUUID()); - builder.minimumMasterNodesOnPublishingMaster(currentState.getMinimumMasterNodesOnPublishingMaster()); if (request.nodes()) { builder.nodes(currentState.nodes()); diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index 79493ea2e66e3..ce080433640d6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -177,19 +177,17 @@ default boolean isPrivate() { private final boolean wasReadFromDiff; - private final int minimumMasterNodesOnPublishingMaster; - // built on demand private volatile RoutingNodes routingNodes; public ClusterState(long version, String stateUUID, ClusterState state) { this(state.clusterName, version, stateUUID, state.metaData(), state.routingTable(), state.nodes(), state.blocks(), - state.customs(), -1, false); + state.customs(), false); } public ClusterState(ClusterName clusterName, long version, String stateUUID, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs, - int minimumMasterNodesOnPublishingMaster, boolean wasReadFromDiff) { + boolean wasReadFromDiff) { this.version = version; this.stateUUID = stateUUID; this.clusterName = clusterName; @@ -198,7 +196,6 @@ public ClusterState(ClusterName clusterName, long version, String stateUUID, Met this.nodes = nodes; this.blocks = blocks; this.customs = customs; - this.minimumMasterNodesOnPublishingMaster = minimumMasterNodesOnPublishingMaster; this.wasReadFromDiff = wasReadFromDiff; } @@ -292,22 +289,6 @@ public Set getVotingConfigExclusions() { return coordinationMetaData().getVotingConfigExclusions(); } - /** - * The node-level `discovery.zen.minimum_master_nodes` setting on the master node that published this cluster state, for use in rolling - * upgrades from 6.x to 7.x. Once all the 6.x master-eligible nodes have left the cluster, the 7.x nodes use this value to determine how - * many master-eligible nodes must be discovered before the cluster can be bootstrapped. Note that this method returns the node-level - * value of this setting, and ignores any cluster-level override that was set via the API. Callers are expected to combine this value - * with any value set in the cluster-level settings. This should be removed once we no longer need support for {@link Version#V_6_7_0}. - */ - public int getMinimumMasterNodesOnPublishingMaster() { - return minimumMasterNodesOnPublishingMaster; - } - - // Used for testing and logging to determine how this cluster state was send over the wire - public boolean wasReadFromDiff() { - return wasReadFromDiff; - } - /** * Returns a built (on demand) routing nodes view of the routing table. */ @@ -657,7 +638,6 @@ public static class Builder { private ClusterBlocks blocks = ClusterBlocks.EMPTY_CLUSTER_BLOCK; private final ImmutableOpenMap.Builder customs; private boolean fromDiff; - private int minimumMasterNodesOnPublishingMaster = -1; public Builder(ClusterState state) { this.clusterName = state.clusterName; @@ -668,7 +648,6 @@ public Builder(ClusterState state) { this.metaData = state.metaData(); this.blocks = state.blocks(); this.customs = ImmutableOpenMap.builder(state.customs()); - this.minimumMasterNodesOnPublishingMaster = state.minimumMasterNodesOnPublishingMaster; this.fromDiff = false; } @@ -729,11 +708,6 @@ public Builder stateUUID(String uuid) { return this; } - public Builder minimumMasterNodesOnPublishingMaster(int minimumMasterNodesOnPublishingMaster) { - this.minimumMasterNodesOnPublishingMaster = minimumMasterNodesOnPublishingMaster; - return this; - } - public Builder putCustom(String type, Custom custom) { customs.put(type, custom); return this; @@ -758,8 +732,7 @@ public ClusterState build() { if (UNKNOWN_UUID.equals(uuid)) { uuid = UUIDs.randomBase64UUID(); } - return new ClusterState(clusterName, version, uuid, metaData, routingTable, nodes, blocks, customs.build(), - minimumMasterNodesOnPublishingMaster, fromDiff); + return new ClusterState(clusterName, version, uuid, metaData, routingTable, nodes, blocks, customs.build(), fromDiff); } public static byte[] toBytes(ClusterState state) throws IOException { @@ -802,7 +775,9 @@ public static ClusterState readFrom(StreamInput in, DiscoveryNode localNode) thr Custom customIndexMetaData = in.readNamedWriteable(Custom.class); builder.putCustom(customIndexMetaData.getWriteableName(), customIndexMetaData); } - builder.minimumMasterNodesOnPublishingMaster = in.getVersion().onOrAfter(Version.V_6_7_0) ? in.readVInt() : -1; + if (in.getVersion().before(Version.V_8_0_0)) { + in.readVInt(); // used to be minimumMasterNodesOnPublishingMaster, which was used in 7.x for BWC with 6.x + } return builder.build(); } @@ -828,8 +803,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeNamedWriteable(cursor.value); } } - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeVInt(minimumMasterNodesOnPublishingMaster); + if (out.getVersion().before(Version.V_8_0_0)) { + out.writeVInt(-1); // used to be minimumMasterNodesOnPublishingMaster, which was used in 7.x for BWC with 6.x } } @@ -853,8 +828,6 @@ private static class ClusterStateDiff implements Diff { private final Diff> customs; - private final int minimumMasterNodesOnPublishingMaster; - ClusterStateDiff(ClusterState before, ClusterState after) { fromUuid = before.stateUUID; toUuid = after.stateUUID; @@ -865,7 +838,6 @@ private static class ClusterStateDiff implements Diff { metaData = after.metaData.diff(before.metaData); blocks = after.blocks.diff(before.blocks); customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer(), CUSTOM_VALUE_SERIALIZER); - minimumMasterNodesOnPublishingMaster = after.minimumMasterNodesOnPublishingMaster; } ClusterStateDiff(StreamInput in, DiscoveryNode localNode) throws IOException { @@ -878,7 +850,9 @@ private static class ClusterStateDiff implements Diff { metaData = MetaData.readDiffFrom(in); blocks = ClusterBlocks.readDiffFrom(in); customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), CUSTOM_VALUE_SERIALIZER); - minimumMasterNodesOnPublishingMaster = in.getVersion().onOrAfter(Version.V_6_7_0) ? in.readVInt() : -1; + if (in.getVersion().before(Version.V_8_0_0)) { + in.readVInt(); // used to be minimumMasterNodesOnPublishingMaster, which was used in 7.x for BWC with 6.x + } } @Override @@ -892,8 +866,8 @@ public void writeTo(StreamOutput out) throws IOException { metaData.writeTo(out); blocks.writeTo(out); customs.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeVInt(minimumMasterNodesOnPublishingMaster); + if (out.getVersion().before(Version.V_8_0_0)) { + out.writeVInt(-1); // used to be minimumMasterNodesOnPublishingMaster, which was used in 7.x for BWC with 6.x } } @@ -914,7 +888,6 @@ public ClusterState apply(ClusterState state) { builder.metaData(metaData.apply(state.metaData)); builder.blocks(blocks.apply(state.blocks)); builder.customs(customs.apply(state.customs)); - builder.minimumMasterNodesOnPublishingMaster(minimumMasterNodesOnPublishingMaster); builder.fromDiff(true); return builder.build(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index a8fa5e4de14c0..061defa600219 100644 --- a/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -67,8 +67,7 @@ public void testClusterStateSerialization() throws Exception { .add(newNode("node3")).localNodeId("node1").masterNodeId("node2").build(); ClusterState clusterState = ClusterState.builder(new ClusterName("clusterName1")) - .nodes(nodes).metaData(metaData).routingTable(routingTable) - .minimumMasterNodesOnPublishingMaster(randomIntBetween(-1, 10)).build(); + .nodes(nodes).metaData(metaData).routingTable(routingTable).build(); AllocationService strategy = createAllocationService(); clusterState = ClusterState.builder(clusterState).routingTable(strategy.reroute(clusterState, "reroute").routingTable()).build(); @@ -79,9 +78,6 @@ public void testClusterStateSerialization() throws Exception { assertThat(serializedClusterState.getClusterName().value(), equalTo(clusterState.getClusterName().value())); assertThat(serializedClusterState.routingTable().toString(), equalTo(clusterState.routingTable().toString())); - - assertThat(serializedClusterState.getMinimumMasterNodesOnPublishingMaster(), - equalTo(clusterState.getMinimumMasterNodesOnPublishingMaster())); } public void testRoutingTableSerialization() throws Exception { From 0bf8a12618c39855eba7cc028269ec43ae27d055 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 1 Mar 2019 12:29:04 +0100 Subject: [PATCH 23/30] refactor static method in SeedHostsResolver --- .../discovery/SeedHostsResolver.java | 35 +--- .../FileBasedSeedHostsProviderTests.java | 21 +-- .../discovery/SeedHostsResolverTests.java | 153 ++++++++++-------- 3 files changed, 107 insertions(+), 102 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java index 769c573c60793..5ba0402389aa6 100644 --- a/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java +++ b/server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java @@ -51,7 +51,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -public class SeedHostsResolver extends AbstractLifecycleComponent implements ConfiguredHostsResolver { +public class SeedHostsResolver extends AbstractLifecycleComponent implements ConfiguredHostsResolver, SeedHostsProvider.HostsResolver { public static final Setting DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING = Setting.intSetting("discovery.seed_resolver.max_concurrent_resolvers", 10, 0, Setting.Property.NodeScope); public static final Setting DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING = @@ -86,31 +86,11 @@ public static TimeValue getResolveTimeout(Settings settings) { return DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.get(settings); } - /** - * Resolves a list of hosts to a list of transport addresses. Each host is resolved into a transport address (or a collection of - * addresses if the number of ports is greater than one). Host lookups are done in parallel using specified executor service up - * to the specified resolve timeout. - * - * @param executorService the executor service used to parallelize hostname lookups - * @param logger logger used for logging messages regarding hostname lookups - * @param hosts the hosts to resolve - * @param limitPortCounts the number of ports to resolve (should be 1 for non-local transport) - * @param transportService the transport service - * @param resolveTimeout the timeout before returning from hostname lookups - * @return a list of resolved transport addresses - */ - public static List resolveHostsLists( - final ExecutorService executorService, - final Logger logger, + @Override + public List resolveHosts( final List hosts, - final int limitPortCounts, - final TransportService transportService, - final TimeValue resolveTimeout) { - Objects.requireNonNull(executorService); - Objects.requireNonNull(logger); + final int limitPortCounts) { Objects.requireNonNull(hosts); - Objects.requireNonNull(transportService); - Objects.requireNonNull(resolveTimeout); if (resolveTimeout.nanos() < 0) { throw new IllegalArgumentException("resolve timeout must be non-negative but was [" + resolveTimeout + "]"); } @@ -122,7 +102,7 @@ public static List resolveHostsLists( .collect(Collectors.toList()); final List> futures; try { - futures = executorService.invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS); + futures = executorService.get().invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return Collections.emptyList(); @@ -201,10 +181,7 @@ protected void doRun() { return; } - List providedAddresses - = hostsProvider.getSeedAddresses((hosts, limitPortCounts) - -> resolveHostsLists(executorService.get(), logger, hosts, limitPortCounts, - transportService, resolveTimeout)); + List providedAddresses = hostsProvider.getSeedAddresses(SeedHostsResolver.this); consumer.accept(providedAddresses); } diff --git a/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java b/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java index fc89bd1a2f30b..1f82b74a8e7ec 100644 --- a/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; @@ -114,11 +113,12 @@ public void testEmptyUnicastHostsFile() throws Exception { } public void testUnicastHostsDoesNotExist() { - final FileBasedSeedHostsProvider provider = new FileBasedSeedHostsProvider(createTempDir().toAbsolutePath()); - final List addresses = provider.getSeedAddresses((hosts, limitPortCounts) -> - SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService, - TimeValue.timeValueSeconds(10))); - assertEquals(0, addresses.size()); + final FileBasedSeedHostsProvider fileBasedSeedHostsProvider = new FileBasedSeedHostsProvider(createTempDir().toAbsolutePath()); + SeedHostsResolver seedHostsResolver = new SeedHostsResolver("test", Settings.EMPTY, transportService, fileBasedSeedHostsProvider); + seedHostsResolver.start(); + List results = fileBasedSeedHostsProvider.getSeedAddresses(seedHostsResolver); + seedHostsResolver.stop(); + assertEquals(0, results.size()); } public void testInvalidHostEntries() throws Exception { @@ -145,8 +145,11 @@ private List setupAndRunHostProvider(final List hostEn writer.write(String.join("\n", hostEntries)); } - return new FileBasedSeedHostsProvider(configPath).getSeedAddresses((hosts, limitPortCounts) -> - SeedHostsResolver.resolveHostsLists(executorService, logger, hosts, limitPortCounts, transportService, - TimeValue.timeValueSeconds(10))); + FileBasedSeedHostsProvider fileBasedSeedHostsProvider = new FileBasedSeedHostsProvider(configPath); + SeedHostsResolver seedHostsResolver = new SeedHostsResolver("test", Settings.EMPTY, transportService, fileBasedSeedHostsProvider); + seedHostsResolver.start(); + List results = fileBasedSeedHostsProvider.getSeedAddresses(seedHostsResolver); + seedHostsResolver.stop(); + return results; } } diff --git a/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java b/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java index 0506f5c48e803..3527d6de3da64 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java @@ -19,20 +19,23 @@ package org.elasticsearch.discovery; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; @@ -40,7 +43,6 @@ import org.elasticsearch.transport.nio.MockNioTransport; import org.junit.After; import org.junit.Before; -import org.mockito.Matchers; import java.io.Closeable; import java.io.IOException; @@ -54,7 +56,6 @@ import java.util.Set; import java.util.Stack; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; @@ -66,10 +67,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.core.IsNull.nullValue; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; public class SeedHostsResolverTests extends ESTestCase { @@ -90,8 +88,7 @@ public void startResolver() { TransportService transportService = mock(TransportService.class); when(transportService.getThreadPool()).thenReturn(threadPool); - seedHostsResolver = new SeedHostsResolver("test_node", Settings.EMPTY, transportService, hostsResolver -> transportAddresses); - seedHostsResolver.start(); + recreateSeedHostsResolver(transportService); final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory("[" + getClass().getName() + "]"); executorService = @@ -100,6 +97,14 @@ public void startResolver() { closeables = new Stack<>(); } + private void recreateSeedHostsResolver(TransportService transportService) { + if (seedHostsResolver != null) { + seedHostsResolver.stop(); + } + seedHostsResolver = new SeedHostsResolver("test_node", Settings.EMPTY, transportService, hostsResolver -> transportAddresses); + seedHostsResolver.start(); + } + @After public void stopResolver() throws IOException { seedHostsResolver.stop(); @@ -171,21 +176,17 @@ public BoundTransportAddress boundAddress() { new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); + recreateSeedHostsResolver(transportService); final int limitPortCounts = randomIntBetween(1, 10); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Collections.singletonList("127.0.0.1"), - limitPortCounts, - transportService, - TimeValue.timeValueSeconds(30)); + final List transportAddresses = seedHostsResolver.resolveHosts(Collections.singletonList("127.0.0.1"), + limitPortCounts); assertThat(transportAddresses, hasSize(limitPortCounts)); final Set ports = new HashSet<>(); for (final TransportAddress address : transportAddresses) { assertTrue(address.address().getAddress().isLoopbackAddress()); ports.add(address.getPort()); } - assertThat(ports, equalTo(IntStream.range(9300, 9300 + limitPortCounts).mapToObj(m -> m).collect(Collectors.toSet()))); + assertThat(ports, equalTo(IntStream.range(9300, 9300 + limitPortCounts).boxed().collect(Collectors.toSet()))); } public void testRemovingLocalAddresses() { @@ -216,24 +217,20 @@ public BoundTransportAddress boundAddress() { new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, + recreateSeedHostsResolver(transportService); + final List transportAddresses = seedHostsResolver.resolveHosts( Collections.singletonList(NetworkAddress.format(loopbackAddress)), - 10, - transportService, - TimeValue.timeValueSeconds(30)); + 10); assertThat(transportAddresses, hasSize(7)); final Set ports = new HashSet<>(); for (final TransportAddress address : transportAddresses) { assertTrue(address.address().getAddress().isLoopbackAddress()); ports.add(address.getPort()); } - assertThat(ports, equalTo(IntStream.range(9303, 9310).mapToObj(m -> m).collect(Collectors.toSet()))); + assertThat(ports, equalTo(IntStream.range(9303, 9310).boxed().collect(Collectors.toSet()))); } - public void testUnknownHost() { - final Logger logger = mock(Logger.class); + public void testUnknownHost() throws IllegalAccessException { final NetworkService networkService = new NetworkService(Collections.emptyList()); final String hostname = randomAlphaOfLength(8); final UnknownHostException unknownHostException = new UnknownHostException(hostname); @@ -266,22 +263,33 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); + recreateSeedHostsResolver(transportService); + + final Logger logger = LogManager.getLogger(SeedHostsResolver.class); + final MockLogAppender appender = new MockLogAppender(); + appender.start(); + appender.addExpectation( + new MockLogAppender.ExceptionSeenEventExpectation( + getTestName(), + logger.getName(), + Level.WARN, + "failed to resolve host [" + hostname + "]", + UnknownHostException.class, + unknownHostException.getMessage())); + + try { + Loggers.addAppender(logger, appender); + final List transportAddresses = seedHostsResolver.resolveHosts(Collections.singletonList(hostname), 1); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Arrays.asList(hostname), - 1, - transportService, - TimeValue.timeValueSeconds(30) - ); - - assertThat(transportAddresses, empty()); - verify(logger).warn("failed to resolve host [" + hostname + "]", unknownHostException); + assertThat(transportAddresses, empty()); + appender.assertAllExpectationsMatched(); + } finally { + Loggers.removeAppender(logger, appender); + appender.stop(); + } } - public void testResolveTimeout() { - final Logger logger = mock(Logger.class); + public void testResolveTimeout() throws IllegalAccessException { final NetworkService networkService = new NetworkService(Collections.emptyList()); final CountDownLatch latch = new CountDownLatch(1); final Transport transport = new MockNioTransport( @@ -324,29 +332,32 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); - final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(3, 5)); + recreateSeedHostsResolver(transportService); + + final Logger logger = LogManager.getLogger(SeedHostsResolver.class); + final MockLogAppender appender = new MockLogAppender(); + appender.start(); + appender.addExpectation( + new MockLogAppender.SeenEventExpectation( + getTestName(), + logger.getName(), + Level.WARN, + "timed out after [" + SeedHostsResolver.getResolveTimeout(Settings.EMPTY) + "] resolving host [hostname2]")); + try { - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Arrays.asList("hostname1", "hostname2"), - 1, - transportService, - resolveTimeout); + Loggers.addAppender(logger, appender); + final List transportAddresses = seedHostsResolver.resolveHosts(Arrays.asList("hostname1", "hostname2"), 1); assertThat(transportAddresses, hasSize(1)); - verify(logger).trace( - "resolved host [{}] to {}", "hostname1", - new TransportAddress[]{new TransportAddress(TransportAddress.META_ADDRESS, 9300)}); - verify(logger).warn("timed out after [{}] resolving host [{}]", resolveTimeout, "hostname2"); - verifyNoMoreInteractions(logger); + appender.assertAllExpectationsMatched(); } finally { + Loggers.removeAppender(logger, appender); + appender.stop(); latch.countDown(); } } - public void testInvalidHosts() { - final Logger logger = mock(Logger.class); + public void testInvalidHosts() throws IllegalAccessException { final Transport transport = new MockNioTransport( Settings.EMPTY, Version.CURRENT, @@ -369,16 +380,30 @@ public BoundTransportAddress boundAddress() { new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); - final List transportAddresses = SeedHostsResolver.resolveHostsLists( - executorService, - logger, - Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"), - 1, - transportService, - TimeValue.timeValueSeconds(30)); - assertThat(transportAddresses, hasSize(1)); // only one of the two is valid and will be used - assertThat(transportAddresses.get(0).getAddress(), equalTo("127.0.0.1")); - assertThat(transportAddresses.get(0).getPort(), equalTo(9301)); - verify(logger).warn(eq("failed to resolve host [127.0.0.1:9300:9300]"), Matchers.any(ExecutionException.class)); + recreateSeedHostsResolver(transportService); + + + final Logger logger = LogManager.getLogger(SeedHostsResolver.class); + final MockLogAppender appender = new MockLogAppender(); + appender.start(); + appender.addExpectation( + new MockLogAppender.SeenEventExpectation( + getTestName(), + logger.getName(), + Level.WARN, + "failed to resolve host [127.0.0.1:9300:9300]")); + + try { + Loggers.addAppender(logger, appender); + final List transportAddresses = seedHostsResolver.resolveHosts( + Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"), 1); + assertThat(transportAddresses, hasSize(1)); // only one of the two is valid and will be used + assertThat(transportAddresses.get(0).getAddress(), equalTo("127.0.0.1")); + assertThat(transportAddresses.get(0).getPort(), equalTo(9301)); + appender.assertAllExpectationsMatched(); + } finally { + Loggers.removeAppender(logger, appender); + appender.stop(); + } } } From d61827dd2e5c29de7ed73672f09554748d5e482e Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 1 Mar 2019 14:32:58 +0100 Subject: [PATCH 24/30] more Zen1 removal --- .../coordination/ClusterBootstrapService.java | 12 ++----- .../cluster/coordination/Coordinator.java | 34 +++---------------- .../coordination/PreVoteCollector.java | 2 -- .../cluster/coordination/Reconfigurator.java | 1 - .../ClusterBootstrapServiceTests.java | 30 ---------------- 5 files changed, 8 insertions(+), 71 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java index 526503aefd552..c4643771fb790 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java @@ -98,7 +98,7 @@ public static boolean discoveryIsConfigured(Settings settings) { void onFoundPeersUpdated() { final Set nodes = getDiscoveredNodes(); if (bootstrappingPermitted.get() && transportService.getLocalNode().isMasterNode() && bootstrapRequirements.isEmpty() == false - && isBootstrappedSupplier.getAsBoolean() == false && nodes.stream().noneMatch(Coordinator::isZen1Node)) { + && isBootstrappedSupplier.getAsBoolean() == false) { final Tuple,List> requirementMatchingResult; try { @@ -143,13 +143,8 @@ void scheduleUnconfiguredBootstrap() { @Override public void run() { final Set discoveredNodes = getDiscoveredNodes(); - final List zen1Nodes = discoveredNodes.stream().filter(Coordinator::isZen1Node).collect(Collectors.toList()); - if (zen1Nodes.isEmpty()) { - logger.debug("performing best-effort cluster bootstrapping with {}", discoveredNodes); - startBootstrap(discoveredNodes, emptyList()); - } else { - logger.info("avoiding best-effort cluster bootstrapping due to discovery of pre-7.0 nodes {}", zen1Nodes); - } + logger.debug("performing best-effort cluster bootstrapping with {}", discoveredNodes); + startBootstrap(discoveredNodes, emptyList()); } @Override @@ -166,7 +161,6 @@ private Set getDiscoveredNodes() { private void startBootstrap(Set discoveryNodes, List unsatisfiedRequirements) { assert discoveryNodes.stream().allMatch(DiscoveryNode::isMasterNode) : discoveryNodes; - assert discoveryNodes.stream().noneMatch(Coordinator::isZen1Node) : discoveryNodes; assert unsatisfiedRequirements.size() < discoveryNodes.size() : discoveryNodes + " smaller than " + unsatisfiedRequirements; if (bootstrappingPermitted.compareAndSet(true, false)) { doBootstrap(new VotingConfiguration(Stream.concat(discoveryNodes.stream().map(DiscoveryNode::getId), diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 9664de95c40a1..f2e3a05e5f554 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; @@ -43,7 +42,6 @@ import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; import org.elasticsearch.cluster.service.MasterService; -import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; @@ -62,8 +60,8 @@ import org.elasticsearch.discovery.DiscoveryStats; import org.elasticsearch.discovery.HandshakingTransportAddressConnector; import org.elasticsearch.discovery.PeerFinder; -import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.discovery.SeedHostsProvider; +import org.elasticsearch.discovery.SeedHostsResolver; import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.TransportResponse.Empty; import org.elasticsearch.transport.TransportService; @@ -368,11 +366,7 @@ private void startElection() { final StartJoinRequest startJoinRequest = new StartJoinRequest(getLocalNode(), Math.max(getCurrentTerm(), maxTermSeen) + 1); logger.debug("starting election with {}", startJoinRequest); - getDiscoveredNodes().forEach(node -> { - if (isZen1Node(node) == false) { - joinHelper.sendStartJoinRequest(startJoinRequest, node); - } - }); + getDiscoveredNodes().forEach(node -> joinHelper.sendStartJoinRequest(startJoinRequest, node)); } } } @@ -383,11 +377,7 @@ private void abdicateTo(DiscoveryNode newMaster) { assert newMaster.isMasterNode() : "should only abdicate to master-eligible node but was " + newMaster; final StartJoinRequest startJoinRequest = new StartJoinRequest(newMaster, Math.max(getCurrentTerm(), maxTermSeen) + 1); logger.info("abdicating to {} with term {}", newMaster, startJoinRequest.getTerm()); - getLastAcceptedState().nodes().mastersFirstStream().forEach(node -> { - if (isZen1Node(node) == false) { - joinHelper.sendStartJoinRequest(startJoinRequest, node); - } - }); + getLastAcceptedState().nodes().mastersFirstStream().forEach(node -> joinHelper.sendStartJoinRequest(startJoinRequest, node)); // handling of start join messages on the local node will be dispatched to the generic thread-pool assert mode == Mode.LEADER : "should still be leader after sending abdication messages " + mode; // explicitly move node to candidate state so that the next cluster state update task yields an onNoLongerMaster event @@ -836,7 +826,7 @@ ClusterState improveConfiguration(ClusterState clusterState) { assert Thread.holdsLock(mutex) : "Coordinator mutex not held"; final Set liveNodes = StreamSupport.stream(clusterState.nodes().spliterator(), false) - .filter(this::hasJoinVoteFrom).filter(discoveryNode -> isZen1Node(discoveryNode) == false).collect(Collectors.toSet()); + .filter(this::hasJoinVoteFrom).collect(Collectors.toSet()); final VotingConfiguration newConfig = reconfigurator.reconfigure(liveNodes, clusterState.getVotingConfigExclusions().stream().map(VotingConfigExclusion::getNodeId).collect(Collectors.toSet()), getLocalNode(), clusterState.getLastAcceptedConfiguration()); @@ -1133,10 +1123,7 @@ public void run() { if (prevotingRound != null) { prevotingRound.close(); } - final List discoveredNodes - = getDiscoveredNodes().stream().filter(n -> isZen1Node(n) == false).collect(Collectors.toList()); - - prevotingRound = preVoteCollector.start(lastAcceptedState, discoveredNodes); + prevotingRound = preVoteCollector.start(lastAcceptedState, getDiscoveredNodes()); } } } @@ -1361,15 +1348,4 @@ protected void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest app publicationContext.sendApplyCommit(destination, applyCommit, wrapWithMutex(responseActionListener)); } } - - // TODO: only here temporarily for BWC development, remove once complete - public static Settings.Builder addZen1Attribute(boolean isZen1Node, Settings.Builder builder) { - return builder.put("node.attr.zen1", isZen1Node); - } - - // TODO: only here temporarily for BWC development, remove once complete - public static boolean isZen1Node(DiscoveryNode discoveryNode) { - return discoveryNode.getVersion().before(Version.V_7_0_0) || - (Booleans.isTrue(discoveryNode.getAttributes().getOrDefault("zen1", "false"))); - } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index bb15f08c58e13..c72925aea1689 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -38,7 +38,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongConsumer; -import java.util.stream.StreamSupport; import static org.elasticsearch.cluster.coordination.CoordinationState.isElectionQuorum; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet; @@ -141,7 +140,6 @@ private class PreVotingRound implements Releasable { } void start(final Iterable broadcastNodes) { - assert StreamSupport.stream(broadcastNodes.spliterator(), false).noneMatch(Coordinator::isZen1Node) : broadcastNodes; logger.debug("{} requesting pre-votes from {}", this, broadcastNodes); broadcastNodes.forEach(n -> transportService.sendRequest(n, REQUEST_PRE_VOTE_ACTION_NAME, preVoteRequest, new TransportResponseHandler() { diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java index ebca37bdac0b1..44e6532df2daa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java @@ -97,7 +97,6 @@ public String toString() { */ public VotingConfiguration reconfigure(Set liveNodes, Set retiredNodeIds, DiscoveryNode currentMaster, VotingConfiguration currentConfig) { - assert liveNodes.stream().noneMatch(Coordinator::isZen1Node) : liveNodes; assert liveNodes.contains(currentMaster) : "liveNodes = " + liveNodes + " master = " + currentMaster; logger.trace("{} reconfiguring {} based on liveNodes={}, retiredNodeIds={}, currentMaster={}", this, currentConfig, liveNodes, retiredNodeIds, currentMaster); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java index 7f32c086f975f..a2dee54b3c6fc 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java @@ -43,7 +43,6 @@ import static java.util.Collections.emptySet; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; -import static java.util.Collections.singletonMap; import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.BOOTSTRAP_PLACEHOLDER_PREFIX; import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING; import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING; @@ -153,19 +152,6 @@ private void testDoesNothingWithSettings(Settings.Builder builder) { deterministicTaskQueue.runAllTasks(); } - public void testDoesNothingByDefaultIfZen1NodesDiscovered() { - final DiscoveryNode zen1Node = new DiscoveryNode("zen1", buildNewFakeTransportAddress(), singletonMap("zen1", "true"), - singleton(Role.MASTER), Version.CURRENT); - ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(Settings.EMPTY, transportService, () -> - Stream.of(localNode, zen1Node).collect(Collectors.toSet()), () -> false, vc -> { - throw new AssertionError("should not be called"); - }); - transportService.start(); - clusterBootstrapService.scheduleUnconfiguredBootstrap(); - deterministicTaskQueue.runAllTasks(); - } - - public void testThrowsExceptionOnDuplicates() { final IllegalArgumentException illegalArgumentException = expectThrows(IllegalArgumentException.class, () -> { new ClusterBootstrapService(builder().putList( @@ -352,23 +338,7 @@ public void testDoesNotBootstrapsIfNotConfigured() { deterministicTaskQueue.runAllTasks(); } - public void testDoesNotBootstrapsIfZen1NodesDiscovered() { - final DiscoveryNode zen1Node = new DiscoveryNode("zen1", buildNewFakeTransportAddress(), singletonMap("zen1", "true"), - singleton(Role.MASTER), Version.CURRENT); - - ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(Settings.builder().putList( - INITIAL_MASTER_NODES_SETTING.getKey(), localNode.getName(), otherNode1.getName(), otherNode2.getName()).build(), - transportService, () -> Stream.of(otherNode1, otherNode2, zen1Node).collect(Collectors.toList()), () -> false, vc -> { - throw new AssertionError("should not be called"); - }); - - transportService.start(); - clusterBootstrapService.onFoundPeersUpdated(); - deterministicTaskQueue.runAllTasks(); - } - public void testRetriesBootstrappingOnException() { - final AtomicLong bootstrappingAttempts = new AtomicLong(); ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(Settings.builder().putList( INITIAL_MASTER_NODES_SETTING.getKey(), localNode.getName(), otherNode1.getName(), otherNode2.getName()).build(), From c99dce0db097f72b2331f59acd256ee7bb8d8049 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Mar 2019 12:45:20 +0100 Subject: [PATCH 25/30] remove ZEN1_BWC_TERM --- .../elasticsearch/cluster/ClusterState.java | 11 +----- .../coordination/CoordinationState.java | 34 ++++++------------- .../cluster/coordination/Coordinator.java | 12 +------ .../coordination/PreVoteCollector.java | 2 +- 4 files changed, 13 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index ce080433640d6..93ee8af779472 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -22,14 +22,13 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - import org.elasticsearch.Version; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.coordination.CoordinationMetaData; -import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion; +import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; @@ -69,8 +68,6 @@ import java.util.Optional; import java.util.Set; -import static org.elasticsearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM; - /** * Represents the current state of the cluster. *

@@ -211,12 +208,6 @@ public long getVersion() { return version(); } - public long getVersionOrMetaDataVersion() { - // When following a Zen1 master, the cluster state version is not guaranteed to increase, so instead it is preferable to use the - // metadata version to determine the freshest node. However when following a Zen2 master the cluster state version should be used. - return term() == ZEN1_BWC_TERM ? metaData().version() : version(); - } - /** * This stateUUID is automatically generated for for each version of cluster state. It is used to make sure that * we are applying diffs to the right previous state. diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index dff6b5add0b09..a38a383b269d5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -32,8 +32,6 @@ import java.util.Map; import java.util.Optional; -import static org.elasticsearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM; - /** * The core class of the cluster state coordination algorithm, directly implementing the * formal model @@ -86,10 +84,6 @@ public long getLastAcceptedVersion() { return getLastAcceptedState().version(); } - private long getLastAcceptedVersionOrMetaDataVersion() { - return getLastAcceptedState().getVersionOrMetaDataVersion(); - } - public VotingConfiguration getLastCommittedConfiguration() { return getLastAcceptedState().getLastCommittedConfiguration(); } @@ -198,7 +192,7 @@ public Join handleStartJoin(StartJoinRequest startJoinRequest) { publishVotes = new VoteCollection(); return new Join(localNode, startJoinRequest.getSourceNode(), getCurrentTerm(), getLastAcceptedTerm(), - getLastAcceptedVersionOrMetaDataVersion()); + getLastAcceptedVersion()); } /** @@ -231,12 +225,12 @@ public boolean handleJoin(Join join) { " of join higher than current last accepted term " + lastAcceptedTerm); } - if (join.getLastAcceptedTerm() == lastAcceptedTerm && join.getLastAcceptedVersion() > getLastAcceptedVersionOrMetaDataVersion()) { + if (join.getLastAcceptedTerm() == lastAcceptedTerm && join.getLastAcceptedVersion() > getLastAcceptedVersion()) { logger.debug( "handleJoin: ignored join as joiner has a better last accepted version (expected: <=[{}], actual: [{}]) in term {}", - getLastAcceptedVersionOrMetaDataVersion(), join.getLastAcceptedVersion(), lastAcceptedTerm); + getLastAcceptedVersion(), join.getLastAcceptedVersion(), lastAcceptedTerm); throw new CoordinationStateRejectedException("incoming last accepted version " + join.getLastAcceptedVersion() + - " of join higher than current last accepted version " + getLastAcceptedVersionOrMetaDataVersion() + " of join higher than current last accepted version " + getLastAcceptedVersion() + " in term " + lastAcceptedTerm); } @@ -332,16 +326,10 @@ public PublishResponse handlePublishRequest(PublishRequest publishRequest) { getCurrentTerm()); } if (clusterState.term() == getLastAcceptedTerm() && clusterState.version() <= getLastAcceptedVersion()) { - if (clusterState.term() == ZEN1_BWC_TERM - && clusterState.nodes().getMasterNode().equals(getLastAcceptedState().nodes().getMasterNode()) == false) { - logger.debug("handling publish request in compatibility mode despite version mismatch (expected: >[{}], actual: [{}])", - getLastAcceptedVersion(), clusterState.version()); - } else { - logger.debug("handlePublishRequest: ignored publish request due to version mismatch (expected: >[{}], actual: [{}])", - getLastAcceptedVersion(), clusterState.version()); - throw new CoordinationStateRejectedException("incoming version " + clusterState.version() + - " lower or equal to current version " + getLastAcceptedVersion()); - } + logger.debug("handlePublishRequest: ignored publish request due to version mismatch (expected: >[{}], actual: [{}])", + getLastAcceptedVersion(), clusterState.version()); + throw new CoordinationStateRejectedException("incoming version " + clusterState.version() + + " lower or equal to current version " + getLastAcceptedVersion()); } logger.trace("handlePublishRequest: accepting publish request for version [{}] and term [{}]", @@ -484,10 +472,8 @@ default void markLastAcceptedStateAsCommitted() { metaDataBuilder = MetaData.builder(lastAcceptedState.metaData()); metaDataBuilder.coordinationMetaData(coordinationMetaData); } - // if we receive a commit from a Zen1 master that has not recovered its state yet, the cluster uuid might not been known yet. - assert lastAcceptedState.metaData().clusterUUID().equals(MetaData.UNKNOWN_CLUSTER_UUID) == false || - lastAcceptedState.term() == ZEN1_BWC_TERM : - "received cluster state with empty cluster uuid but not Zen1 BWC term: " + lastAcceptedState; + assert lastAcceptedState.metaData().clusterUUID().equals(MetaData.UNKNOWN_CLUSTER_UUID) == false : + "received cluster state with empty cluster uuid: " + lastAcceptedState; if (lastAcceptedState.metaData().clusterUUID().equals(MetaData.UNKNOWN_CLUSTER_UUID) == false && lastAcceptedState.metaData().clusterUUIDCommitted() == false) { if (metaDataBuilder == null) { diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index f2e3a05e5f554..00ef8398cfc2d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -85,8 +85,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery { - public static final long ZEN1_BWC_TERM = 0; - private static final Logger logger = LogManager.getLogger(Coordinator.class); // the timeout for the publication of each value @@ -269,14 +267,6 @@ PublishWithJoinResponse handlePublishRequest(PublishRequest publishRequest) { throw new CoordinationStateRejectedException("no longer leading this publication's term: " + publishRequest); } - if (publishRequest.getAcceptedState().term() == ZEN1_BWC_TERM && getCurrentTerm() == ZEN1_BWC_TERM - && mode == Mode.FOLLOWER && Optional.of(sourceNode).equals(lastKnownLeader) == false) { - - logger.debug("received cluster state from {} but currently following {}, rejecting", sourceNode, lastKnownLeader); - throw new CoordinationStateRejectedException("received cluster state from " + sourceNode + " but currently following " - + lastKnownLeader + ", rejecting"); - } - final ClusterState localState = coordinationState.get().getLastAcceptedState(); if (localState.metaData().clusterUUIDCommitted() && @@ -601,7 +591,7 @@ public ClusterTasksResult execute(ClusterState currentSt private PreVoteResponse getPreVoteResponse() { return new PreVoteResponse(getCurrentTerm(), coordinationState.get().getLastAcceptedTerm(), - coordinationState.get().getLastAcceptedState().getVersionOrMetaDataVersion()); + coordinationState.get().getLastAcceptedState().version()); } // package-visible for testing diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java index c72925aea1689..5001e3be18183 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java @@ -180,7 +180,7 @@ private void handlePreVoteResponse(final PreVoteResponse response, final Discove if (response.getLastAcceptedTerm() > clusterState.term() || (response.getLastAcceptedTerm() == clusterState.term() - && response.getLastAcceptedVersion() > clusterState.getVersionOrMetaDataVersion())) { + && response.getLastAcceptedVersion() > clusterState.version())) { logger.debug("{} ignoring {} from {} as it is fresher", this, response, sender); return; } From 2edd8b451e08a6b67c645b668c7ecd6ce4479ecd Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Mar 2019 12:48:13 +0100 Subject: [PATCH 26/30] remove legacy-zen discovery type --- .../java/org/elasticsearch/discovery/DiscoveryModule.java | 1 - .../org/elasticsearch/bootstrap/BootstrapChecksTests.java | 7 +++---- .../java/org/elasticsearch/test/InternalTestCluster.java | 6 ------ .../org/elasticsearch/xpack/security/SecurityTests.java | 3 +-- 4 files changed, 4 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index d680837bf1b8c..ab95f2a430497 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -66,7 +66,6 @@ public class DiscoveryModule { private static final Logger logger = LogManager.getLogger(DiscoveryModule.class); - public static final String ZEN_DISCOVERY_TYPE = "legacy-zen"; public static final String ZEN2_DISCOVERY_TYPE = "zen"; public static final Setting DISCOVERY_TYPE_SETTING = diff --git a/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java b/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java index e6ecb54df9f07..dbd58f6d5feb6 100644 --- a/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java +++ b/server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java @@ -45,7 +45,6 @@ import java.util.function.Consumer; import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; -import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; import static org.hamcrest.CoreMatchers.allOf; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; @@ -105,7 +104,7 @@ public void testEnforceLimitsWhenBoundToNonLocalAddress() { when(boundTransportAddress.boundAddresses()).thenReturn(transportAddresses.toArray(new TransportAddress[0])); when(boundTransportAddress.publishAddress()).thenReturn(publishAddress); - final String discoveryType = randomFrom(ZEN_DISCOVERY_TYPE, ZEN2_DISCOVERY_TYPE, "single-node"); + final String discoveryType = randomFrom(ZEN2_DISCOVERY_TYPE, "single-node"); assertEquals(BootstrapChecks.enforceLimits(boundTransportAddress, discoveryType), !"single-node".equals(discoveryType)); } @@ -123,7 +122,7 @@ public void testEnforceLimitsWhenPublishingToNonLocalAddress() { when(boundTransportAddress.boundAddresses()).thenReturn(transportAddresses.toArray(new TransportAddress[0])); when(boundTransportAddress.publishAddress()).thenReturn(publishAddress); - final String discoveryType = randomFrom(ZEN_DISCOVERY_TYPE, ZEN2_DISCOVERY_TYPE, "single-node"); + final String discoveryType = randomFrom(ZEN2_DISCOVERY_TYPE, "single-node"); assertEquals(BootstrapChecks.enforceLimits(boundTransportAddress, discoveryType), !"single-node".equals(discoveryType)); } @@ -713,7 +712,7 @@ public void testDiscoveryConfiguredCheck() throws NodeValidationException { // not enforced for non-zen2 discovery BootstrapChecks.check(createTestContext(Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), - randomFrom(ZEN_DISCOVERY_TYPE, "single-node", randomAlphaOfLength(5))).build(), MetaData.EMPTY_META_DATA), true, checks); + randomFrom("single-node", randomAlphaOfLength(5))).build(), MetaData.EMPTY_META_DATA), true, checks); final NodeValidationException e = expectThrows(NodeValidationException.class, () -> BootstrapChecks.check(zen2Context, true, checks)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index cae76718378aa..fb652a5b4cffb 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -148,7 +148,6 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_TYPE_SETTING; import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; -import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; import static org.elasticsearch.node.Node.INITIAL_STATE_TIMEOUT_SETTING; import static org.elasticsearch.discovery.FileBasedSeedHostsProvider.UNICAST_HOSTS_FILE; import static org.elasticsearch.test.ESTestCase.assertBusy; @@ -407,10 +406,6 @@ public InternalTestCluster( EsExecutors.daemonThreadFactory("test_" + clusterName), new ThreadContext(Settings.EMPTY)); } - private static boolean usingZen1(Settings settings) { - return ZEN_DISCOVERY_TYPE.equals(DISCOVERY_TYPE_SETTING.get(settings)); - } - /** * Sets {@link #bootstrapMasterNodeIndex} to the given value, see {@link #bootstrapMasterNodeWithSpecifiedIndex(List)} * for the description of how this field is used. @@ -638,7 +633,6 @@ private Settings getNodeSettings(final int nodeId, final long seed, final Settin final String discoveryType = DISCOVERY_TYPE_SETTING.get(updatedSettings.build()); final boolean usingSingleNodeDiscovery = discoveryType.equals("single-node"); - final boolean usingZen1 = usingZen1(updatedSettings.build()); if (usingSingleNodeDiscovery == false) { if (autoManageMinMasterNodes) { assertThat("automatically managing min master nodes require nodes to complete a join cycle when starting", diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index 5b7ce8b1d03eb..35180ab8f31d0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -66,7 +66,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_FORMAT_SETTING; import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; -import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_INDEX_FORMAT; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; import static org.hamcrest.Matchers.containsString; @@ -253,7 +252,7 @@ public void testTLSJoinValidator() throws Exception { int numIters = randomIntBetween(1, 10); for (int i = 0; i < numIters; i++) { boolean tlsOn = randomBoolean(); - String discoveryType = randomFrom("single-node", ZEN_DISCOVERY_TYPE, ZEN2_DISCOVERY_TYPE, randomAlphaOfLength(4)); + String discoveryType = randomFrom("single-node", ZEN2_DISCOVERY_TYPE, randomAlphaOfLength(4)); Security.ValidateTLSOnJoin validator = new Security.ValidateTLSOnJoin(tlsOn, discoveryType); MetaData.Builder builder = MetaData.builder(); License license = TestUtils.generateSignedLicense(TimeValue.timeValueHours(24)); From d3201dd338cee8fd7b02b67c6fd71e7ad2346cbf Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Mar 2019 12:50:55 +0100 Subject: [PATCH 27/30] remove from transport.tracer.exclude --- .../java/org/elasticsearch/transport/TransportSettings.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/transport/TransportSettings.java b/server/src/main/java/org/elasticsearch/transport/TransportSettings.java index 60e230004cac1..2d6614639519f 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportSettings.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportSettings.java @@ -155,7 +155,7 @@ public final class TransportSettings { listSetting("transport.tracer.include", emptyList(), Function.identity(), Setting.Property.Dynamic, Setting.Property.NodeScope); public static final Setting> TRACE_LOG_EXCLUDE_SETTING = listSetting("transport.tracer.exclude", - Arrays.asList("internal:discovery/zen/fd*", "internal:coordination/fault_detection/*", TransportLivenessAction.NAME), + Arrays.asList("internal:coordination/fault_detection/*", TransportLivenessAction.NAME), Function.identity(), Setting.Property.Dynamic, Setting.Property.NodeScope); private TransportSettings() { From 7add5f477a12dc5caf786925c70ff59513438840 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Mar 2019 12:51:11 +0100 Subject: [PATCH 28/30] update docs for transport.tracer.exclude --- docs/reference/modules/transport.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/modules/transport.asciidoc b/docs/reference/modules/transport.asciidoc index 180a8190868f0..8ee2f55a9b1d1 100644 --- a/docs/reference/modules/transport.asciidoc +++ b/docs/reference/modules/transport.asciidoc @@ -156,7 +156,7 @@ PUT _cluster/settings { "transient" : { "transport.tracer.include" : "*", - "transport.tracer.exclude" : "internal:discovery/zen/fd*" + "transport.tracer.exclude" : "internal:coordination/fault_detection/*" } } -------------------------------------------------- From 9d5d034bbac7f3978502ac92351ba1b456190ad3 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Mar 2019 12:52:09 +0100 Subject: [PATCH 29/30] do not mention Zen discovery in Wireshark section --- docs/plugins/integrations.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/plugins/integrations.asciidoc b/docs/plugins/integrations.asciidoc index 6d543408f679f..efbc83f2f8f87 100644 --- a/docs/plugins/integrations.asciidoc +++ b/docs/plugins/integrations.asciidoc @@ -194,7 +194,7 @@ releases 2.0 and later do not support rivers. A pluggable elastic JavaScript query DSL builder for Elasticsearch * https://www.wireshark.org/[Wireshark]: - Protocol dissection for Zen discovery, HTTP and the binary protocol + Protocol dissection for HTTP and the transport protocol * https://www.itemsapi.com/[ItemsAPI]: Search backend for mobile and web From 2bde97c955b8b7074b003e3fde0b72b2ff43c8f3 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 4 Mar 2019 12:54:08 +0100 Subject: [PATCH 30/30] no zen in startup logs --- docs/reference/getting-started.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index bee2ae5194477..9e7571e038315 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -248,8 +248,8 @@ If everything goes well with installation, you should see a bunch of messages th [2018-09-13T12:20:05,202][INFO ][o.e.t.TransportService ] [localhost.localdomain] publish_address {127.0.0.1:9300}, bound_addresses {[::1]:9300}, {127.0.0.1:9300} [2018-09-13T12:20:05,221][WARN ][o.e.b.BootstrapChecks ] [localhost.localdomain] max file descriptors [4096] for elasticsearch process is too low, increase to at least [65535] [2018-09-13T12:20:05,221][WARN ][o.e.b.BootstrapChecks ] [localhost.localdomain] max virtual memory areas vm.max_map_count [65530] is too low, increase to at least [262144] -[2018-09-13T12:20:08,355][INFO ][o.e.c.s.MasterService ] [localhost.localdomain] zen-disco-elected-as-master ([0] nodes joined)[, ], reason: master node changed {previous [], current [{localhost.localdomain}{B0aEHNagTiWx7SYj-l4NTw}{hzsQz6CVQMCTpMCVLM4IHg}{127.0.0.1}{127.0.0.1:9300}{testattr=test}]} -[2018-09-13T12:20:08,360][INFO ][o.e.c.s.ClusterApplierService] [localhost.localdomain] master node changed {previous [], current [{localhost.localdomain}{B0aEHNagTiWx7SYj-l4NTw}{hzsQz6CVQMCTpMCVLM4IHg}{127.0.0.1}{127.0.0.1:9300}{testattr=test}]}, reason: apply cluster state (from master [master {localhost.localdomain}{B0aEHNagTiWx7SYj-l4NTw}{hzsQz6CVQMCTpMCVLM4IHg}{127.0.0.1}{127.0.0.1:9300}{testattr=test} committed version [1] source [zen-disco-elected-as-master ([0] nodes joined)[, ]]]) +[2018-09-13T12:20:08,355][INFO ][o.e.c.s.MasterService ] [localhost.localdomain] elected-as-master ([0] nodes joined)[, ], reason: master node changed {previous [], current [{localhost.localdomain}{B0aEHNagTiWx7SYj-l4NTw}{hzsQz6CVQMCTpMCVLM4IHg}{127.0.0.1}{127.0.0.1:9300}{testattr=test}]} +[2018-09-13T12:20:08,360][INFO ][o.e.c.s.ClusterApplierService] [localhost.localdomain] master node changed {previous [], current [{localhost.localdomain}{B0aEHNagTiWx7SYj-l4NTw}{hzsQz6CVQMCTpMCVLM4IHg}{127.0.0.1}{127.0.0.1:9300}{testattr=test}]}, reason: apply cluster state (from master [master {localhost.localdomain}{B0aEHNagTiWx7SYj-l4NTw}{hzsQz6CVQMCTpMCVLM4IHg}{127.0.0.1}{127.0.0.1:9300}{testattr=test} committed version [1] source [elected-as-master ([0] nodes joined)[, ]]]) [2018-09-13T12:20:08,384][INFO ][o.e.h.n.Netty4HttpServerTransport] [localhost.localdomain] publish_address {127.0.0.1:9200}, bound_addresses {[::1]:9200}, {127.0.0.1:9200} [2018-09-13T12:20:08,384][INFO ][o.e.n.Node ] [localhost.localdomain] started