1
0
Эх сурвалжийг харах

Remove Zen1 (#39466)

Removes all traces of Zen1 from the code base. Some of these commits will also be backported to
7.0/7.x (#39470) as the cluster.coordination package was making use of some things in
discovery.zen and we want to keep 7.x as close as possible to master.
Yannick Welsch 6 жил өмнө
parent
commit
3b71a31557
86 өөрчлөгдсөн 646 нэмэгдсэн , 10610 устгасан
  1. 0 13
      buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy
  2. 15 43
      buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy
  3. 1 1
      docs/plugins/integrations.asciidoc
  4. 2 2
      docs/reference/getting-started.asciidoc
  5. 2 0
      docs/reference/migration/migrate_8_0.asciidoc
  6. 32 0
      docs/reference/migration/migrate_8_0/discovery.asciidoc
  7. 4 23
      docs/reference/modules/discovery/discovery-settings.asciidoc
  8. 1 1
      docs/reference/modules/transport.asciidoc
  9. 0 1
      qa/logging-config/build.gradle
  10. 0 1
      qa/rolling-upgrade/build.gradle
  11. 2 4
      server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java
  12. 16 54
      server/src/main/java/org/elasticsearch/cluster/ClusterState.java
  13. 3 4
      server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java
  14. 4 13
      server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java
  15. 10 24
      server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java
  16. 10 41
      server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java
  17. 1 18
      server/src/main/java/org/elasticsearch/cluster/coordination/FollowersChecker.java
  18. 4 41
      server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java
  19. 0 5
      server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java
  20. 1 27
      server/src/main/java/org/elasticsearch/cluster/coordination/LeaderChecker.java
  21. 0 6
      server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java
  22. 1 1
      server/src/main/java/org/elasticsearch/cluster/coordination/PendingClusterStateStats.java
  23. 1 3
      server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java
  24. 2 27
      server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java
  25. 1 1
      server/src/main/java/org/elasticsearch/cluster/coordination/PublishClusterStateStats.java
  26. 0 1
      server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java
  27. 29 5
      server/src/main/java/org/elasticsearch/cluster/coordination/ValidateJoinRequest.java
  28. 4 31
      server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
  29. 2 21
      server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java
  30. 0 93
      server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java
  31. 2 2
      server/src/main/java/org/elasticsearch/discovery/DiscoveryStats.java
  32. 86 8
      server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java
  33. 1 13
      server/src/main/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProvider.java
  34. 0 223
      server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java
  35. 0 116
      server/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java
  36. 0 444
      server/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java
  37. 0 239
      server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java
  38. 0 381
      server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java
  39. 0 370
      server/src/main/java/org/elasticsearch/discovery/zen/NodesFaultDetection.java
  40. 0 309
      server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java
  41. 0 663
      server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java
  42. 0 698
      server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java
  43. 0 1251
      server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
  44. 0 196
      server/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java
  45. 1 4
      server/src/main/java/org/elasticsearch/gateway/Gateway.java
  46. 0 3
      server/src/main/java/org/elasticsearch/gateway/GatewayService.java
  47. 4 2
      server/src/main/java/org/elasticsearch/node/Node.java
  48. 1 1
      server/src/main/java/org/elasticsearch/transport/TransportSettings.java
  49. 2 2
      server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java
  50. 3 7
      server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java
  51. 1 1
      server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java
  52. 0 207
      server/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java
  53. 0 317
      server/src/test/java/org/elasticsearch/cluster/ack/AckIT.java
  54. 0 30
      server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java
  55. 0 1
      server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java
  56. 6 7
      server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java
  57. 2 2
      server/src/test/java/org/elasticsearch/cluster/coordination/JoinTaskExecutorTests.java
  58. 0 33
      server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java
  59. 44 1
      server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java
  60. 0 15
      server/src/test/java/org/elasticsearch/cluster/coordination/ZenDiscoveryIT.java
  61. 1 5
      server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java
  62. 4 20
      server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java
  63. 2 25
      server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java
  64. 12 10
      server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java
  65. 0 38
      server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java
  66. 316 2
      server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java
  67. 0 18
      server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderTests.java
  68. 0 359
      server/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java
  69. 0 62
      server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryIT.java
  70. 0 157
      server/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java
  71. 0 868
      server/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java
  72. 0 268
      server/src/test/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueueTests.java
  73. 0 961
      server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java
  74. 0 940
      server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java
  75. 0 601
      server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java
  76. 0 90
      server/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java
  77. 3 4
      server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java
  78. 1 27
      test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java
  79. 1 92
      test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
  80. 1 1
      test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java
  81. 2 2
      test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java
  82. 1 1
      x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java
  83. 1 2
      x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java
  84. 0 2
      x-pack/qa/rolling-upgrade-basic/build.gradle
  85. 0 2
      x-pack/qa/rolling-upgrade-multi-cluster/build.gradle
  86. 0 2
      x-pack/qa/rolling-upgrade/build.gradle

+ 0 - 13
buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy

@@ -63,19 +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<Integer> 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

+ 15 - 43
buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy

@@ -127,42 +127,25 @@ class ClusterFormationTasks {
             nodes.add(node)
             Closure<Map> 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))
         }
 
@@ -381,17 +364,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

+ 1 - 1
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

+ 2 - 2
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
 

+ 2 - 0
docs/reference/migration/migrate_8_0.asciidoc

@@ -12,6 +12,7 @@ See also <<release-highlights>> and <<es-release-notes>>.
 coming[8.0.0]
 
 * <<breaking_80_analysis_changes>>
+* <<breaking_80_discovery_changes>>
 * <<breaking_80_mappings_changes>>
 * <<breaking_80_snapshots_changes>>
 
@@ -32,5 +33,6 @@ 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[]
 include::migrate_8_0/snapshots.asciidoc[]

+ 32 - 0
docs/reference/migration/migrate_8_0/discovery.asciidoc

@@ -0,0 +1,32 @@
+[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.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`
+- `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`

+ 4 - 23
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 <<unicast.hosts>>. 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 <<unicast.hosts>>.
 
 `discovery.seed_providers`::
 
     Specifies which types of <<built-in-hosts-providers,seed hosts provider>>
     to use to obtain the addresses of the seed nodes used to start the
     discovery process. By default, it is the
-    <<settings-based-hosts-provider,settings-based seed hosts provider>>.  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.
+    <<settings-based-hosts-provider,settings-based seed hosts provider>>.
 
 `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.
-
 --

+ 1 - 1
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/*"
    }
 }
 --------------------------------------------------

+ 0 - 1
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 

+ 0 - 1
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 }

+ 2 - 4
server/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java

@@ -27,6 +27,7 @@ import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
 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 org.elasticsearch.transport.TransportService;
 import java.io.IOException;
 import java.util.function.Predicate;
 
-import static org.elasticsearch.discovery.zen.PublishClusterStateAction.serializeFullClusterState;
-
 public class TransportClusterStateAction extends TransportMasterNodeReadAction<ClusterStateRequest, ClusterStateResponse> {
 
 
@@ -127,7 +126,6 @@ public class TransportClusterStateAction extends TransportMasterNodeReadAction<C
         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());
@@ -185,7 +183,7 @@ public class TransportClusterStateAction extends TransportMasterNodeReadAction<C
             }
         }
         listener.onResponse(new ClusterStateResponse(currentState.getClusterName(), builder.build(),
-            serializeFullClusterState(currentState, Version.CURRENT).length(), false));
+            PublicationTransportHandler.serializeFullClusterState(currentState, Version.CURRENT).length(), false));
     }
 
 

+ 16 - 54
server/src/main/java/org/elasticsearch/cluster/ClusterState.java

@@ -22,14 +22,13 @@ package org.elasticsearch.cluster;
 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;
@@ -60,7 +59,6 @@ import org.elasticsearch.common.xcontent.ToXContentFragment;
 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;
@@ -70,8 +68,6 @@ import java.util.Map;
 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.
  * <p>
@@ -80,9 +76,8 @@ import static org.elasticsearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM;
  * 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.
  * <p>
  * 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
@@ -179,19 +174,17 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
 
     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<String, Custom> customs,
-                        int minimumMasterNodesOnPublishingMaster, boolean wasReadFromDiff) {
+                        boolean wasReadFromDiff) {
         this.version = version;
         this.stateUUID = stateUUID;
         this.clusterName = clusterName;
@@ -200,7 +193,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
         this.nodes = nodes;
         this.blocks = blocks;
         this.customs = customs;
-        this.minimumMasterNodesOnPublishingMaster = minimumMasterNodesOnPublishingMaster;
         this.wasReadFromDiff = wasReadFromDiff;
     }
 
@@ -216,12 +208,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
         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.
@@ -294,22 +280,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
         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.
      */
@@ -659,7 +629,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
         private ClusterBlocks blocks = ClusterBlocks.EMPTY_CLUSTER_BLOCK;
         private final ImmutableOpenMap.Builder<String, Custom> customs;
         private boolean fromDiff;
-        private int minimumMasterNodesOnPublishingMaster = -1;
 
         public Builder(ClusterState state) {
             this.clusterName = state.clusterName;
@@ -670,7 +639,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             this.metaData = state.metaData();
             this.blocks = state.blocks();
             this.customs = ImmutableOpenMap.builder(state.customs());
-            this.minimumMasterNodesOnPublishingMaster = state.minimumMasterNodesOnPublishingMaster;
             this.fromDiff = false;
         }
 
@@ -731,11 +699,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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;
@@ -760,8 +723,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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 {
@@ -804,7 +766,9 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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();
     }
 
@@ -830,8 +794,8 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
                 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
         }
     }
 
@@ -855,8 +819,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
 
         private final Diff<ImmutableOpenMap<String, Custom>> customs;
 
-        private final int minimumMasterNodesOnPublishingMaster;
-
         ClusterStateDiff(ClusterState before, ClusterState after) {
             fromUuid = before.stateUUID;
             toUuid = after.stateUUID;
@@ -867,7 +829,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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 {
@@ -880,7 +841,9 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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
@@ -894,8 +857,8 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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
             }
         }
 
@@ -916,7 +879,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
             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();
         }

+ 3 - 4
server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java

@@ -21,6 +21,8 @@ package org.elasticsearch.cluster;
 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.unit.TimeValue;
 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 @@ import static org.elasticsearch.common.settings.Setting.positiveTimeSetting;
  * 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);

+ 4 - 13
server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java

@@ -49,9 +49,7 @@ import java.util.stream.StreamSupport;
 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,15 +91,14 @@ public class ClusterBootstrapService {
     }
 
     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));
     }
 
     void onFoundPeersUpdated() {
         final Set<DiscoveryNode> nodes = getDiscoveredNodes();
         if (bootstrappingPermitted.get() && transportService.getLocalNode().isMasterNode() && bootstrapRequirements.isEmpty() == false
-            && isBootstrappedSupplier.getAsBoolean() == false && nodes.stream().noneMatch(Coordinator::isZen1Node)) {
+            && isBootstrappedSupplier.getAsBoolean() == false) {
 
             final Tuple<Set<DiscoveryNode>,List<String>> requirementMatchingResult;
             try {
@@ -146,13 +143,8 @@ public class ClusterBootstrapService {
             @Override
             public void run() {
                 final Set<DiscoveryNode> discoveredNodes = getDiscoveredNodes();
-                final List<DiscoveryNode> 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
@@ -169,7 +161,6 @@ public class ClusterBootstrapService {
 
     private void startBootstrap(Set<DiscoveryNode> discoveryNodes, List<String> 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),

+ 10 - 24
server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java

@@ -32,8 +32,6 @@ import java.util.HashMap;
 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
  * <a href="https://github.com/elastic/elasticsearch-formal-models/blob/master/ZenWithTerms/tla/ZenWithTerms.tla">formal model</a>
@@ -86,10 +84,6 @@ public class CoordinationState {
         return getLastAcceptedState().version();
     }
 
-    private long getLastAcceptedVersionOrMetaDataVersion() {
-        return getLastAcceptedState().getVersionOrMetaDataVersion();
-    }
-
     public VotingConfiguration getLastCommittedConfiguration() {
         return getLastAcceptedState().getLastCommittedConfiguration();
     }
@@ -198,7 +192,7 @@ public class CoordinationState {
         publishVotes = new VoteCollection();
 
         return new Join(localNode, startJoinRequest.getSourceNode(), getCurrentTerm(), getLastAcceptedTerm(),
-            getLastAcceptedVersionOrMetaDataVersion());
+            getLastAcceptedVersion());
     }
 
     /**
@@ -231,12 +225,12 @@ public class CoordinationState {
                 " 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 class CoordinationState {
                 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 @@ public class CoordinationState {
                 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) {

+ 10 - 41
server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java

@@ -22,7 +22,6 @@ import org.apache.logging.log4j.LogManager;
 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.routing.allocation.AllocationService;
 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,9 +60,8 @@ import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.discovery.DiscoveryStats;
 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.discovery.SeedHostsResolver;
 import org.elasticsearch.threadpool.ThreadPool.Names;
 import org.elasticsearch.transport.TransportResponse.Empty;
 import org.elasticsearch.transport.TransportService;
@@ -88,8 +85,6 @@ import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK
 
 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
@@ -272,14 +267,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
                 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() &&
@@ -369,11 +356,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
                 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));
             }
         }
     }
@@ -384,11 +367,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
         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
@@ -612,7 +591,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
 
     private PreVoteResponse getPreVoteResponse() {
         return new PreVoteResponse(getCurrentTerm(), coordinationState.get().getLastAcceptedTerm(),
-            coordinationState.get().getLastAcceptedState().getVersionOrMetaDataVersion());
+            coordinationState.get().getLastAcceptedState().version());
     }
 
     // package-visible for testing
@@ -837,7 +816,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
         assert Thread.holdsLock(mutex) : "Coordinator mutex not held";
 
         final Set<DiscoveryNode> 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());
@@ -1062,6 +1041,10 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
         }
     }
 
+    public Collection<BiConsumer<DiscoveryNode, ClusterState>> getOnJoinValidators() {
+        return onJoinValidators;
+    }
+
     public enum Mode {
         CANDIDATE, LEADER, FOLLOWER
     }
@@ -1130,10 +1113,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
                         if (prevotingRound != null) {
                             prevotingRound.close();
                         }
-                        final List<DiscoveryNode> discoveredNodes
-                            = getDiscoveredNodes().stream().filter(n -> isZen1Node(n) == false).collect(Collectors.toList());
-
-                        prevotingRound = preVoteCollector.start(lastAcceptedState, discoveredNodes);
+                        prevotingRound = preVoteCollector.start(lastAcceptedState, getDiscoveredNodes());
                     }
                 }
             }
@@ -1358,15 +1338,4 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
             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")));
-    }
 }

+ 1 - 18
server/src/main/java/org/elasticsearch/cluster/coordination/FollowersChecker.java

@@ -22,8 +22,6 @@ package org.elasticsearch.cluster.coordination;
 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.Setting;
 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 class FollowersChecker {
         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 @@ public class FollowersChecker {
             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<Empty>() {
                     @Override

+ 4 - 41
server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java

@@ -37,8 +37,6 @@ import org.elasticsearch.common.settings.Setting;
 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 class JoinHelper {
         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) -> {
@@ -131,7 +124,7 @@ public class JoinHelper {
             });
 
         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() &&
@@ -143,21 +136,6 @@ public class JoinHelper {
                 joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState()));
                 channel.sendResponse(Empty.INSTANCE);
             });
-
-        transportService.registerRequestHandler(MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME,
-            MembershipAction.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 class JoinHelper {
         final Tuple<DiscoveryNode, JoinRequest> 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<Empty>() {
                     @Override
@@ -269,14 +238,8 @@ public class JoinHelper {
     }
 
     public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, ActionListener<TransportResponse.Empty> 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,
-            new MembershipAction.ValidateJoinRequest(state),
+        transportService.sendRequest(node, VALIDATE_JOIN_ACTION_NAME,
+            new ValidateJoinRequest(state),
             TransportRequestOptions.builder().withTimeout(joinTimeout).build(),
             new EmptyTransportResponseHandler(ThreadPool.Names.GENERIC) {
                 @Override

+ 0 - 5
server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java

@@ -30,7 +30,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
 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<JoinTaskExecut
 
     private final Logger logger;
 
-    private final int minimumMasterNodesOnLocalNode;
-
     public static class Task {
 
         private final DiscoveryNode node;
@@ -87,7 +84,6 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor<JoinTaskExecut
     public JoinTaskExecutor(Settings settings, AllocationService allocationService, Logger logger) {
         this.allocationService = allocationService;
         this.logger = logger;
-        minimumMasterNodesOnLocalNode = ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
     }
 
     @Override
@@ -191,7 +187,6 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor<JoinTaskExecut
         ClusterState tmpState = ClusterState.builder(currentState).nodes(nodesBuilder).blocks(ClusterBlocks.builder()
             .blocks(currentState.blocks())
             .removeGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_ID))
-            .minimumMasterNodesOnPublishingMaster(minimumMasterNodesOnLocalNode)
             .build();
         logger.trace("becomeMasterAndTrimConflictingNodes: {}", tmpState.nodes());
         allocationService.cleanCaches();

+ 1 - 27
server/src/main/java/org/elasticsearch/cluster/coordination/LeaderChecker.java

@@ -22,7 +22,6 @@ package org.elasticsearch.cluster.coordination;
 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.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
 import org.elasticsearch.common.Nullable;
@@ -32,7 +31,6 @@ import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.discovery.zen.MasterFaultDetection;
 import org.elasticsearch.threadpool.ThreadPool.Names;
 import org.elasticsearch.transport.ConnectTransportException;
 import org.elasticsearch.transport.TransportConnectionListener;
@@ -103,16 +101,6 @@ public class LeaderChecker {
                 channel.sendResponse(Empty.INSTANCE);
             });
 
-        transportService.registerRequestHandler(MasterFaultDetection.MASTER_PING_ACTION_NAME, MasterFaultDetection.MasterPingRequest::new,
-            Names.SAME, false, false, (request, channel, task) -> {
-                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 @@ public class LeaderChecker {
 
             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<TransportResponse.Empty>() {

+ 0 - 6
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<ClusterBlock> 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<ClusterBlock> 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) {

+ 1 - 1
server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStateStats.java → 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;

+ 1 - 3
server/src/main/java/org/elasticsearch/cluster/coordination/PreVoteCollector.java

@@ -38,7 +38,6 @@ import java.io.IOException;
 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 @@ public class PreVoteCollector {
         }
 
         void start(final Iterable<DiscoveryNode> 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<PreVoteResponse>() {
@@ -182,7 +180,7 @@ public class PreVoteCollector {
 
             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;
             }

+ 2 - 27
server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java

@@ -40,13 +40,10 @@ import org.elasticsearch.common.io.stream.StreamInput;
 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.discovery.zen.PublishClusterStateStats;
 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;
@@ -55,7 +52,6 @@ import org.elasticsearch.transport.TransportService;
 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;
@@ -195,16 +191,7 @@ public class PublicationTransportHandler {
             @Override
             public void sendApplyCommit(DiscoveryNode destination, ApplyCommitRequest applyCommitRequest,
                                         ActionListener<TransportResponse.Empty> 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<TransportResponse.Empty>() {
 
                         @Override
@@ -268,19 +255,7 @@ public class PublicationTransportHandler {
                         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);

+ 1 - 1
server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateStats.java → 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;

+ 0 - 1
server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java

@@ -97,7 +97,6 @@ public class Reconfigurator {
      */
     public VotingConfiguration reconfigure(Set<DiscoveryNode> liveNodes, Set<String> 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);

+ 29 - 5
server/src/main/java/org/elasticsearch/discovery/zen/PingContextProvider.java → server/src/main/java/org/elasticsearch/cluster/coordination/ValidateJoinRequest.java

@@ -16,13 +16,37 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-package org.elasticsearch.discovery.zen;
+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);
+    }
 
-public interface PingContextProvider {
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        super.writeTo(out);
+        this.state.writeTo(out);
+    }
 
-    /** return the current cluster state of the node */
-    ClusterState clusterState();
+    public ClusterState getState() {
+        return state;
+    }
 }

+ 4 - 31
server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

@@ -64,13 +64,9 @@ import org.elasticsearch.common.util.PageCacheRecycler;
 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;
-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;
 import org.elasticsearch.gateway.GatewayService;
@@ -228,11 +224,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
                     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,
-                    NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING,
                     GatewayService.EXPECTED_DATA_NODES_SETTING,
                     GatewayService.EXPECTED_MASTER_NODES_SETTING,
                     GatewayService.EXPECTED_NODES_SETTING,
@@ -290,7 +282,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
                     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,
@@ -385,30 +376,12 @@ public final class ClusterSettings extends AbstractScopedSettings {
                     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,
-                    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,
-                    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,
                     SearchService.DEFAULT_KEEPALIVE_SETTING,
                     SearchService.KEEPALIVE_INTERVAL_SETTING,
                     SearchService.MAX_KEEPALIVE_SETTING,

+ 2 - 21
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.Setting.Property;
 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;
@@ -67,14 +66,10 @@ import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
 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<String> DISCOVERY_TYPE_SETTING =
         new Setting<>("discovery.type", ZEN2_DISCOVERY_TYPE, Function.identity(), Property.NodeScope);
-    public static final Setting<List<String>> LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING =
-        Setting.listSetting("discovery.zen.hosts_provider", Collections.emptyList(), Function.identity(),
-            Property.NodeScope, Property.Deprecated);
     public static final Setting<List<String>> DISCOVERY_SEED_PROVIDERS_SETTING =
         Setting.listSetting("discovery.seed_providers", Collections.emptyList(), Function.identity(),
             Property.NodeScope);
@@ -101,7 +96,7 @@ public class DiscoveryModule {
             }
         }
 
-        List<String> seedProviderNames = getSeedProviderNames(settings);
+        List<String> seedProviderNames = DISCOVERY_SEED_PROVIDERS_SETTING.get(settings);
         // for bwc purposes, add settings provider even if not explicitly specified
         if (seedProviderNames.contains("settings") == false) {
             List<String> extendedSeedProviderNames = new ArrayList<>();
@@ -128,9 +123,6 @@ public class DiscoveryModule {
         };
 
         Map<String, Supplier<Discovery>> 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,
@@ -146,17 +138,6 @@ public class DiscoveryModule {
         discovery = Objects.requireNonNull(discoverySupplier.get());
     }
 
-    private List<String> 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;
     }

+ 0 - 93
server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java

@@ -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<TimeValue> 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<TimeValue> 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<Boolean> PUBLISH_DIFF_ENABLE_SETTING =
-        Setting.boolSetting("discovery.zen.publish_diff.enable", true, Property.Dynamic, Property.NodeScope, Property.Deprecated);
-    public static final Setting<TimeValue> 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;}
-
-}

+ 2 - 2
server/src/main/java/org/elasticsearch/discovery/DiscoveryStats.java

@@ -25,8 +25,8 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 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;
 

+ 86 - 8
server/src/main/java/org/elasticsearch/discovery/SeedHostsResolver.java

@@ -23,24 +23,40 @@ import org.apache.logging.log4j.LogManager;
 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, SeedHostsProvider.HostsResolver {
+    public static final Setting<Integer> DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING =
+        Setting.intSetting("discovery.seed_resolver.max_concurrent_resolvers", 10, 0, Setting.Property.NodeScope);
+    public static final Setting<TimeValue> DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING =
+        Setting.positiveTimeSetting("discovery.seed_resolver.timeout", TimeValue.timeValueSeconds(5), Setting.Property.NodeScope);
 
-public class SeedHostsResolver extends AbstractLifecycleComponent implements ConfiguredHostsResolver {
     private static final Logger logger = LogManager.getLogger(SeedHostsResolver.class);
 
     private final Settings settings;
@@ -58,8 +74,73 @@ public class SeedHostsResolver extends AbstractLifecycleComponent implements Con
         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) {
+        return DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING.get(settings);
+    }
+
+    public static TimeValue getResolveTimeout(Settings settings) {
+        return DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING.get(settings);
+    }
+
+    @Override
+    public List<TransportAddress> resolveHosts(
+        final List<String> hosts,
+        final int limitPortCounts) {
+        Objects.requireNonNull(hosts);
+        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<Callable<TransportAddress[]>> callables =
+            hosts
+                .stream()
+                .map(hn -> (Callable<TransportAddress[]>) () -> transportService.addressesFromString(hn, limitPortCounts))
+                .collect(Collectors.toList());
+        final List<Future<TransportAddress[]>> futures;
+        try {
+            futures = executorService.get().invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            return Collections.emptyList();
+        }
+        final List<TransportAddress> transportAddresses = new ArrayList<>();
+        final Set<TransportAddress> 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<String> it = hosts.iterator();
+        for (final Future<TransportAddress[]> 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
@@ -100,10 +181,7 @@ public class SeedHostsResolver extends AbstractLifecycleComponent implements Con
                         return;
                     }
 
-                    List<TransportAddress> providedAddresses
-                        = hostsProvider.getSeedAddresses((hosts, limitPortCounts)
-                        -> UnicastZenPing.resolveHostsLists(executorService.get(), logger, hosts, limitPortCounts,
-                        transportService, resolveTimeout));
+                    List<TransportAddress> providedAddresses = hostsProvider.getSeedAddresses(SeedHostsResolver.this);
 
                     consumer.accept(providedAddresses);
                 }

+ 1 - 13
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<List<String>> 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<List<String>> 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;

+ 0 - 223
server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java

@@ -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<Integer> 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<DiscoveryNode> nodes) {
-        int count = 0;
-        for (DiscoveryNode node : nodes) {
-            if (node.isMasterNode()) {
-                count++;
-            }
-        }
-        return count;
-    }
-
-    public boolean hasEnoughCandidates(Collection<MasterCandidate> 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<MasterCandidate> candidates) {
-        assert hasEnoughCandidates(candidates);
-        List<MasterCandidate> 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<DiscoveryNode> activeMasters) {
-        return activeMasters.stream().min(ElectMasterService::compareNodes).get();
-    }
-
-    public boolean hasEnoughMasterNodes(Iterable<DiscoveryNode> nodes) {
-        final int count = countMasterNodes(nodes);
-        return count > 0 && (minimumMasterNodes < 0 || count >= minimumMasterNodes);
-    }
-
-    public boolean hasTooManyMasterNodes(Iterable<DiscoveryNode> 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<DiscoveryNode> sortByMasterLikelihood(Iterable<DiscoveryNode> nodes) {
-        ArrayList<DiscoveryNode> sortedNodes = CollectionUtils.iterableAsArrayList(nodes);
-        CollectionUtil.introSort(sortedNodes, ElectMasterService::compareNodes);
-        return sortedNodes;
-    }
-
-    /**
-     * Returns a list of the next possible masters.
-     */
-    public DiscoveryNode[] nextPossibleMasters(ObjectContainer<DiscoveryNode> nodes, int numberOfPossibleMasters) {
-        List<DiscoveryNode> sortedNodes = sortedMasterNodes(Arrays.asList(nodes.toArray(DiscoveryNode.class)));
-        if (sortedNodes == null) {
-            return new DiscoveryNode[0];
-        }
-        List<DiscoveryNode> 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<DiscoveryNode> sortedMasterNodes(Iterable<DiscoveryNode> nodes) {
-        List<DiscoveryNode> 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());
-    }
-}

+ 0 - 116
server/src/main/java/org/elasticsearch/discovery/zen/FaultDetection.java

@@ -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} &amp; {@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<Boolean> CONNECT_ON_NETWORK_DISCONNECT_SETTING =
-        Setting.boolSetting("discovery.zen.fd.connect_on_network_disconnect", false, Property.NodeScope, Property.Deprecated);
-    public static final Setting<TimeValue> PING_INTERVAL_SETTING =
-        Setting.positiveTimeSetting("discovery.zen.fd.ping_interval", timeValueSeconds(1), Property.NodeScope, Property.Deprecated);
-    public static final Setting<TimeValue> PING_TIMEOUT_SETTING =
-        Setting.timeSetting("discovery.zen.fd.ping_timeout", timeValueSeconds(30), Property.NodeScope, Property.Deprecated);
-    public static final Setting<Integer> PING_RETRIES_SETTING =
-        Setting.intSetting("discovery.zen.fd.ping_retries", 3, Property.NodeScope, Property.Deprecated);
-    public static final Setting<Boolean> 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);
-        }
-    }
-
-}

+ 0 - 444
server/src/main/java/org/elasticsearch/discovery/zen/MasterFaultDetection.java

@@ -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<ClusterState> clusterStateSupplier;
-    private final CopyOnWriteArrayList<Listener> 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<ClusterState> 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<MasterPingResponseResponse>() {
-                        @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<MasterPingRequest> {
-
-        @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);
-        }
-    }
-}

+ 0 - 239
server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java

@@ -1,239 +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.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<BiConsumer<DiscoveryNode,ClusterState>> 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<JoinRequest> {
-
-        @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);
-                    }
-                }
-            });
-        }
-    }
-
-    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<ValidateJoinRequest> {
-        private final Supplier<DiscoveryNode> localNodeSupplier;
-        private final Collection<BiConsumer<DiscoveryNode, ClusterState>> joinValidators;
-
-        ValidateJoinRequestRequestHandler(Supplier<DiscoveryNode> localNodeSupplier,
-                                          Collection<BiConsumer<DiscoveryNode, ClusterState>> 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.state));
-            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<LeaveRequest> {
-
-        @Override
-        public void messageReceived(LeaveRequest request, TransportChannel channel, Task task) throws Exception {
-            listener.onLeave(request.node);
-            channel.sendResponse(TransportResponse.Empty.INSTANCE);
-        }
-    }
-}

+ 0 - 381
server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java

@@ -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
-     * <p>
-     * You must start accumulating joins before calling this method. See {@link #startElectionContext()}
-     * <p>
-     * 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.
-     * <p>
-     * 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<DiscoveryNode, List<MembershipAction.JoinCallback>> 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<JoinTaskExecutor.Task, ClusterStateTaskListener> getPendingAsTasks(String reason) {
-            Map<JoinTaskExecutor.Task, ClusterStateTaskListener> 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<JoinTaskExecutor.Task, ClusterStateTaskListener> 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<JoinTaskExecutor.Task, ClusterStateTaskListener> 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<MembershipAction.JoinCallback> callbacks;
-        private final Logger logger;
-
-        JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) {
-            this(Collections.singletonList(callback), logger);
-        }
-
-        JoinTaskListener(List<MembershipAction.JoinCallback> 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);
-                }
-            }
-        }
-    }
-
-}

+ 0 - 370
server/src/main/java/org/elasticsearch/discovery/zen/NodesFaultDetection.java

@@ -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<Listener> listeners = new CopyOnWriteArrayList<>();
-
-    private final ConcurrentMap<DiscoveryNode, NodeFD> nodesFD = newConcurrentMap();
-
-    private final Supplier<ClusterState> clusterStateSupplier;
-
-    private volatile DiscoveryNode localNode;
-
-    public NodesFaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService,
-                               Supplier<ClusterState> 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<DiscoveryNode> 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<PingResponse>() {
-                        @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<PingRequest> {
-        @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);
-        }
-    }
-}

+ 0 - 309
server/src/main/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueue.java

@@ -1,309 +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.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.
- * <p>
- * 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 &amp; 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.
- * <p>
- * The class is fully thread safe and can be used concurrently.
- */
-public class PendingClusterStatesQueue {
-
-    interface StateProcessedListener {
-
-        void onNewClusterStateProcessed();
-
-        void onNewClusterStateFailed(Exception e);
-    }
-
-    final ArrayList<ClusterStateContext> 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<ClusterStateContext> 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.
-     * <p>
-     * 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<ClusterStateContext> 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.
-     * <p>
-     * 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<ClusterState> 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);
-    }
-
-}

+ 0 - 663
server/src/main/java/org/elasticsearch/discovery/zen/PublishClusterStateAction.java

@@ -1,663 +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.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<Void> 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.
-     * <p>
-     * 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<DiscoveryNode> nodesToPublishTo;
-        final Map<Version, BytesReference> serializedStates;
-        final Map<Version, BytesReference> 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<DiscoveryNode> nodesToPublishTo,
-                              final SendingController sendingController, final Discovery.AckListener ackListener,
-                              final boolean sendFullVersion, final Map<Version, BytesReference> serializedStates,
-                              final Map<Version, BytesReference> 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<DiscoveryNode> 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<DiscoveryNode> nodesToPublishTo,
-                                             boolean sendFullVersion, Map<Version, BytesReference> serializedStates,
-                                             Map<Version, BytesReference> serializedDiffs) {
-        Diff<ClusterState> 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<Version, BytesReference> 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<Version, BytesReference> serializedDiffs, Map<Version, BytesReference> 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<Version, BytesReference> 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<ClusterState> 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<Void>() {
-
-            @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<BytesTransportRequest> {
-
-        @Override
-        public void messageReceived(BytesTransportRequest request, final TransportChannel channel, Task task) throws Exception {
-            handleIncomingClusterStateRequest(request, channel);
-        }
-    }
-
-    private class CommitClusterStateRequestHandler implements TransportRequestHandler<CommitClusterStateRequest> {
-        @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<DiscoveryNode> 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());
-    }
-}

+ 0 - 698
server/src/main/java/org/elasticsearch/discovery/zen/UnicastZenPing.java

@@ -1,698 +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.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.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;
-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";
-
-    public static final Setting<Integer> 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<TimeValue> 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<Integer> DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING =
-        Setting.intSetting("discovery.seed_resolver.max_concurrent_resolvers", 10, 0, Property.NodeScope);
-    public static final Setting<TimeValue> 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;
-
-    private final PingContextProvider contextProvider;
-
-    private final AtomicInteger pingingRoundIdGenerator = new AtomicInteger();
-
-    private final Map<Integer, PingingRound> activePingingRounds = newConcurrentMap();
-
-    // a list of temporal responses a node will return for a request (holds responses from other nodes)
-    private final Queue<PingResponse> 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 = getMaxConcurrentResolvers(settings);
-        resolveTimeout = 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());
-    }
-
-    /**
-     * 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<TransportAddress> resolveHostsLists(
-        final ExecutorService executorService,
-        final Logger logger,
-        final List<String> 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<Callable<TransportAddress[]>> callables =
-            hosts
-                .stream()
-                .map(hn -> (Callable<TransportAddress[]>) () -> transportService.addressesFromString(hn, limitPortCounts))
-                .collect(Collectors.toList());
-        final List<Future<TransportAddress[]>> futures;
-        try {
-            futures = executorService.invokeAll(callables, resolveTimeout.nanos(), TimeUnit.NANOSECONDS);
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            return Collections.emptyList();
-        }
-        final List<TransportAddress> transportAddresses = new ArrayList<>();
-        final Set<TransportAddress> 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<String> it = hosts.iterator();
-        for (final Future<TransportAddress[]> 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,
-            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<PingCollection> 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<PingCollection> resultsConsumer,
-                        final TimeValue scheduleDuration,
-                        final TimeValue requestDuration) {
-        final List<TransportAddress> 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<DiscoveryNode> 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<TransportAddress, Connection> tempConnections = new HashMap<>();
-        private final KeyedLock<TransportAddress> connectionLock = new KeyedLock<>(true);
-        private final PingCollection pingCollection;
-        private final List<TransportAddress> seedAddresses;
-        private final Consumer<PingCollection> pingListener;
-        private final DiscoveryNode localNode;
-        private final ConnectionProfile connectionProfile;
-
-        private AtomicBoolean closed = new AtomicBoolean(false);
-
-        PingingRound(int id, List<TransportAddress> seedAddresses, Consumer<PingCollection> 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<TransportAddress> 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<Connection> 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<TransportAddress> 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<TransportAddress> uniqueAddresses = Stream.concat(pingingRound.getSeedAddresses().stream(),
-            temporalAddresses.stream()).distinct();
-
-        // resolve what we can via the latest cluster state
-        final Set<DiscoveryNode> 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<UnicastPingResponse> getPingResponseHandler(final PingingRound pingingRound,
-                                                                                   final DiscoveryNode node) {
-        return new TransportResponseHandler<UnicastPingResponse>() {
-
-            @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<PingResponse> pingResponses = CollectionUtils.iterableAsArrayList(temporalResponses);
-        pingResponses.add(createPingResponse(contextProvider.clusterState()));
-
-        return new UnicastPingResponse(request.id, pingResponses.toArray(new PingResponse[pingResponses.size()]));
-    }
-
-    class UnicastPingRequestHandler implements TransportRequestHandler<UnicastPingRequest> {
-
-        @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
-    }
-
-    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);
-    }
-}

+ 0 - 1251
server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java

@@ -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<TimeValue> PING_TIMEOUT_SETTING =
-        Setting.positiveTimeSetting("discovery.zen.ping_timeout", timeValueSeconds(3), Property.NodeScope);
-    public static final Setting<TimeValue> 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<Integer> JOIN_RETRY_ATTEMPTS_SETTING =
-        Setting.intSetting("discovery.zen.join_retry_attempts", 3, 1, Property.NodeScope, Property.Deprecated);
-    public static final Setting<TimeValue> JOIN_RETRY_DELAY_SETTING =
-        Setting.positiveTimeSetting("discovery.zen.join_retry_delay", TimeValue.timeValueMillis(100),
-            Property.NodeScope, Property.Deprecated);
-    public static final Setting<Integer> 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<Boolean> SEND_LEAVE_REQUEST_SETTING =
-        Setting.boolSetting("discovery.zen.send_leave_request", true, Property.NodeScope, Property.Deprecated);
-    public static final Setting<TimeValue> 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<Boolean> 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<Integer> 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<ClusterState> committedState; // last committed cluster state
-    private final Object stateMutex = new Object();
-    private final Collection<BiConsumer<DiscoveryNode, ClusterState>> onJoinValidators;
-
-    public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService transportService,
-                        NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, ClusterApplier clusterApplier,
-                        ClusterSettings clusterSettings, SeedHostsProvider hostsProvider, AllocationService allocationService,
-                        Collection<BiConsumer<DiscoveryNode, ClusterState>> 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<Void> 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<DiscoveryNode> 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<ZenPing.PingResponse> 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<ZenPing.PingResponse> pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger);
-
-        List<DiscoveryNode> 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<ElectMasterService.MasterCandidate> 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<ZenPing.PingResponse> filterPingResponses(List<ZenPing.PingResponse> fullPingResponses,
-                                                          boolean masterElectionIgnoreNonMasters, Logger logger) {
-        List<ZenPing.PingResponse> 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<ZenPing.PingCollection> 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<Void> 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<RejoinClusterRequest> {
-        @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<Thread> 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<BiConsumer<DiscoveryNode, ClusterState>> getOnJoinValidators() {
-        return onJoinValidators;
-    }
-
-    static class ZenNodeRemovalClusterStateTaskExecutor extends NodeRemovalClusterStateTaskExecutor {
-
-        private final ElectMasterService electMasterService;
-        private final Consumer<String> rejoin;
-
-        ZenNodeRemovalClusterStateTaskExecutor(
-            final AllocationService allocationService,
-            final ElectMasterService electMasterService,
-            final Consumer<String> rejoin,
-            final Logger logger) {
-            super(allocationService, logger);
-            this.electMasterService = electMasterService;
-            this.rejoin = rejoin;
-        }
-
-        @Override
-        protected ClusterTasksResult<Task> getTaskClusterTasksResult(ClusterState currentState, List<Task> tasks,
-                                                                     ClusterState remainingNodesClusterState) {
-            if (electMasterService.hasEnoughMasterNodes(remainingNodesClusterState.nodes()) == false) {
-                final ClusterTasksResult.Builder<Task> resultBuilder = ClusterTasksResult.<Task>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);
-            }
-        }
-    }
-}

+ 0 - 196
server/src/main/java/org/elasticsearch/discovery/zen/ZenPing.java

@@ -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<PingCollection> 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<DiscoveryNode, PingResponse> 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<PingResponse> toList() {
-            return new ArrayList<>(pings.values());
-        }
-
-        /** the number of nodes for which there are known pings */
-        public synchronized int size() {
-            return pings.size();
-        }
-    }
-}

+ 1 - 4
server/src/main/java/org/elasticsearch/gateway/Gateway.java

@@ -29,7 +29,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
 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 class Gateway {
         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 class Gateway {
         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()) {

+ 0 - 3
server/src/main/java/org/elasticsearch/gateway/GatewayService.java

@@ -40,7 +40,6 @@ import org.elasticsearch.common.settings.Settings;
 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 class GatewayService extends AbstractLifecycleComponent implements Cluste
         // 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;
         }

+ 4 - 2
server/src/main/java/org/elasticsearch/node/Node.java

@@ -86,7 +86,6 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 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<TimeValue> 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 class Node implements Closeable {
             : "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) {

+ 1 - 1
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<List<String>> 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() {

+ 2 - 2
server/src/test/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStatsTests.java

@@ -23,8 +23,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
 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;

+ 3 - 7
server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java

@@ -45,7 +45,6 @@ import java.util.concurrent.atomic.AtomicReference;
 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 class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
         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 class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
         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 class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
 
         // 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));
@@ -730,8 +729,5 @@ public class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
         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()));
     }
 }

+ 1 - 1
server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java

@@ -60,7 +60,7 @@ import static org.hamcrest.Matchers.notNullValue;
 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

+ 0 - 207
server/src/test/java/org/elasticsearch/cluster/ack/AckClusterUpdateSettingsIT.java

@@ -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<Class<? extends Plugin>> 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"));
-    }
-}

+ 0 - 317
server/src/test/java/org/elasticsearch/cluster/ack/AckIT.java

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

+ 0 - 30
server/src/test/java/org/elasticsearch/cluster/coordination/ClusterBootstrapServiceTests.java

@@ -43,7 +43,6 @@ import static java.util.Collections.emptyMap;
 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 @@ public class ClusterBootstrapServiceTests extends ESTestCase {
         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 class ClusterBootstrapServiceTests extends ESTestCase {
         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(),

+ 0 - 1
server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java

@@ -65,7 +65,6 @@ import org.elasticsearch.common.unit.TimeValue;
 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;

+ 6 - 7
server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java

@@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterState;
 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;
@@ -49,7 +48,7 @@ import static org.hamcrest.Matchers.containsString;
 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)
@@ -149,7 +148,7 @@ public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
     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 class ElasticsearchNodeCommandIT extends ESIntegTestCase {
     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 class ElasticsearchNodeCommandIT extends ESIntegTestCase {
 
         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 class ElasticsearchNodeCommandIT extends ESIntegTestCase {
 
         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)

+ 2 - 2
server/src/test/java/org/elasticsearch/discovery/zen/MembershipActionTests.java → 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.randomCompatibleVersion;
 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();

+ 0 - 33
server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java

@@ -25,7 +25,6 @@ import org.elasticsearch.test.ESTestCase;
 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 @@ public class NoMasterBlockServiceTests extends ESTestCase {
         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 class NoMasterBlockServiceTests extends ESTestCase {
             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 class NoMasterBlockServiceTests extends ESTestCase {
         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();
-    }
 }

+ 44 - 1
server/src/test/java/org/elasticsearch/cluster/coordination/PublicationTests.java

@@ -25,11 +25,12 @@ import org.elasticsearch.cluster.ClusterState;
 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.List;
 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 java.util.stream.Stream;
 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 @@ public class PublicationTests extends ESTestCase {
                 return ts.stream();
             });
     }
+
+    public static class AssertingAckListener implements Discovery.AckListener {
+        private final List<Tuple<DiscoveryNode, Throwable>> errors = new CopyOnWriteArrayList<>();
+        private final Set<DiscoveryNode> 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<DiscoveryNode> await(long timeout, TimeUnit unit) throws InterruptedException {
+            assertThat(awaitErrors(timeout, unit), emptyIterable());
+            assertTrue(commitCountDown.await(timeout, unit));
+            return new HashSet<>(successfulAcks);
+        }
+
+        public List<Tuple<DiscoveryNode, Throwable>> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException {
+            countDown.await(timeout, unit);
+            return errors;
+        }
+
+    }
 }

+ 0 - 15
server/src/test/java/org/elasticsearch/cluster/coordination/ZenDiscoveryIT.java

@@ -35,7 +35,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 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 @@ import static org.hamcrest.Matchers.notNullValue;
 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 class ZenDiscoveryIT extends ESIntegTestCase {
     }
 
     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();

+ 1 - 5
server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java

@@ -67,8 +67,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase {
             .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 class ClusterSerializationTests extends ESAllocationTestCase {
         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 {

+ 4 - 20
server/src/test/java/org/elasticsearch/discovery/DiscoveryDisruptionIT.java

@@ -26,9 +26,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
 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 java.util.concurrent.CountDownLatch;
 
 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 class DiscoveryDisruptionIT extends AbstractDisruptionTestCase {
         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);
@@ -117,7 +110,7 @@ public class DiscoveryDisruptionIT extends AbstractDisruptionTestCase {
         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 ...");
 
@@ -143,15 +136,6 @@ public class DiscoveryDisruptionIT extends AbstractDisruptionTestCase {
         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<String> nonPreferredNodes = new HashSet<>(nodes);
         nonPreferredNodes.remove(preferredMasterName);

+ 2 - 25
server/src/test/java/org/elasticsearch/discovery/DiscoveryModuleTests.java

@@ -30,7 +30,6 @@ import org.elasticsearch.common.settings.ClusterSettings;
 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;
@@ -117,27 +116,6 @@ public class DiscoveryModuleTests extends ESTestCase {
         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, () ->
@@ -194,15 +172,14 @@ public class DiscoveryModuleTests extends ESTestCase {
 
     public void testJoinValidator() {
         BiConsumer<DiscoveryNode, ClusterState> 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<DiscoveryNode, ClusterState> getJoinValidator() {
                 return consumer;
             }
         }));
-        ZenDiscovery discovery = (ZenDiscovery) module.getDiscovery();
+        Coordinator discovery = (Coordinator) module.getDiscovery();
         Collection<BiConsumer<DiscoveryNode, ClusterState>> onJoinValidators = discovery.getOnJoinValidators();
         assertEquals(2, onJoinValidators.size());
         assertTrue(onJoinValidators.contains(consumer));

+ 12 - 10
server/src/test/java/org/elasticsearch/discovery/FileBasedSeedHostsProviderTests.java

@@ -25,9 +25,7 @@ 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.discovery.zen.UnicastZenPing;
 import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.transport.MockTransportService;
@@ -115,11 +113,12 @@ public class FileBasedSeedHostsProviderTests extends ESTestCase {
     }
 
     public void testUnicastHostsDoesNotExist() {
-        final FileBasedSeedHostsProvider provider = new FileBasedSeedHostsProvider(createTempDir().toAbsolutePath());
-        final List<TransportAddress> addresses = provider.getSeedAddresses((hosts, limitPortCounts) ->
-            UnicastZenPing.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<TransportAddress> results = fileBasedSeedHostsProvider.getSeedAddresses(seedHostsResolver);
+        seedHostsResolver.stop();
+        assertEquals(0, results.size());
     }
 
     public void testInvalidHostEntries() throws Exception {
@@ -146,8 +145,11 @@ public class FileBasedSeedHostsProviderTests extends ESTestCase {
             writer.write(String.join("\n", hostEntries));
         }
 
-        return new FileBasedSeedHostsProvider(configPath).getSeedAddresses((hosts, limitPortCounts) ->
-            UnicastZenPing.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<TransportAddress> results = fileBasedSeedHostsProvider.getSeedAddresses(seedHostsResolver);
+        seedHostsResolver.stop();
+        return results;
     }
 }

+ 0 - 38
server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java

@@ -35,8 +35,6 @@ import org.elasticsearch.common.collect.Tuple;
 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 class MasterDisruptionIT extends AbstractDisruptionTestCase {
         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 class MasterDisruptionIT extends AbstractDisruptionTestCase {
 
         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 class MasterDisruptionIT extends AbstractDisruptionTestCase {
         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 class MasterDisruptionIT extends AbstractDisruptionTestCase {
         });
 
     }
-
-    private void assertDiscoveryCompleted(List<String> 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
-                    )
-            );
-        }
-    }
 }

+ 316 - 2
server/src/test/java/org/elasticsearch/discovery/SeedHostsResolverTests.java

@@ -19,22 +19,53 @@
 
 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.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;
 import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.transport.nio.MockNioTransport;
 import org.junit.After;
 import org.junit.Before;
 
+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.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.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -44,6 +75,10 @@ public class SeedHostsResolverTests extends ESTestCase {
     private List<TransportAddress> transportAddresses;
     private SeedHostsResolver seedHostsResolver;
     private ThreadPool threadPool;
+    private ExecutorService executorService;
+    // close in reverse order as opened
+    private Stack<Closeable> closeables;
+
 
     @Before
     public void startResolver() {
@@ -53,14 +88,38 @@ public class SeedHostsResolverTests extends ESTestCase {
         TransportService transportService = mock(TransportService.class);
         when(transportService.getThreadPool()).thenReturn(threadPool);
 
+        recreateSeedHostsResolver(transportService);
+
+        final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory("[" + getClass().getName() + "]");
+        executorService =
+            EsExecutors.newScaling(
+                getClass().getName() + "/" + getTestName(), 0, 2, 60, TimeUnit.SECONDS, threadFactory, threadPool.getThreadContext());
+        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() {
+    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<Closeable> 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 +151,259 @@ public class SeedHostsResolverTests extends ESTestCase {
         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);
+        recreateSeedHostsResolver(transportService);
+        final int limitPortCounts = randomIntBetween(1, 10);
+        final List<TransportAddress> transportAddresses = seedHostsResolver.resolveHosts(Collections.singletonList("127.0.0.1"),
+            limitPortCounts);
+        assertThat(transportAddresses, hasSize(limitPortCounts));
+        final Set<Integer> 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).boxed().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);
+        recreateSeedHostsResolver(transportService);
+        final List<TransportAddress> transportAddresses = seedHostsResolver.resolveHosts(
+            Collections.singletonList(NetworkAddress.format(loopbackAddress)),
+            10);
+        assertThat(transportAddresses, hasSize(7));
+        final Set<Integer> ports = new HashSet<>();
+        for (final TransportAddress address : transportAddresses) {
+            assertTrue(address.address().getAddress().isLoopbackAddress());
+            ports.add(address.getPort());
+        }
+        assertThat(ports, equalTo(IntStream.range(9303, 9310).boxed().collect(Collectors.toSet())));
+    }
+
+    public void testUnknownHost() throws IllegalAccessException {
+        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);
+        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<TransportAddress> transportAddresses = seedHostsResolver.resolveHosts(Collections.singletonList(hostname), 1);
+
+            assertThat(transportAddresses, empty());
+            appender.assertAllExpectationsMatched();
+        } finally {
+            Loggers.removeAppender(logger, appender);
+            appender.stop();
+        }
+    }
+
+    public void testResolveTimeout() throws IllegalAccessException {
+        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);
+        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 {
+            Loggers.addAppender(logger, appender);
+            final List<TransportAddress> transportAddresses = seedHostsResolver.resolveHosts(Arrays.asList("hostname1", "hostname2"), 1);
+
+            assertThat(transportAddresses, hasSize(1));
+            appender.assertAllExpectationsMatched();
+        } finally {
+            Loggers.removeAppender(logger, appender);
+            appender.stop();
+            latch.countDown();
+        }
+    }
+
+    public void testInvalidHosts() throws IllegalAccessException {
+        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);
+        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<TransportAddress> 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();
+        }
+    }
 }

+ 0 - 18
server/src/test/java/org/elasticsearch/discovery/SettingsBasedSeedHostsProviderTests.java

@@ -22,7 +22,6 @@ import org.elasticsearch.common.settings.Settings;
 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 class SettingsBasedSeedHostsProviderTests extends ESTestCase {
             .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));
-    }
 }

+ 0 - 359
server/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java

@@ -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<String> 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<ClusterState> 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<String> 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<ClusterState> 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<ClusterState> 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<Tuple<DiscoveryNode, Long>> inflightPings = Collections.newSetFromMap(new ConcurrentHashMap<>());
-        private final Set<Tuple<DiscoveryNode, Long>> 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<DiscoveryNode, Long> 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();
-        }
-    }
-}

+ 0 - 62
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 @@ public class SingleNodeDiscoveryIT extends ESIntegTestCase {
                 .build();
     }
 
-    public void testDoesNotRespondToZenPings() throws Exception {
-        final Settings settings =
-                Settings.builder().put("cluster.name", internalCluster().getClusterName()).build();
-        final Version version = Version.CURRENT;
-        final Stack<Closeable> 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<ZenPing.PingCollection> 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();

+ 0 - 157
server/src/test/java/org/elasticsearch/discovery/zen/ElectMasterServiceTests.java

@@ -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<DiscoveryNode> generateRandomNodes() {
-        int count = scaledRandomIntBetween(1, 100);
-        ArrayList<DiscoveryNode> nodes = new ArrayList<>(count);
-        for (int i = 0; i < count; i++) {
-            Set<DiscoveryNode.Role> 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<MasterCandidate> generateRandomCandidates() {
-        int count = scaledRandomIntBetween(1, 100);
-        ArrayList<MasterCandidate> candidates = new ArrayList<>(count);
-        for (int i = 0; i < count; i++) {
-            Set<DiscoveryNode.Role> 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<DiscoveryNode> nodes = generateRandomNodes();
-        List<DiscoveryNode> 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<DiscoveryNode> 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<DiscoveryNode> 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<MasterCandidate> 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<MasterCandidate> 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<DiscoveryNode> nodes = generateRandomNodes();
-        ElectMasterService service = electMasterService();
-
-        int masterNodes = 0;
-
-        for (DiscoveryNode node : nodes) {
-            if (node.isMasterNode()) {
-                masterNodes++;
-            }
-        }
-
-        assertEquals(masterNodes, service.countMasterNodes(nodes));
-    }
-}

+ 0 - 868
server/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java

@@ -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<DiscoveryNode> 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<Future<Void>> 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<Void> joinFuture : pendingJoins) {
-                assertThat(joinFuture.isDone(), equalTo(true));
-            }
-        }
-        for (Future<Void> 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<Future<Void>> pendingJoins = new ArrayList<>();
-        nodeJoinController.startElectionContext();
-        for (int i = 1 + randomInt(5); i > 0; i--) {
-            DiscoveryNode node = newNode(nodeId++);
-            final Future<Void> future = joinNodeAsync(node);
-            pendingJoins.add(future);
-            assertThat(future.isDone(), equalTo(false));
-        }
-        nodeJoinController.stopElectionContext("test");
-        for (Future<Void> 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<SimpleFuture> pendingJoins = new ArrayList<>();
-        ArrayList<DiscoveryNode> 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<SimpleFuture> pendingJoins = new ArrayList<>();
-        ArrayList<DiscoveryNode> 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<Throwable> 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<DiscoveryNode> nodes = new ArrayList<>();
-        nodes.add(discoveryState(masterService).nodes().getLocalNode());
-        final CyclicBarrier barrier = new CyclicBarrier(threads.length);
-        final List<Throwable> 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<DiscoveryNode> nodes = new ArrayList<>();
-        nodes.add(discoveryState(masterService).nodes().getLocalNode());
-        final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1);
-        final List<Throwable> 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<Throwable> 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<ShardRouting> 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<DiscoveryNode> 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<Void> {
-        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<DiscoveryNode.Role> 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);
-    }
-}

+ 0 - 268
server/src/test/java/org/elasticsearch/discovery/zen/PendingClusterStatesQueueTests.java

@@ -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<ClusterState> 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<ClusterState> stateToDrop = states.subList(0, numberOfStateToDrop);
-        final int queueSize = states.size() - numberOfStateToDrop;
-        PendingClusterStatesQueue queue = createQueueWithStates(stateToDrop, queueSize);
-        List<ClusterStateContext> 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<ClusterState> 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<ClusterState> states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4");
-        PendingClusterStatesQueue queue = createQueueWithStates(states);
-        List<ClusterStateContext> 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<ClusterState> states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4");
-        PendingClusterStatesQueue queue = createQueueWithStates(states);
-        List<ClusterStateContext> committedContexts = randomCommitStates(queue);
-        ClusterState toFail = randomFrom(committedContexts).state;
-        queue.markAsFailed(toFail, new ElasticsearchException("boo!"));
-        final Map<String, ClusterStateContext> 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<ClusterState> states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4");
-        PendingClusterStatesQueue queue = createQueueWithStates(states);
-        List<ClusterStateContext> 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<ClusterState> 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<ClusterStateContext> 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<ClusterStateContext> randomCommitStates(PendingClusterStatesQueue queue) {
-        List<ClusterStateContext> 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<ClusterState> states) {
-        return createQueueWithStates(states, states.size() * 2); // we don't care about limits (there are dedicated tests for that)
-    }
-
-    PendingClusterStatesQueue createQueueWithStates(List<ClusterState> states, int maxQueueSize) {
-        PendingClusterStatesQueue queue;
-        queue = new PendingClusterStatesQueue(logger, maxQueueSize);
-        for (ClusterState state : states) {
-            queue.addPending(state);
-        }
-        return queue;
-    }
-
-    List<ClusterState> randomStates(int count, String... masters) {
-        ArrayList<ClusterState> 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;
-        }
-    }
-
-}

+ 0 - 961
server/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java

@@ -1,961 +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.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<String, MockNode> 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<Void> 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<String, MockNode> 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<ClusterState> diff(ClusterState previousState) {
-                return new Diff<ClusterState>() {
-                    @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<ClusterState> 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<String, IndexMetaData> 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<Tuple<DiscoveryNode, Throwable>> errors = new CopyOnWriteArrayList<>();
-        private final Set<DiscoveryNode> 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<DiscoveryNode> await(long timeout, TimeUnit unit) throws InterruptedException {
-            assertThat(awaitErrors(timeout, unit), emptyIterable());
-            assertTrue(commitCountDown.await(timeout, unit));
-            return new HashSet<>(successfulAcks);
-        }
-
-        public List<Tuple<DiscoveryNode, Throwable>> 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<TransportResponse> response = new AtomicReference<>();
-        AtomicReference<Throwable> 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";
-        }
-    }
-}

+ 0 - 940
server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java

@@ -1,940 +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.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<Closeable> 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<Closeable> 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<Settings, Version, Transport> 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<ZenPing.PingResponse> 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<String, TransportAddress[]> addresses = new HashMap<>();
-        final BiFunction<Settings, Version, Transport> 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<ZenPing.PingResponse> 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<TransportAddress, Integer> 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<ZenPing.PingResponse> secondPingResponses = zenPingA.pingAndWait().toList();
-            assertThat(secondPingResponses.size(), equalTo(2));
-            final Set<String> 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<TransportAddress> transportAddresses = UnicastZenPing.resolveHostsLists(
-            executorService,
-            logger,
-            Collections.singletonList("127.0.0.1"),
-            limitPortCounts,
-            transportService,
-            TimeValue.timeValueSeconds(30));
-        assertThat(transportAddresses, hasSize(limitPortCounts));
-        final Set<Integer> 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<TransportAddress> transportAddresses = UnicastZenPing.resolveHostsLists(
-            executorService,
-            logger,
-            Collections.singletonList(NetworkAddress.format(loopbackAddress)),
-            10,
-            transportService,
-            TimeValue.timeValueSeconds(30));
-        assertThat(transportAddresses, hasSize(7));
-        final Set<Integer> 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<TransportAddress> transportAddresses = UnicastZenPing.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<TransportAddress> transportAddresses = UnicastZenPing.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<Settings, Version, Transport> 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<ZenPing.PingResponse> 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<Settings, Version, Transport> 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<ZenPing.PingResponse> 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<ZenPing.PingResponse> 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<TransportAddress> transportAddresses = UnicastZenPing.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<Settings, Version, Transport> 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<Settings, Version, Transport> supplier,
-        final Set<Role> 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<TransportAddress, AtomicInteger> 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<TransportAddress, AtomicInteger> counters;
-
-        NetworkHandle(
-            final TransportAddress address,
-            final TransportService transportService,
-            final DiscoveryNode discoveryNode,
-            final ConcurrentMap<TransportAddress, AtomicInteger> 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<PingCollection> 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<UnicastPingResponse> getPingResponseHandler(PingingRound pingingRound, DiscoveryNode node) {
-            markTaskAsStarted("ping [" + node + "]");
-            TransportResponseHandler<UnicastPingResponse> original = super.getPingResponseHandler(pingingRound, node);
-            return new TransportResponseHandler<UnicastPingResponse>() {
-                @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();
-                }
-            };
-        }
-    }
-
-}

+ 0 - 601
server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java

@@ -1,601 +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.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<ZenPing.PingResponse> responses = new ArrayList<>();
-        ArrayList<DiscoveryNode> masterNodes = new ArrayList<>();
-        ArrayList<DiscoveryNode> allNodes = new ArrayList<>();
-        for (int i = randomIntBetween(10, 20); i >= 0; i--) {
-            Set<Role> 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<ZenPing.PingResponse> filtered = ZenDiscovery.filterPingResponses(responses, ignore, logger);
-        final List<DiscoveryNode> 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<Closeable> toClose = new ArrayDeque<>();
-        try {
-            Set<DiscoveryNode> 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<Closeable> 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<Void> {
-        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<ClusterState> 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<DiscoveryNode> fdNodesForState(ClusterState clusterState, DiscoveryNode localNode) {
-        final Set<DiscoveryNode> 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 MembershipAction.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() {
-
-                    @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<String> submitRejoin = source -> rejoinCalled.set(true);
-
-        final AtomicReference<ClusterState> 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<NodeRemovalClusterStateTaskExecutor.Task> 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<NodeRemovalClusterStateTaskExecutor.Task> 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);
-    }
-}

+ 0 - 90
server/src/test/java/org/elasticsearch/discovery/zen/ZenPingTests.java

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

+ 3 - 4
server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java

@@ -43,7 +43,6 @@ import org.elasticsearch.common.Priority;
 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;
@@ -492,7 +491,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase {
         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 {
@@ -506,7 +505,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase {
         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();
@@ -514,7 +513,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase {
         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);
     }
 }

+ 1 - 27
test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java

@@ -106,9 +106,6 @@ import org.elasticsearch.common.xcontent.XContentType;
 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.AtomicInteger;
 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.assertAcke
 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 @@ public abstract class ESIntegTestCase extends ESTestCase {
                         assertThat("test leaves persistent cluster metadata behind", persistentKeys, empty());
 
                         final Set<String> 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 abstract class ESIntegTestCase extends ESTestCase {
         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;
 

+ 1 - 92
test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

@@ -30,7 +30,6 @@ import com.carrotsearch.randomizedtesting.generators.RandomStrings;
 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.EsExecutors;
 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;
@@ -151,15 +148,12 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
 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.discovery.DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING;
-import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_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;
 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;
@@ -412,10 +406,6 @@ public final class InternalTestCluster extends TestCluster {
                 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.
@@ -433,11 +423,6 @@ public final class InternalTestCluster extends TestCluster {
         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);
     }
@@ -648,25 +633,10 @@ public final class InternalTestCluster extends TestCluster {
 
         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("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 +945,6 @@ public final class InternalTestCluster extends TestCluster {
             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 +976,6 @@ public final class InternalTestCluster extends TestCluster {
                     .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<Class<? extends Plugin>> plugins = node.getClasspathPlugins();
             node = new MockNode(finalSettings, plugins);
             node.injector().getInstance(TransportService.class).addLifecycleListener(new LifecycleListener() {
@@ -1126,9 +1074,6 @@ public final class InternalTestCluster extends TestCluster {
             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 +1600,6 @@ public final class InternalTestCluster extends TestCluster {
                 .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<Future<?>> futures = nodeAndClients.stream().map(node -> executor.submit(node::startNode)).collect(Collectors.toList());
 
@@ -1678,7 +1618,6 @@ public final class InternalTestCluster extends TestCluster {
                 getMinMasterNodes(currentMasters + newMasters) > currentMasters) {
                 // update once masters have joined
                 validateClusterFormed();
-                updateMinMasterNodes(currentMasters + newMasters);
             }
         }
     }
@@ -1813,10 +1752,6 @@ public final class InternalTestCluster extends TestCluster {
             // the fact it left
             validateClusterFormed(nodeAndClient.name);
         }
-
-        if (excludedNodeIds.isEmpty() == false) {
-            updateMinMasterNodes(getMasterNodesCount());
-        }
     }
 
     private NodeAndClient removeNode(NodeAndClient nodeAndClient) {
@@ -1852,10 +1787,6 @@ public final class InternalTestCluster extends TestCluster {
                     throw new AssertionError("unexpected", e);
                 }
             }
-
-            if (stoppingMasters > 0) {
-                updateMinMasterNodes(getMasterNodesCount() - Math.toIntExact(stoppingMasters));
-            }
         }
         return excludedNodeIds;
     }
@@ -2146,28 +2077,6 @@ public final class InternalTestCluster extends TestCluster {
                 .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;

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java

@@ -81,7 +81,7 @@ import java.util.function.Supplier;
  * 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 {

+ 2 - 2
test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java

@@ -27,9 +27,9 @@ import org.elasticsearch.common.network.NetworkModule;
 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 class InternalTestClusterTests extends ESTestCase {
                 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();

+ 1 - 1
x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java

@@ -66,7 +66,7 @@ import static org.hamcrest.Matchers.hasItem;
 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 =

+ 1 - 2
x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java

@@ -66,7 +66,6 @@ import java.util.stream.Collectors;
 
 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 class SecurityTests extends ESTestCase {
         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));

+ 0 - 2
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 }

+ 0 - 2
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 }

+ 0 - 2
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 }