Browse Source

Remove DiscoveryService and reduce guice to just Discovery #16821

DiscoveryService was a bridge into the discovery universe. This is unneeded and we can just access discovery directly or do things in a different way.

One of those different ways, is not having a dedicated discovery implementation for each our dicovery plugins but rather reuse ZenDiscovery.

UnicastHostProviders are now classified by discovery type, removing unneeded checks on plugins.

Closes #16821
Boaz Leskes 9 years ago
parent
commit
195b43d66e
37 changed files with 254 additions and 543 deletions
  1. 5 0
      core/src/main/java/org/elasticsearch/cluster/ClusterService.java
  2. 1 2
      core/src/main/java/org/elasticsearch/cluster/ClusterState.java
  3. 6 0
      core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java
  4. 28 8
      core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java
  5. 1 1
      core/src/main/java/org/elasticsearch/common/Randomness.java
  6. 4 5
      core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
  7. 1 5
      core/src/main/java/org/elasticsearch/discovery/Discovery.java
  8. 14 5
      core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java
  9. 0 140
      core/src/main/java/org/elasticsearch/discovery/DiscoveryService.java
  10. 1 2
      core/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java
  11. 8 32
      core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java
  12. 7 34
      core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
  13. 2 7
      core/src/main/java/org/elasticsearch/gateway/GatewayService.java
  14. 81 30
      core/src/main/java/org/elasticsearch/node/Node.java
  15. 21 25
      core/src/main/java/org/elasticsearch/tribe/TribeService.java
  16. 2 3
      core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java
  17. 6 3
      core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java
  18. 1 1
      core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java
  19. 1 1
      core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java
  20. 2 2
      core/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java
  21. 6 11
      core/src/test/java/org/elasticsearch/test/NoopDiscovery.java
  22. 3 3
      plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureDiscoveryModule.java
  23. 0 48
      plugins/discovery-azure/src/main/java/org/elasticsearch/discovery/azure/AzureDiscovery.java
  24. 4 3
      plugins/discovery-azure/src/main/java/org/elasticsearch/plugin/discovery/azure/AzureDiscoveryPlugin.java
  25. 0 17
      plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/Ec2Module.java
  26. 0 49
      plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2Discovery.java
  27. 5 5
      plugins/discovery-ec2/src/main/java/org/elasticsearch/plugin/discovery/ec2/Ec2DiscoveryPlugin.java
  28. 0 16
      plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoverySettingsTests.java
  29. 0 59
      plugins/discovery-gce/src/main/java/org/elasticsearch/discovery/gce/GceDiscovery.java
  30. 9 1
      plugins/discovery-gce/src/main/java/org/elasticsearch/discovery/gce/GceUnicastHostsProvider.java
  31. 7 7
      plugins/discovery-gce/src/main/java/org/elasticsearch/plugin/discovery/gce/GceDiscoveryPlugin.java
  32. 4 4
      plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java
  33. 7 7
      qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java
  34. 2 2
      qa/evil-tests/src/test/resources/org/elasticsearch/tribe/elasticsearch.yml
  35. 6 5
      test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
  36. 5 0
      test/framework/src/main/java/org/elasticsearch/test/cluster/NoopClusterService.java
  37. 4 0
      test/framework/src/main/java/org/elasticsearch/test/cluster/TestClusterService.java

+ 5 - 0
core/src/main/java/org/elasticsearch/cluster/ClusterService.java

@@ -56,6 +56,11 @@ public interface ClusterService extends LifecycleComponent<ClusterService> {
      */
     void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException;
 
+    /**
+     * Remove an initial block to be set on the first cluster state created.
+     */
+    void removeInitialStateBlock(int blockId) throws IllegalStateException;
+
     /**
      * The operation routing.
      */

+ 1 - 2
core/src/main/java/org/elasticsearch/cluster/ClusterState.java

@@ -51,7 +51,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.discovery.Discovery;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.discovery.local.LocalDiscovery;
 import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
 
@@ -70,7 +69,7 @@ import java.util.Set;
  * and cluster state {@link #status}, which is updated during cluster state publishing and applying
  * processing.  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 InternalClusterService}. After every update the
- * {@link DiscoveryService#publish} method publishes new version of the cluster state to all other nodes in the
+ * {@link Discovery#publish} method publishes 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. For example, for local discovery it is implemented by the {@link LocalDiscovery#publish}
  * method. In the Zen Discovery it is handled in the {@link PublishClusterStateAction#publish} method. The

+ 6 - 0
core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java

@@ -340,6 +340,12 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
             return this;
         }
 
+        public Builder removeGlobalBlock(int blockId) {
+            global.removeIf(block -> block.id() == blockId);
+            return this;
+        }
+
+
         public Builder addIndexBlock(String index, ClusterBlock block) {
             if (!indices.containsKey(index)) {
                 indices.put(index, new HashSet<>());

+ 28 - 8
core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java

@@ -44,6 +44,7 @@ import org.elasticsearch.cluster.routing.OperationRouting;
 import org.elasticsearch.cluster.routing.RoutingTable;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Priority;
+import org.elasticsearch.common.Randomness;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.inject.Inject;
@@ -64,7 +65,6 @@ import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor;
 import org.elasticsearch.common.util.concurrent.PrioritizedRunnable;
 import org.elasticsearch.common.util.iterable.Iterables;
 import org.elasticsearch.discovery.Discovery;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.tasks.TaskManager;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
@@ -76,7 +76,9 @@ 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.Random;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executor;
@@ -84,6 +86,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.BiConsumer;
 import java.util.stream.Collectors;
 
 import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadFactory;
@@ -97,9 +100,12 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
     public static final Setting<TimeValue> CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING = Setting.positiveTimeSetting("cluster.service.reconnect_interval",  TimeValue.timeValueSeconds(10), false, Setting.Scope.CLUSTER);
 
     public static final String UPDATE_THREAD_NAME = "clusterService#updateTask";
+    public static final Setting<Long> NODE_ID_SEED_SETTING =
+            // don't use node.id.seed so it won't be seen as an attribute
+            Setting.longSetting("node_id.seed", 0L, Long.MIN_VALUE, false, Setting.Scope.CLUSTER);
     private final ThreadPool threadPool;
 
-    private final DiscoveryService discoveryService;
+    private BiConsumer<ClusterChangedEvent, Discovery.AckListener> clusterStatePublisher;
 
     private final OperationRouting operationRouting;
 
@@ -139,12 +145,11 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
     private volatile ScheduledFuture reconnectToNodes;
 
     @Inject
-    public InternalClusterService(Settings settings, DiscoveryService discoveryService, OperationRouting operationRouting, TransportService transportService,
+    public InternalClusterService(Settings settings, OperationRouting operationRouting, TransportService transportService,
                                   ClusterSettings clusterSettings, ThreadPool threadPool, ClusterName clusterName, DiscoveryNodeService discoveryNodeService, Version version) {
         super(settings);
         this.operationRouting = operationRouting;
         this.transportService = transportService;
-        this.discoveryService = discoveryService;
         this.threadPool = threadPool;
         this.clusterSettings = clusterSettings;
         this.discoveryNodeService = discoveryNodeService;
@@ -161,7 +166,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
 
         localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
 
-        initialBlocks = ClusterBlocks.builder().addGlobalBlock(discoveryService.getNoMasterBlock());
+        initialBlocks = ClusterBlocks.builder();
 
         taskManager = transportService.getTaskManager();
     }
@@ -170,6 +175,10 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
         this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
     }
 
+    public void setClusterStatePublisher(BiConsumer<ClusterChangedEvent, Discovery.AckListener> publisher) {
+        clusterStatePublisher = publisher;
+    }
+
     @Override
     public void addInitialStateBlock(ClusterBlock block) throws IllegalStateException {
         if (lifecycle.started()) {
@@ -180,14 +189,20 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
 
     @Override
     public void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException {
+        removeInitialStateBlock(block.id());
+    }
+
+    @Override
+    public void removeInitialStateBlock(int blockId) throws IllegalStateException {
         if (lifecycle.started()) {
             throw new IllegalStateException("can't set initial block when started");
         }
-        initialBlocks.removeGlobalBlock(block);
+        initialBlocks.removeGlobalBlock(blockId);
     }
 
     @Override
     protected void doStart() {
+        Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting");
         add(localNodeMasterListeners);
         add(taskManager);
         this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
@@ -195,7 +210,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
         this.reconnectToNodes = threadPool.schedule(reconnectInterval, ThreadPool.Names.GENERIC, new ReconnectToNodes());
         Map<String, String> nodeAttributes = discoveryNodeService.buildAttributes();
         // note, we rely on the fact that its a new id each time we start, see FD and "kill -9" handling
-        final String nodeId = DiscoveryService.generateNodeId(settings);
+        final String nodeId = generateNodeId(settings);
         final TransportAddress publishAddress = transportService.boundAddress().publishAddress();
         DiscoveryNode localNode = new DiscoveryNode(settings.get("node.name"), nodeId, publishAddress, nodeAttributes, version);
         DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder().put(localNode).localNodeId(localNode.id());
@@ -572,7 +587,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
             if (newClusterState.nodes().localNodeMaster()) {
                 logger.debug("publishing cluster state version [{}]", newClusterState.version());
                 try {
-                    discoveryService.publish(clusterChangedEvent, ackListener);
+                    clusterStatePublisher.accept(clusterChangedEvent, ackListener);
                 } catch (Discovery.FailedToCommitClusterStateException t) {
                     logger.warn("failing [{}]: failed to commit cluster state version [{}]", t, source, newClusterState.version());
                     proccessedListeners.forEach(task -> task.listener.onFailure(task.source, t));
@@ -853,6 +868,11 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
         }
     }
 
+    public static String generateNodeId(Settings settings) {
+        Random random = Randomness.get(settings, NODE_ID_SEED_SETTING);
+        return Strings.randomBase64UUID(random);
+    }
+
     private boolean nodeRequiresConnection(DiscoveryNode node) {
         return localNode().shouldConnectTo(node);
     }

+ 1 - 1
core/src/main/java/org/elasticsearch/common/Randomness.java

@@ -41,7 +41,7 @@ import java.util.concurrent.ThreadLocalRandom;
  * setting a reproducible seed. When running the Elasticsearch server
  * process, non-reproducible sources of randomness are provided (unless
  * a setting is provided for a module that exposes a seed setting (e.g.,
- * DiscoveryService#DISCOVERY_SEED_SETTING)).
+ * DiscoveryService#NODE_ID_SEED_SETTING)).
  */
 public final class Randomness {
     private static final Method currentMethod;

+ 4 - 5
core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

@@ -48,7 +48,6 @@ import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
 import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.zen.ZenDiscovery;
 import org.elasticsearch.discovery.zen.elect.ElectMasterService;
@@ -62,11 +61,11 @@ import org.elasticsearch.http.HttpTransportSettings;
 import org.elasticsearch.http.netty.NettyHttpServerTransport;
 import org.elasticsearch.index.IndexSettings;
 import org.elasticsearch.index.store.IndexStoreConfig;
+import org.elasticsearch.indices.IndicesQueryCache;
+import org.elasticsearch.indices.IndicesRequestCache;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.indices.analysis.HunspellService;
 import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
-import org.elasticsearch.indices.IndicesQueryCache;
-import org.elasticsearch.indices.IndicesRequestCache;
 import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.indices.store.IndicesStore;
@@ -324,8 +323,8 @@ public final class ClusterSettings extends AbstractScopedSettings {
                     Environment.PATH_SCRIPTS_SETTING,
                     Environment.PATH_SHARED_DATA_SETTING,
                     Environment.PIDFILE_SETTING,
-                    DiscoveryService.DISCOVERY_SEED_SETTING,
-                    DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING,
+                    InternalClusterService.NODE_ID_SEED_SETTING,
+                    DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING,
                     DiscoveryModule.DISCOVERY_TYPE_SETTING,
                     DiscoveryModule.ZEN_MASTER_SERVICE_TYPE_SETTING,
                     FaultDetection.PING_RETRIES_SETTING,

+ 1 - 5
core/src/main/java/org/elasticsearch/discovery/Discovery.java

@@ -39,10 +39,6 @@ public interface Discovery extends LifecycleComponent<Discovery> {
 
     DiscoveryNode localNode();
 
-    void addListener(InitialStateDiscoveryListener listener);
-
-    void removeListener(InitialStateDiscoveryListener listener);
-
     String nodeDescription();
 
     /**
@@ -93,13 +89,13 @@ public interface Discovery extends LifecycleComponent<Discovery> {
      */
     DiscoveryStats stats();
 
+    DiscoverySettings getDiscoverySettings();
 
     /**
      * Triggers the first join cycle
      */
     void startInitialJoin();
 
-
     /***
      * @return the current value of minimum master nodes, or -1 for not set
      */

+ 14 - 5
core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java

@@ -34,6 +34,7 @@ import org.elasticsearch.discovery.zen.ping.unicast.UnicastHostsProvider;
 import org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -50,7 +51,7 @@ public class DiscoveryModule extends AbstractModule {
             "zen", Function.identity(), false, Setting.Scope.CLUSTER);
 
     private final Settings settings;
-    private final List<Class<? extends UnicastHostsProvider>> unicastHostProviders = new ArrayList<>();
+    private final Map<String, List<Class<? extends UnicastHostsProvider>>> unicastHostProviders = new HashMap<>();
     private final ExtensionPoint.ClassSet<ZenPing> zenPings = new ExtensionPoint.ClassSet<>("zen_ping", ZenPing.class);
     private final Map<String, Class<? extends Discovery>> discoveryTypes = new HashMap<>();
     private final Map<String, Class<? extends ElectMasterService>> masterServiceType = new HashMap<>();
@@ -66,9 +67,17 @@ public class DiscoveryModule extends AbstractModule {
 
     /**
      * Adds a custom unicast hosts provider to build a dynamic list of unicast hosts list when doing unicast discovery.
+     *
+     * @param type discovery for which this provider is relevant
+     * @param unicastHostProvider the host provider
      */
-    public void addUnicastHostProvider(Class<? extends UnicastHostsProvider> unicastHostProvider) {
-        unicastHostProviders.add(unicastHostProvider);
+    public void addUnicastHostProvider(String type, Class<? extends UnicastHostsProvider> unicastHostProvider) {
+        List<Class<? extends UnicastHostsProvider>> providerList = unicastHostProviders.get(type);
+        if (providerList == null) {
+            providerList = new ArrayList<>();
+            unicastHostProviders.put(type, providerList);
+        }
+        providerList.add(unicastHostProvider);
     }
 
     /**
@@ -116,12 +125,12 @@ public class DiscoveryModule extends AbstractModule {
             }
             bind(ZenPingService.class).asEagerSingleton();
             Multibinder<UnicastHostsProvider> unicastHostsProviderMultibinder = Multibinder.newSetBinder(binder(), UnicastHostsProvider.class);
-            for (Class<? extends UnicastHostsProvider> unicastHostProvider : unicastHostProviders) {
+            for (Class<? extends UnicastHostsProvider> unicastHostProvider :
+                    unicastHostProviders.getOrDefault(discoveryType, Collections.emptyList())) {
                 unicastHostsProviderMultibinder.addBinding().to(unicastHostProvider);
             }
             zenPings.bind(binder());
         }
         bind(Discovery.class).to(discoveryClass).asEagerSingleton();
-        bind(DiscoveryService.class).asEagerSingleton();
     }
 }

+ 0 - 140
core/src/main/java/org/elasticsearch/discovery/DiscoveryService.java

@@ -1,140 +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.ElasticsearchTimeoutException;
-import org.elasticsearch.cluster.ClusterChangedEvent;
-import org.elasticsearch.cluster.block.ClusterBlock;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.Randomness;
-import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Setting;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
-
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- *
- */
-public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryService> {
-
-    public static final Setting<TimeValue> INITIAL_STATE_TIMEOUT_SETTING = Setting.positiveTimeSetting("discovery.initial_state_timeout", TimeValue.timeValueSeconds(30), false, Setting.Scope.CLUSTER);
-    public static final Setting<Long> DISCOVERY_SEED_SETTING = Setting.longSetting("discovery.id.seed", 0L, Long.MIN_VALUE, false, Setting.Scope.CLUSTER);
-
-    private static class InitialStateListener implements InitialStateDiscoveryListener {
-
-        private final CountDownLatch latch = new CountDownLatch(1);
-        private volatile boolean initialStateReceived;
-
-        @Override
-        public void initialStateProcessed() {
-            initialStateReceived = true;
-            latch.countDown();
-        }
-
-        public boolean waitForInitialState(TimeValue timeValue) throws InterruptedException {
-            if (timeValue.millis() > 0) {
-                latch.await(timeValue.millis(), TimeUnit.MILLISECONDS);
-            }
-            return initialStateReceived;
-        }
-    }
-
-    private final TimeValue initialStateTimeout;
-    private final Discovery discovery;
-    private InitialStateListener initialStateListener;
-    private final DiscoverySettings discoverySettings;
-
-    @Inject
-    public DiscoveryService(Settings settings, DiscoverySettings discoverySettings, Discovery discovery) {
-        super(settings);
-        this.discoverySettings = discoverySettings;
-        this.discovery = discovery;
-        this.initialStateTimeout = INITIAL_STATE_TIMEOUT_SETTING.get(settings);
-    }
-
-    public ClusterBlock getNoMasterBlock() {
-        return discoverySettings.getNoMasterBlock();
-    }
-
-    @Override
-    protected void doStart() {
-        initialStateListener = new InitialStateListener();
-        discovery.addListener(initialStateListener);
-        discovery.start();
-        logger.info(discovery.nodeDescription());
-    }
-
-    public void joinClusterAndWaitForInitialState() {
-        try {
-            discovery.startInitialJoin();
-            if (!initialStateListener.waitForInitialState(initialStateTimeout)) {
-                logger.warn("waited for {} and no initial state was set by the discovery", initialStateTimeout);
-            }
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new ElasticsearchTimeoutException("Interrupted while waiting for initial discovery state");
-        }
-    }
-
-    @Override
-    protected void doStop() {
-        if (initialStateListener != null) {
-            discovery.removeListener(initialStateListener);
-        }
-        discovery.stop();
-    }
-
-    @Override
-    protected void doClose() {
-        discovery.close();
-    }
-
-    public DiscoveryNode localNode() {
-        return discovery.localNode();
-    }
-
-    public String nodeDescription() {
-        return discovery.nodeDescription();
-    }
-
-    /**
-     * Publish all the changes to the cluster from the master (can be called just by the master). The publish
-     * process should not publish this state to the master as well! (the master is sending it...).
-     * <p>
-     * The {@link org.elasticsearch.discovery.Discovery.AckListener} allows to acknowledge the publish
-     * event based on the response gotten from all nodes
-     */
-    public void publish(ClusterChangedEvent clusterChangedEvent, Discovery.AckListener ackListener) {
-        if (lifecycle.started()) {
-            discovery.publish(clusterChangedEvent, ackListener);
-        }
-    }
-
-    public static String generateNodeId(Settings settings) {
-        Random random = Randomness.get(settings, DiscoveryService.DISCOVERY_SEED_SETTING);
-        return Strings.randomBase64UUID(random);
-    }
-}

+ 1 - 2
core/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java

@@ -22,7 +22,6 @@ package org.elasticsearch.discovery;
 import org.elasticsearch.cluster.block.ClusterBlock;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.common.component.AbstractComponent;
-import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
@@ -52,6 +51,7 @@ public class DiscoverySettings extends AbstractComponent {
     public static final Setting<TimeValue> COMMIT_TIMEOUT_SETTING = new Setting<>("discovery.zen.commit_timeout", (s) -> PUBLISH_TIMEOUT_SETTING.getRaw(s), (s) -> TimeValue.parseTimeValue(s, TimeValue.timeValueSeconds(30), "discovery.zen.commit_timeout"), true, Setting.Scope.CLUSTER);
     public static final Setting<ClusterBlock> NO_MASTER_BLOCK_SETTING = new Setting<>("discovery.zen.no_master_block", "write", DiscoverySettings::parseNoMasterBlock, true, Setting.Scope.CLUSTER);
     public static final Setting<Boolean> PUBLISH_DIFF_ENABLE_SETTING = Setting.boolSetting("discovery.zen.publish_diff.enable", true, true, Setting.Scope.CLUSTER);
+    public static final Setting<TimeValue> INITIAL_STATE_TIMEOUT_SETTING = Setting.positiveTimeSetting("discovery.initial_state_timeout", TimeValue.timeValueSeconds(30), false, Setting.Scope.CLUSTER);
 
     private volatile ClusterBlock noMasterBlock;
     private volatile TimeValue publishTimeout;
@@ -59,7 +59,6 @@ public class DiscoverySettings extends AbstractComponent {
     private volatile TimeValue commitTimeout;
     private volatile boolean publishDiff;
 
-    @Inject
     public DiscoverySettings(Settings settings, ClusterSettings clusterSettings) {
         super(settings);
         clusterSettings.addSettingsUpdateConsumer(NO_MASTER_BLOCK_SETTING, this::setNoMasterBlock);

+ 8 - 32
core/src/main/java/org/elasticsearch/discovery/local/LocalDiscovery.java

@@ -36,6 +36,7 @@ import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.inject.internal.Nullable;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@@ -44,14 +45,12 @@ import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.DiscoveryStats;
-import org.elasticsearch.discovery.InitialStateDiscoveryListener;
 import org.elasticsearch.node.service.NodeService;
 
 import java.util.HashSet;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.elasticsearch.cluster.ClusterState.Builder;
@@ -73,19 +72,16 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
 
     private final AtomicBoolean initialStateSent = new AtomicBoolean();
 
-    private final CopyOnWriteArrayList<InitialStateDiscoveryListener> initialStateListeners = new CopyOnWriteArrayList<>();
-
     private static final ConcurrentMap<ClusterName, ClusterGroup> clusterGroups = ConcurrentCollections.newConcurrentMap();
 
     private volatile ClusterState lastProcessedClusterState;
 
     @Inject
-    public LocalDiscovery(Settings settings, ClusterName clusterName, ClusterService clusterService,
-                          DiscoverySettings discoverySettings) {
+    public LocalDiscovery(Settings settings, ClusterName clusterName, ClusterService clusterService, ClusterSettings clusterSettings) {
         super(settings);
         this.clusterName = clusterName;
         this.clusterService = clusterService;
-        this.discoverySettings = discoverySettings;
+        this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
     }
 
     @Override
@@ -150,11 +146,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
                     public void onFailure(String source, Throwable t) {
                         logger.error("unexpected failure during [{}]", t, source);
                     }
-
-                    @Override
-                    public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                        sendInitialStateEventIfNeeded();
-                    }
                 });
             } else if (firstMaster != null) {
                 // tell the master to send the fact that we are here
@@ -182,7 +173,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
 
                     @Override
                     public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                        sendInitialStateEventIfNeeded();
                         // we reroute not in the same cluster state update since in certain areas we rely on
                         // the node to be in the cluster state (sampled from ClusterService#state) to be there, also
                         // shard transitions need to better be handled in such cases
@@ -266,16 +256,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
         return clusterService.localNode();
     }
 
-    @Override
-    public void addListener(InitialStateDiscoveryListener listener) {
-        this.initialStateListeners.add(listener);
-    }
-
-    @Override
-    public void removeListener(InitialStateDiscoveryListener listener) {
-        this.initialStateListeners.remove(listener);
-    }
-
     @Override
     public String nodeDescription() {
         return clusterName.value() + "/" + localNode().id();
@@ -304,6 +284,11 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
         return new DiscoveryStats(null);
     }
 
+    @Override
+    public DiscoverySettings getDiscoverySettings() {
+        return discoverySettings;
+    }
+
     @Override
     public int getMinimumMasterNodes() {
         return -1;
@@ -403,7 +388,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
 
                         @Override
                         public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                            sendInitialStateEventIfNeeded();
                             publishResponseHandler.onResponse(discovery.localNode());
                         }
                     });
@@ -436,14 +420,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
         }
     }
 
-    private void sendInitialStateEventIfNeeded() {
-        if (initialStateSent.compareAndSet(false, true)) {
-            for (InitialStateDiscoveryListener listener : initialStateListeners) {
-                listener.initialStateProcessed();
-            }
-        }
-    }
-
     private class ClusterGroup {
 
         private Queue<LocalDiscovery> members = ConcurrentCollections.newQueue();

+ 7 - 34
core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java

@@ -51,7 +51,6 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.DiscoveryStats;
-import org.elasticsearch.discovery.InitialStateDiscoveryListener;
 import org.elasticsearch.discovery.zen.elect.ElectMasterService;
 import org.elasticsearch.discovery.zen.fd.MasterFaultDetection;
 import org.elasticsearch.discovery.zen.fd.NodesFaultDetection;
@@ -76,7 +75,6 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -134,13 +132,8 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
     private final boolean masterElectionFilterDataNodes;
     private final TimeValue masterElectionWaitForJoinsTimeout;
 
-
-    private final CopyOnWriteArrayList<InitialStateDiscoveryListener> initialStateListeners = new CopyOnWriteArrayList<>();
-
     private final JoinThreadControl joinThreadControl;
 
-    private final AtomicBoolean initialStateSent = new AtomicBoolean();
-
     /** counts the time this node has joined the cluster or have elected it self as master */
     private final AtomicLong clusterJoinsCounter = new AtomicLong();
 
@@ -154,13 +147,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
     @Inject
     public ZenDiscovery(Settings settings, ClusterName clusterName, ThreadPool threadPool,
                         TransportService transportService, final ClusterService clusterService, ClusterSettings clusterSettings,
-                        ZenPingService pingService, ElectMasterService electMasterService,
-                        DiscoverySettings discoverySettings) {
+                        ZenPingService pingService, ElectMasterService electMasterService) {
         super(settings);
         this.clusterName = clusterName;
         this.clusterService = clusterService;
         this.transportService = transportService;
-        this.discoverySettings = discoverySettings;
+        this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
         this.pingService = pingService;
         this.electMaster = electMasterService;
         this.pingTimeout = PING_TIMEOUT_SETTING.get(settings);
@@ -248,7 +240,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
         pingService.stop();
         masterFD.stop("zen disco stop");
         nodesFD.stop();
-        initialStateSent.set(false);
         DiscoveryNodes nodes = nodes();
         if (sendLeaveRequest) {
             if (nodes.masterNode() == null) {
@@ -290,16 +281,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
         return clusterService.localNode();
     }
 
-    @Override
-    public void addListener(InitialStateDiscoveryListener listener) {
-        this.initialStateListeners.add(listener);
-    }
-
-    @Override
-    public void removeListener(InitialStateDiscoveryListener listener) {
-        this.initialStateListeners.remove(listener);
-    }
-
     @Override
     public String nodeDescription() {
         return clusterName.value() + "/" + clusterService.localNode().id();
@@ -357,6 +338,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
         return new DiscoveryStats(queueStats);
     }
 
+    @Override
+    public DiscoverySettings getDiscoverySettings() {
+        return discoverySettings;
+    }
+
     @Override
     public int getMinimumMasterNodes() {
         return electMaster.minimumMasterNodes();
@@ -403,7 +389,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
                             joinThreadControl.markThreadAsDone(currentThread);
                             // we only starts nodesFD if we are master (it may be that we received a cluster state while pinging)
                             nodesFD.updateNodesAndPing(state); // start the nodes FD
-                            sendInitialStateEventIfNeeded();
                             long count = clusterJoinsCounter.incrementAndGet();
                             logger.trace("cluster joins counter set to [{}] (elected as master)", count);
                         }
@@ -591,7 +576,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
 
             @Override
             public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                sendInitialStateEventIfNeeded();
             }
         });
     }
@@ -630,7 +614,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
 
             @Override
             public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                sendInitialStateEventIfNeeded();
             }
         });
     }
@@ -679,7 +662,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
 
             @Override
             public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                sendInitialStateEventIfNeeded();
             }
 
         });
@@ -773,7 +755,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
             @Override
             public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
                 try {
-                    sendInitialStateEventIfNeeded();
                     if (newClusterState != null) {
                         publishClusterState.pendingStatesQueue().markAsProcessed(newClusterState);
                     }
@@ -1005,14 +986,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
         }
     }
 
-    private void sendInitialStateEventIfNeeded() {
-        if (initialStateSent.compareAndSet(false, true)) {
-            for (InitialStateDiscoveryListener listener : initialStateListeners) {
-                listener.initialStateProcessed();
-            }
-        }
-    }
-
     private class NewPendingClusterStateListener implements PublishClusterStateAction.NewPendingClusterStateListener {
 
         @Override

+ 2 - 7
core/src/main/java/org/elasticsearch/gateway/GatewayService.java

@@ -40,7 +40,6 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.discovery.Discovery;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -79,8 +78,6 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
 
     private final ClusterService clusterService;
 
-    private final DiscoveryService discoveryService;
-
     private final TimeValue recoverAfterTime;
     private final int recoverAfterNodes;
     private final int expectedNodes;
@@ -95,14 +92,12 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
 
     @Inject
     public GatewayService(Settings settings, AllocationService allocationService, ClusterService clusterService,
-                          DiscoveryService discoveryService, ThreadPool threadPool,
-                          NodeEnvironment nodeEnvironment, GatewayMetaState metaState,
+                          ThreadPool threadPool, NodeEnvironment nodeEnvironment, GatewayMetaState metaState,
                           TransportNodesListGatewayMetaState listGatewayMetaState, Discovery discovery) {
         super(settings);
         this.gateway = new Gateway(settings, clusterService, nodeEnvironment, metaState, listGatewayMetaState, discovery);
         this.allocationService = allocationService;
         this.clusterService = clusterService;
-        this.discoveryService = discoveryService;
         this.threadPool = threadPool;
         // allow to control a delay of when indices will get created
         this.expectedNodes = EXPECTED_NODES_SETTING.get(this.settings);
@@ -162,7 +157,7 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
         }
 
         DiscoveryNodes nodes = state.nodes();
-        if (state.blocks().hasGlobalBlock(discoveryService.getNoMasterBlock())) {
+        if (state.nodes().masterNodeId() == null) {
             logger.debug("not recovering from gateway, no master elected yet");
         } else if (recoverAfterNodes != -1 && (nodes.masterAndDataNodes().size()) < recoverAfterNodes) {
             logger.debug("not recovering from gateway, nodes_size (data+master) [" + nodes.masterAndDataNodes().size() + "] < recover_after_nodes [" + recoverAfterNodes + "]");

+ 81 - 30
core/src/main/java/org/elasticsearch/node/Node.java

@@ -22,6 +22,7 @@ package org.elasticsearch.node;
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.Build;
 import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ElasticsearchTimeoutException;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionModule;
 import org.elasticsearch.cache.recycler.PageCacheRecycler;
@@ -30,9 +31,13 @@ import org.elasticsearch.client.node.NodeClientModule;
 import org.elasticsearch.cluster.ClusterModule;
 import org.elasticsearch.cluster.ClusterNameModule;
 import org.elasticsearch.cluster.ClusterService;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ClusterStateObserver;
+import org.elasticsearch.cluster.MasterNodeChangePredicate;
 import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.RoutingService;
+import org.elasticsearch.cluster.service.InternalClusterService;
 import org.elasticsearch.common.StopWatch;
 import org.elasticsearch.common.component.Lifecycle;
 import org.elasticsearch.common.component.LifecycleComponent;
@@ -52,9 +57,10 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsModule;
 import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.DiscoveryService;
+import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.EnvironmentModule;
 import org.elasticsearch.env.NodeEnvironment;
@@ -111,6 +117,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 
@@ -154,7 +161,7 @@ public class Node implements Closeable {
 
     protected Node(Environment tmpEnv, Version version, Collection<Class<? extends Plugin>> classpathPlugins) {
         Settings tmpSettings = settingsBuilder().put(tmpEnv.settings())
-            .put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build();
+                .put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build();
         tmpSettings = TribeService.processSettings(tmpSettings);
 
         ESLogger logger = Loggers.getLogger(Node.class, NODE_NAME_SETTING.get(tmpSettings));
@@ -164,7 +171,7 @@ public class Node implements Closeable {
 
         if (logger.isDebugEnabled()) {
             logger.debug("using config [{}], data [{}], logs [{}], plugins [{}]",
-                tmpEnv.configFile(), Arrays.toString(tmpEnv.dataFiles()), tmpEnv.logsFile(), tmpEnv.pluginsFile());
+                    tmpEnv.configFile(), Arrays.toString(tmpEnv.dataFiles()), tmpEnv.logsFile(), tmpEnv.pluginsFile());
         }
 
         this.pluginsService = new PluginsService(tmpSettings, tmpEnv.modulesFile(), tmpEnv.pluginsFile(), classpathPlugins);
@@ -282,28 +289,73 @@ public class Node implements Closeable {
         injector.getInstance(MonitorService.class).start();
         injector.getInstance(RestController.class).start();
 
+        assert injector.getInstance(ClusterService.class) instanceof InternalClusterService :
+                "node cluster service implementation must inherit from InternalClusterService";
+        final InternalClusterService clusterService = (InternalClusterService) injector.getInstance(ClusterService.class);
+
         // TODO hack around circular dependencies problems
-        injector.getInstance(GatewayAllocator.class).setReallocation(injector.getInstance(ClusterService.class), injector.getInstance(RoutingService.class));
+        injector.getInstance(GatewayAllocator.class).setReallocation(clusterService, injector.getInstance(RoutingService.class));
 
         injector.getInstance(ResourceWatcherService.class).start();
         injector.getInstance(GatewayService.class).start();
+        Discovery discovery = injector.getInstance(Discovery.class);
+        clusterService.addInitialStateBlock(discovery.getDiscoverySettings().getNoMasterBlock());
+        clusterService.setClusterStatePublisher(discovery::publish);
+
+        // start before the cluster service since it adds/removes initial Cluster state blocks
+        final TribeService tribeService = injector.getInstance(TribeService.class);
+        tribeService.start();
+
 
         // Start the transport service now so the publish address will be added to the local disco node in ClusterService
         TransportService transportService = injector.getInstance(TransportService.class);
         transportService.start();
-        injector.getInstance(ClusterService.class).start();
+        clusterService.start();
 
         // start after cluster service so the local disco is known
-        DiscoveryService discoService = injector.getInstance(DiscoveryService.class).start();
-
-
+        discovery.start();
         transportService.acceptIncomingRequests();
-        discoService.joinClusterAndWaitForInitialState();
+        discovery.startInitialJoin();
+
+        // tribe nodes don't have a master so we shouldn't register an observer
+        if (DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings).millis() > 0) {
+            final ThreadPool thread = injector.getInstance(ThreadPool.class);
+            ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, thread.getThreadContext());
+            final CountDownLatch latch = new CountDownLatch(1);
+            if (observer.observedState().nodes().masterNodeId() == null) {
+                observer.waitForNextChange(new ClusterStateObserver.Listener() {
+                    @Override
+                    public void onNewClusterState(ClusterState state) {
+                        latch.countDown();
+                    }
+
+                    @Override
+                    public void onClusterServiceClose() {
+                        latch.countDown();
+                    }
+
+                    @Override
+                    public void onTimeout(TimeValue timeout) {
+                        assert false;
+                    }
+                    // use null timeout as we use timeout on the latchwait
+                }, MasterNodeChangePredicate.INSTANCE, null);
+            }
+
+            try {
+                latch.await(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings).millis(), TimeUnit.MILLISECONDS);
+            } catch (InterruptedException e) {
+                throw new ElasticsearchTimeoutException("Interrupted while waiting for initial discovery state");
+            }
+        }
 
         if (settings.getAsBoolean("http.enabled", true)) {
             injector.getInstance(HttpServer.class).start();
         }
-        injector.getInstance(TribeService.class).start();
+
+        // start nodes now, after the http server, because it may take some time
+        tribeService.startNodes();
+
 
         if (WRITE_PORTS_FIELD_SETTING.get(settings)) {
             if (settings.getAsBoolean("http.enabled", true)) {
@@ -340,7 +392,6 @@ public class Node implements Closeable {
         injector.getInstance(IndicesTTLService.class).stop();
         injector.getInstance(RoutingService.class).stop();
         injector.getInstance(ClusterService.class).stop();
-        injector.getInstance(DiscoveryService.class).stop();
         injector.getInstance(MonitorService.class).stop();
         injector.getInstance(GatewayService.class).stop();
         injector.getInstance(SearchService.class).stop();
@@ -378,50 +429,50 @@ public class Node implements Closeable {
         toClose.add(injector.getInstance(TribeService.class));
         toClose.add(() -> stopWatch.stop().start("node_service"));
         toClose.add(injector.getInstance(NodeService.class));
-        toClose.add(() ->stopWatch.stop().start("http"));
+        toClose.add(() -> stopWatch.stop().start("http"));
         if (settings.getAsBoolean("http.enabled", true)) {
             toClose.add(injector.getInstance(HttpServer.class));
         }
-        toClose.add(() ->stopWatch.stop().start("snapshot_service"));
+        toClose.add(() -> stopWatch.stop().start("snapshot_service"));
         toClose.add(injector.getInstance(SnapshotsService.class));
         toClose.add(injector.getInstance(SnapshotShardsService.class));
-        toClose.add(() ->stopWatch.stop().start("client"));
+        toClose.add(() -> stopWatch.stop().start("client"));
         Releasables.close(injector.getInstance(Client.class));
-        toClose.add(() ->stopWatch.stop().start("indices_cluster"));
+        toClose.add(() -> stopWatch.stop().start("indices_cluster"));
         toClose.add(injector.getInstance(IndicesClusterStateService.class));
-        toClose.add(() ->stopWatch.stop().start("indices"));
+        toClose.add(() -> stopWatch.stop().start("indices"));
         toClose.add(injector.getInstance(IndicesTTLService.class));
         toClose.add(injector.getInstance(IndicesService.class));
         // close filter/fielddata caches after indices
         toClose.add(injector.getInstance(IndicesStore.class));
-        toClose.add(() ->stopWatch.stop().start("routing"));
+        toClose.add(() -> stopWatch.stop().start("routing"));
         toClose.add(injector.getInstance(RoutingService.class));
-        toClose.add(() ->stopWatch.stop().start("cluster"));
+        toClose.add(() -> stopWatch.stop().start("cluster"));
         toClose.add(injector.getInstance(ClusterService.class));
-        toClose.add(() ->stopWatch.stop().start("discovery"));
-        toClose.add(injector.getInstance(DiscoveryService.class));
-        toClose.add(() ->stopWatch.stop().start("monitor"));
+        toClose.add(() -> stopWatch.stop().start("discovery"));
+        toClose.add(injector.getInstance(Discovery.class));
+        toClose.add(() -> stopWatch.stop().start("monitor"));
         toClose.add(injector.getInstance(MonitorService.class));
-        toClose.add(() ->stopWatch.stop().start("gateway"));
+        toClose.add(() -> stopWatch.stop().start("gateway"));
         toClose.add(injector.getInstance(GatewayService.class));
-        toClose.add(() ->stopWatch.stop().start("search"));
+        toClose.add(() -> stopWatch.stop().start("search"));
         toClose.add(injector.getInstance(SearchService.class));
-        toClose.add(() ->stopWatch.stop().start("rest"));
+        toClose.add(() -> stopWatch.stop().start("rest"));
         toClose.add(injector.getInstance(RestController.class));
-        toClose.add(() ->stopWatch.stop().start("transport"));
+        toClose.add(() -> stopWatch.stop().start("transport"));
         toClose.add(injector.getInstance(TransportService.class));
-        toClose.add(() ->stopWatch.stop().start("percolator_service"));
+        toClose.add(() -> stopWatch.stop().start("percolator_service"));
         toClose.add(injector.getInstance(PercolatorService.class));
 
         for (Class<? extends LifecycleComponent> plugin : pluginsService.nodeServices()) {
-            toClose.add(() ->stopWatch.stop().start("plugin(" + plugin.getName() + ")"));
+            toClose.add(() -> stopWatch.stop().start("plugin(" + plugin.getName() + ")"));
             toClose.add(injector.getInstance(plugin));
         }
 
-        toClose.add(() ->stopWatch.stop().start("script"));
+        toClose.add(() -> stopWatch.stop().start("script"));
         toClose.add(injector.getInstance(ScriptService.class));
 
-        toClose.add(() ->stopWatch.stop().start("thread_pool"));
+        toClose.add(() -> stopWatch.stop().start("thread_pool"));
         // TODO this should really use ThreadPool.terminate()
         toClose.add(() -> injector.getInstance(ThreadPool.class).shutdown());
         toClose.add(() -> {
@@ -432,7 +483,7 @@ public class Node implements Closeable {
             }
         });
 
-        toClose.add(() ->stopWatch.stop().start("thread_pool_force_shutdown"));
+        toClose.add(() -> stopWatch.stop().start("thread_pool_force_shutdown"));
         toClose.add(() -> injector.getInstance(ThreadPool.class).shutdownNow());
         toClose.add(() -> stopWatch.stop());
 

+ 21 - 25
core/src/main/java/org/elasticsearch/tribe/TribeService.java

@@ -20,7 +20,6 @@
 package org.elasticsearch.tribe;
 
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
-
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
 import org.elasticsearch.cluster.ClusterChangedEvent;
@@ -48,7 +47,7 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.DiscoveryService;
+import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.gateway.GatewayService;
 import org.elasticsearch.node.Node;
@@ -62,7 +61,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.function.Function;
-import java.util.function.Predicate;
 
 import static java.util.Collections.unmodifiableMap;
 
@@ -112,7 +110,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
         sb.put(Node.NODE_CLIENT_SETTING.getKey(), true); // this node should just act as a node client
         sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "local"); // a tribe node should not use zen discovery
         // nothing is going to be discovered, since no master will be elected
-        sb.put(DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
+        sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
         if (sb.get("cluster.name") == null) {
             sb.put("cluster.name", "tribe_" + Strings.randomBase64UUID()); // make sure it won't join other tribe nodes in the same JVM
         }
@@ -138,7 +136,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
                     return s;
                 }
                 throw new IllegalArgumentException(
-                    "Invalid value for [tribe.on_conflict] must be either [any, drop or start with prefer_] but was: [" + s + "]");
+                        "Invalid value for [tribe.on_conflict] must be either [any, drop or start with prefer_] but was: [" + s + "]");
         }
     }, false, Setting.Scope.CLUSTER);
 
@@ -154,7 +152,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
             Collections.emptyList(), Function.identity(), false, Setting.Scope.CLUSTER);
 
     public static final Set<String> TRIBE_SETTING_KEYS = Sets.newHashSet(TRIBE_NAME_SETTING.getKey(), ON_CONFLICT_SETTING.getKey(),
-        BLOCKS_METADATA_INDICES_SETTING.getKey(), BLOCKS_METADATA_SETTING.getKey(), BLOCKS_READ_INDICES_SETTING.getKey(), BLOCKS_WRITE_INDICES_SETTING.getKey(), BLOCKS_WRITE_SETTING.getKey());
+            BLOCKS_METADATA_INDICES_SETTING.getKey(), BLOCKS_METADATA_SETTING.getKey(), BLOCKS_READ_INDICES_SETTING.getKey(), BLOCKS_WRITE_INDICES_SETTING.getKey(), BLOCKS_WRITE_SETTING.getKey());
 
     private final String onConflict;
     private final Set<String> droppedIndices = ConcurrentCollections.newConcurrentSet();
@@ -162,7 +160,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
     private final List<Node> nodes = new CopyOnWriteArrayList<>();
 
     @Inject
-    public TribeService(Settings settings, ClusterService clusterService, DiscoveryService discoveryService) {
+    public TribeService(Settings settings, ClusterService clusterService) {
         super(settings);
         this.clusterService = clusterService;
         Map<String, Settings> nodesSettings = new HashMap<>(settings.getGroups("tribe", true));
@@ -183,38 +181,36 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
             nodes.add(new TribeClientNode(sb.build()));
         }
 
-        String[] blockIndicesWrite = Strings.EMPTY_ARRAY;
-        String[] blockIndicesRead = Strings.EMPTY_ARRAY;
-        String[] blockIndicesMetadata = Strings.EMPTY_ARRAY;
+        this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
+        this.blockIndicesRead = BLOCKS_READ_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
+        this.blockIndicesWrite = BLOCKS_WRITE_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
+
         if (!nodes.isEmpty()) {
-            // remove the initial election / recovery blocks since we are not going to have a
-            // master elected in this single tribe  node local "cluster"
-            clusterService.removeInitialStateBlock(discoveryService.getNoMasterBlock());
-            clusterService.removeInitialStateBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK);
             if (BLOCKS_WRITE_SETTING.get(settings)) {
                 clusterService.addInitialStateBlock(TRIBE_WRITE_BLOCK);
             }
-            blockIndicesWrite = BLOCKS_WRITE_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
             if (BLOCKS_METADATA_SETTING.get(settings)) {
                 clusterService.addInitialStateBlock(TRIBE_METADATA_BLOCK);
             }
-            blockIndicesMetadata =  BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
-            blockIndicesRead =  BLOCKS_READ_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
-            for (Node node : nodes) {
-                node.injector().getInstance(ClusterService.class).add(new TribeClusterStateListener(node));
-            }
         }
-        this.blockIndicesMetadata = blockIndicesMetadata;
-        this.blockIndicesRead = blockIndicesRead;
-        this.blockIndicesWrite = blockIndicesWrite;
 
         this.onConflict = ON_CONFLICT_SETTING.get(settings);
     }
 
     @Override
     protected void doStart() {
+        if (nodes.isEmpty() == false) {
+            // remove the initial election / recovery blocks since we are not going to have a
+            // master elected in this single tribe  node local "cluster"
+            clusterService.removeInitialStateBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
+            clusterService.removeInitialStateBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK);
+        }
+    }
+
+    public void startNodes() {
         for (Node node : nodes) {
             try {
+                node.injector().getInstance(ClusterService.class).add(new TribeClusterStateListener(node));
                 node.start();
             } catch (Throwable e) {
                 // calling close is safe for non started nodes, we can just iterate over all
@@ -410,14 +406,14 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
         }
 
         private void removeIndex(ClusterBlocks.Builder blocks, MetaData.Builder metaData, RoutingTable.Builder routingTable,
-                IndexMetaData index) {
+                                 IndexMetaData index) {
             metaData.remove(index.getIndex().getName());
             routingTable.remove(index.getIndex().getName());
             blocks.removeIndexBlocks(index.getIndex().getName());
         }
 
         private void addNewIndex(ClusterState tribeState, ClusterBlocks.Builder blocks, MetaData.Builder metaData,
-                RoutingTable.Builder routingTable, IndexMetaData tribeIndex) {
+                                 RoutingTable.Builder routingTable, IndexMetaData tribeIndex) {
             Settings tribeSettings = Settings.builder().put(tribeIndex.getSettings()).put(TRIBE_NAME_SETTING.getKey(), tribeName).build();
             metaData.put(IndexMetaData.builder(tribeIndex).settings(tribeSettings));
             routingTable.add(tribeState.routingTable().index(tribeIndex.getIndex()));

+ 2 - 3
core/src/test/java/org/elasticsearch/cluster/action/shard/ShardFailedClusterStateTaskExecutorTests.java

@@ -40,8 +40,8 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
 import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
+import org.elasticsearch.cluster.service.InternalClusterService;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.test.ESAllocationTestCase;
 import org.junit.Before;
@@ -50,7 +50,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -306,7 +305,7 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
             return randomSubsetOf(1, shards.toArray(new ShardRouting[0])).get(0);
         } else {
             return
-                TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), DiscoveryService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values()));
+                    TestShardRouting.newShardRouting(shardRouting.index(), shardRouting.id(), InternalClusterService.generateNodeId(Settings.EMPTY), randomBoolean(), randomFrom(ShardRoutingState.values()));
         }
     }
 

+ 6 - 3
core/src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsIT.java

@@ -26,6 +26,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDeci
 import org.elasticsearch.common.logging.ESLoggerFactory;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.store.IndexStoreConfig;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -57,7 +58,7 @@ public class ClusterSettingsIT extends ESIntegTestCase {
     }
 
     public void testDeleteIsAppliedFirst() {
-        DiscoverySettings discoverySettings = internalCluster().getInstance(DiscoverySettings.class);
+        DiscoverySettings discoverySettings = getDiscoverySettings();
 
         assertEquals(discoverySettings.getPublishTimeout(), DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY));
         assertTrue(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY));
@@ -83,7 +84,7 @@ public class ClusterSettingsIT extends ESIntegTestCase {
     }
 
     public void testResetClusterSetting() {
-        DiscoverySettings discoverySettings = internalCluster().getInstance(DiscoverySettings.class);
+        DiscoverySettings discoverySettings = getDiscoverySettings();
 
         assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
         assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
@@ -243,7 +244,7 @@ public class ClusterSettingsIT extends ESIntegTestCase {
 
     public void testUpdateDiscoveryPublishTimeout() {
 
-        DiscoverySettings discoverySettings = internalCluster().getInstance(DiscoverySettings.class);
+        DiscoverySettings discoverySettings = getDiscoverySettings();
 
         assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
 
@@ -281,6 +282,8 @@ public class ClusterSettingsIT extends ESIntegTestCase {
         assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1L));
     }
 
+    private DiscoverySettings getDiscoverySettings() {return internalCluster().getInstance(Discovery.class).getDiscoverySettings();}
+
     public void testClusterUpdateSettingsWithBlocks() {
         String key1 = "cluster.routing.allocation.enable";
         Settings transientSettings = Settings.builder().put(key1, EnableAllocationDecider.Allocation.NONE.name()).build();

+ 1 - 1
core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java

@@ -958,7 +958,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
         // don't wait for initial state, wat want to add the disruption while the cluster is forming..
         internalCluster().startNodesAsync(3,
                 Settings.builder()
-                        .put(DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "1ms")
+                        .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "1ms")
                         .put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "3s")
                         .build()).get();
 

+ 1 - 1
core/src/test/java/org/elasticsearch/gateway/GatewayServiceTests.java

@@ -34,7 +34,7 @@ public class GatewayServiceTests extends ESTestCase {
                 .put("http.enabled", "false")
                 .put("discovery.type", "local")
                 .put(settings.build()).build(),
-                null, new NoopClusterService(), null, null, null, null, null, new NoopDiscovery());
+                null, new NoopClusterService(), null, null, null, null, new NoopDiscovery());
 
     }
 

+ 2 - 2
core/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java

@@ -30,13 +30,13 @@ import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
 import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.index.recovery.RecoveryStats;
 import org.elasticsearch.index.store.Store;
 import org.elasticsearch.indices.IndicesService;
@@ -558,7 +558,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
         ensureSearchable(indexName);
 
         ClusterStateResponse stateResponse = client().admin().cluster().prepareState().get();
-        final String blueNodeId = internalCluster().getInstance(DiscoveryService.class, blueNodeName).localNode().id();
+        final String blueNodeId = internalCluster().getInstance(ClusterService.class, blueNodeName).localNode().id();
 
         assertFalse(stateResponse.getState().getRoutingNodes().node(blueNodeId).isEmpty());
 

+ 6 - 11
core/src/test/java/org/elasticsearch/test/NoopDiscovery.java

@@ -25,8 +25,8 @@ import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.component.Lifecycle;
 import org.elasticsearch.common.component.LifecycleListener;
 import org.elasticsearch.discovery.Discovery;
+import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.DiscoveryStats;
-import org.elasticsearch.discovery.InitialStateDiscoveryListener;
 import org.elasticsearch.node.service.NodeService;
 
 public class NoopDiscovery implements Discovery {
@@ -37,16 +37,6 @@ public class NoopDiscovery implements Discovery {
         return null;
     }
 
-    @Override
-    public void addListener(InitialStateDiscoveryListener listener) {
-
-    }
-
-    @Override
-    public void removeListener(InitialStateDiscoveryListener listener) {
-
-    }
-
     @Override
     public String nodeDescription() {
         return null;
@@ -72,6 +62,11 @@ public class NoopDiscovery implements Discovery {
         return null;
     }
 
+    @Override
+    public DiscoverySettings getDiscoverySettings() {
+        return null;
+    }
+
     @Override
     public void startInitialJoin() {
 

+ 3 - 3
plugins/discovery-azure/src/main/java/org/elasticsearch/cloud/azure/AzureDiscoveryModule.java

@@ -31,7 +31,7 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.azure.AzureDiscovery;
+import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin;
 
 /**
  * Azure Module
@@ -72,8 +72,8 @@ public class AzureDiscoveryModule extends AbstractModule {
      */
     public static boolean isDiscoveryReady(Settings settings, ESLogger logger) {
         // User set discovery.type: azure
-        if (!AzureDiscovery.AZURE.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings))) {
-            logger.trace("discovery.type not set to {}", AzureDiscovery.AZURE);
+        if (!AzureDiscoveryPlugin.AZURE.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings))) {
+            logger.trace("discovery.type not set to {}", AzureDiscoveryPlugin.AZURE);
             return false;
         }
 

+ 0 - 48
plugins/discovery-azure/src/main/java/org/elasticsearch/discovery/azure/AzureDiscovery.java

@@ -1,48 +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.azure;
-
-import org.elasticsearch.cluster.ClusterName;
-import org.elasticsearch.cluster.ClusterService;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
-import org.elasticsearch.discovery.zen.ZenDiscovery;
-import org.elasticsearch.discovery.zen.elect.ElectMasterService;
-import org.elasticsearch.discovery.zen.ping.ZenPingService;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.TransportService;
-
-/**
- *
- */
-public class AzureDiscovery extends ZenDiscovery {
-
-    public static final String AZURE = "azure";
-
-    @Inject
-    public AzureDiscovery(Settings settings, ClusterName clusterName, ThreadPool threadPool, TransportService transportService,
-                          ClusterService clusterService, ClusterSettings clusterSettings, ZenPingService pingService,
-                          DiscoverySettings discoverySettings, ElectMasterService electMasterService) {
-        super(settings, clusterName, threadPool, transportService, clusterService, clusterSettings,
-                pingService, electMasterService, discoverySettings);
-    }
-}

+ 4 - 3
plugins/discovery-azure/src/main/java/org/elasticsearch/plugin/discovery/azure/AzureDiscoveryPlugin.java

@@ -27,8 +27,8 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsModule;
 import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.azure.AzureDiscovery;
 import org.elasticsearch.discovery.azure.AzureUnicastHostsProvider;
+import org.elasticsearch.discovery.zen.ZenDiscovery;
 import org.elasticsearch.plugins.Plugin;
 
 import java.util.Collection;
@@ -36,6 +36,7 @@ import java.util.Collections;
 
 public class AzureDiscoveryPlugin extends Plugin {
 
+    public static final String AZURE = "azure";
     private final Settings settings;
     protected final ESLogger logger = Loggers.getLogger(AzureDiscoveryPlugin.class);
 
@@ -61,8 +62,8 @@ public class AzureDiscoveryPlugin extends Plugin {
 
     public void onModule(DiscoveryModule discoveryModule) {
         if (AzureDiscoveryModule.isDiscoveryReady(settings, logger)) {
-            discoveryModule.addDiscoveryType("azure", AzureDiscovery.class);
-            discoveryModule.addUnicastHostProvider(AzureUnicastHostsProvider.class);
+            discoveryModule.addDiscoveryType(AZURE, ZenDiscovery.class);
+            discoveryModule.addUnicastHostProvider(AZURE, AzureUnicastHostsProvider.class);
         }
     }
 

+ 0 - 17
plugins/discovery-ec2/src/main/java/org/elasticsearch/cloud/aws/Ec2Module.java

@@ -20,10 +20,6 @@
 package org.elasticsearch.cloud.aws;
 
 import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.common.logging.ESLogger;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.ec2.Ec2Discovery;
 
 public class Ec2Module extends AbstractModule {
 
@@ -32,17 +28,4 @@ public class Ec2Module extends AbstractModule {
         bind(AwsEc2Service.class).to(AwsEc2ServiceImpl.class).asEagerSingleton();
     }
 
-    /**
-     * Check if discovery is meant to start
-     * @return true if we can start discovery features
-     */
-    public static boolean isEc2DiscoveryActive(Settings settings, ESLogger logger) {
-        // User set discovery.type: ec2
-        if (!Ec2Discovery.EC2.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings))) {
-            logger.trace("discovery.type not set to {}", Ec2Discovery.EC2);
-            return false;
-        }
-
-        return true;
-    }
 }

+ 0 - 49
plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2Discovery.java

@@ -1,49 +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.ec2;
-
-import org.elasticsearch.cluster.ClusterName;
-import org.elasticsearch.cluster.ClusterService;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
-import org.elasticsearch.discovery.zen.ZenDiscovery;
-import org.elasticsearch.discovery.zen.elect.ElectMasterService;
-import org.elasticsearch.discovery.zen.ping.ZenPingService;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.TransportService;
-
-/**
- *
- */
-public class Ec2Discovery extends ZenDiscovery {
-
-    public static final String EC2 = "ec2";
-
-    @Inject
-    public Ec2Discovery(Settings settings, ClusterName clusterName, ThreadPool threadPool, TransportService transportService,
-                        ClusterService clusterService, ClusterSettings clusterSettings, ZenPingService pingService,
-                        DiscoverySettings discoverySettings,
-                        ElectMasterService electMasterService) {
-        super(settings, clusterName, threadPool, transportService, clusterService, clusterSettings,
-                pingService, electMasterService, discoverySettings);
-    }
-}

+ 5 - 5
plugins/discovery-ec2/src/main/java/org/elasticsearch/plugin/discovery/ec2/Ec2DiscoveryPlugin.java

@@ -32,7 +32,7 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsModule;
 import org.elasticsearch.discovery.DiscoveryModule;
 import org.elasticsearch.discovery.ec2.AwsEc2UnicastHostsProvider;
-import org.elasticsearch.discovery.ec2.Ec2Discovery;
+import org.elasticsearch.discovery.zen.ZenDiscovery;
 import org.elasticsearch.plugins.Plugin;
 
 import java.security.AccessController;
@@ -45,6 +45,8 @@ import java.util.Collection;
  */
 public class Ec2DiscoveryPlugin extends Plugin {
 
+    public static final String EC2 = "ec2";
+
     // ClientConfiguration clinit has some classloader problems
     // TODO: fix that
     static {
@@ -98,10 +100,8 @@ public class Ec2DiscoveryPlugin extends Plugin {
     }
 
     public void onModule(DiscoveryModule discoveryModule) {
-        if (Ec2Module.isEc2DiscoveryActive(settings, logger)) {
-            discoveryModule.addDiscoveryType("ec2", Ec2Discovery.class);
-            discoveryModule.addUnicastHostProvider(AwsEc2UnicastHostsProvider.class);
-        }
+        discoveryModule.addDiscoveryType(EC2, ZenDiscovery.class);
+        discoveryModule.addUnicastHostProvider(EC2, AwsEc2UnicastHostsProvider.class);
     }
 
     public void onModule(SettingsModule settingsModule) {

+ 0 - 16
plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoverySettingsTests.java

@@ -21,7 +21,6 @@ package org.elasticsearch.discovery.ec2;
 
 import com.amazonaws.Protocol;
 import org.elasticsearch.cloud.aws.AwsEc2Service;
-import org.elasticsearch.cloud.aws.Ec2Module;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.test.ESTestCase;
 
@@ -30,21 +29,6 @@ import static org.hamcrest.Matchers.isEmptyString;
 
 public class Ec2DiscoverySettingsTests extends ESTestCase {
 
-    public void testDiscoveryReady() {
-        Settings settings = Settings.builder()
-                .put("discovery.type", "ec2")
-                .build();
-        boolean discoveryReady = Ec2Module.isEc2DiscoveryActive(settings, logger);
-        assertThat(discoveryReady, is(true));
-    }
-
-    public void testDiscoveryNotReady() {
-        Settings settings = Settings.EMPTY;
-        boolean discoveryReady = Ec2Module.isEc2DiscoveryActive(settings, logger);
-        assertThat(discoveryReady, is(false));
-    }
-
-
     private static final Settings AWS = Settings.builder()
         .put(AwsEc2Service.KEY_SETTING.getKey(), "global-key")
         .put(AwsEc2Service.SECRET_SETTING.getKey(), "global-secret")

+ 0 - 59
plugins/discovery-gce/src/main/java/org/elasticsearch/discovery/gce/GceDiscovery.java

@@ -1,59 +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.gce;
-
-import org.elasticsearch.cluster.ClusterName;
-import org.elasticsearch.cluster.ClusterService;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.ClusterSettings;
-import org.elasticsearch.common.settings.Setting;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
-import org.elasticsearch.discovery.zen.ZenDiscovery;
-import org.elasticsearch.discovery.zen.elect.ElectMasterService;
-import org.elasticsearch.discovery.zen.ping.ZenPingService;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.TransportService;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- *
- */
-public class GceDiscovery extends ZenDiscovery {
-
-    public static final String GCE = "gce";
-
-    /**
-     * discovery.gce.tags: The gce discovery can filter machines to include in the cluster based on tags.
-     */
-    public static final Setting<List<String>> TAGS_SETTING =
-        Setting.listSetting("discovery.gce.tags", Collections.emptyList(), s -> s, false, Setting.Scope.CLUSTER);
-
-    @Inject
-    public GceDiscovery(Settings settings, ClusterName clusterName, ThreadPool threadPool, TransportService transportService,
-                        ClusterService clusterService, ClusterSettings clusterSettings, ZenPingService pingService,
-                        DiscoverySettings discoverySettings,
-                        ElectMasterService electMasterService) {
-        super(settings, clusterName, threadPool, transportService, clusterService, clusterSettings,
-                pingService, electMasterService, discoverySettings);
-    }
-}

+ 9 - 1
plugins/discovery-gce/src/main/java/org/elasticsearch/discovery/gce/GceUnicastHostsProvider.java

@@ -30,6 +30,7 @@ import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.network.NetworkAddress;
 import org.elasticsearch.common.network.NetworkService;
+import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.common.unit.TimeValue;
@@ -40,6 +41,7 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -47,6 +49,12 @@ import java.util.List;
  */
 public class GceUnicastHostsProvider extends AbstractComponent implements UnicastHostsProvider {
 
+    /**
+     * discovery.gce.tags: The gce discovery can filter machines to include in the cluster based on tags.
+     */
+    public static final Setting<List<String>> TAGS_SETTING =
+            Setting.listSetting("discovery.gce.tags", Collections.emptyList(), s -> s, false, Setting.Scope.CLUSTER);
+
     static final class Status {
         private static final String TERMINATED = "TERMINATED";
     }
@@ -79,7 +87,7 @@ public class GceUnicastHostsProvider extends AbstractComponent implements Unicas
         this.project = GceComputeService.PROJECT_SETTING.get(settings);
         this.zones = GceComputeService.ZONE_SETTING.get(settings);
 
-        this.tags = GceDiscovery.TAGS_SETTING.get(settings);
+        this.tags = TAGS_SETTING.get(settings);
         if (logger.isDebugEnabled()) {
             logger.debug("using tags {}", this.tags);
         }

+ 7 - 7
plugins/discovery-gce/src/main/java/org/elasticsearch/plugin/discovery/gce/GceDiscoveryPlugin.java

@@ -21,7 +21,6 @@ package org.elasticsearch.plugin.discovery.gce;
 
 import com.google.api.client.http.HttpHeaders;
 import com.google.api.client.util.ClassInfo;
-
 import org.elasticsearch.SpecialPermission;
 import org.elasticsearch.cloud.gce.GceComputeService;
 import org.elasticsearch.cloud.gce.GceModule;
@@ -32,8 +31,8 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsModule;
 import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.gce.GceDiscovery;
 import org.elasticsearch.discovery.gce.GceUnicastHostsProvider;
+import org.elasticsearch.discovery.zen.ZenDiscovery;
 import org.elasticsearch.plugins.Plugin;
 
 import java.security.AccessController;
@@ -42,6 +41,9 @@ import java.util.Collection;
 import java.util.Collections;
 
 public class GceDiscoveryPlugin extends Plugin {
+
+    public static final String GCE = "gce";
+
     static {
         /*
          * GCE's http client changes access levels because its silly and we
@@ -93,18 +95,16 @@ public class GceDiscoveryPlugin extends Plugin {
     }
 
     public void onModule(DiscoveryModule discoveryModule) {
-        discoveryModule.addDiscoveryType("gce", GceDiscovery.class);
+        discoveryModule.addDiscoveryType(GCE, ZenDiscovery.class);
         // If discovery.type: gce, we add Gce as a unicast provider
-        if (GceDiscovery.GCE.equalsIgnoreCase(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings))) {
-            discoveryModule.addUnicastHostProvider(GceUnicastHostsProvider.class);
-        }
+            discoveryModule.addUnicastHostProvider(GCE, GceUnicastHostsProvider.class);
     }
 
     public void onModule(SettingsModule settingsModule) {
         // Register GCE settings
         settingsModule.registerSetting(GceComputeService.PROJECT_SETTING);
         settingsModule.registerSetting(GceComputeService.ZONE_SETTING);
-        settingsModule.registerSetting(GceDiscovery.TAGS_SETTING);
+        settingsModule.registerSetting(GceUnicastHostsProvider.TAGS_SETTING);
         settingsModule.registerSetting(GceComputeService.REFRESH_SETTING);
         settingsModule.registerSetting(GceComputeService.RETRY_SETTING);
         settingsModule.registerSetting(GceComputeService.MAX_WAIT_SETTING);

+ 4 - 4
plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java

@@ -131,7 +131,7 @@ public class GceDiscoveryTests extends ESTestCase {
         Settings nodeSettings = Settings.builder()
                 .put(GceComputeService.PROJECT_SETTING.getKey(), projectName)
                 .put(GceComputeService.ZONE_SETTING.getKey(), "europe-west1-b")
-                .putArray(GceDiscovery.TAGS_SETTING.getKey(), "elasticsearch")
+                .putArray(GceUnicastHostsProvider.TAGS_SETTING.getKey(), "elasticsearch")
                 .build();
         mock = new GceComputeServiceMock(nodeSettings, networkService);
         List<DiscoveryNode> discoveryNodes = buildDynamicNodes(mock, nodeSettings);
@@ -143,7 +143,7 @@ public class GceDiscoveryTests extends ESTestCase {
         Settings nodeSettings = Settings.builder()
                 .put(GceComputeService.PROJECT_SETTING.getKey(), projectName)
                 .put(GceComputeService.ZONE_SETTING.getKey(), "europe-west1-b")
-                .putArray(GceDiscovery.TAGS_SETTING.getKey(), "elasticsearch", "dev")
+                .putArray(GceUnicastHostsProvider.TAGS_SETTING.getKey(), "elasticsearch", "dev")
                 .build();
         mock = new GceComputeServiceMock(nodeSettings, networkService);
         List<DiscoveryNode> discoveryNodes = buildDynamicNodes(mock, nodeSettings);
@@ -165,7 +165,7 @@ public class GceDiscoveryTests extends ESTestCase {
         Settings nodeSettings = Settings.builder()
                 .put(GceComputeService.PROJECT_SETTING.getKey(), projectName)
                 .put(GceComputeService.ZONE_SETTING.getKey(), "europe-west1-b")
-                .putArray(GceDiscovery.TAGS_SETTING.getKey(), "elasticsearch")
+                .putArray(GceUnicastHostsProvider.TAGS_SETTING.getKey(), "elasticsearch")
                 .build();
         mock = new GceComputeServiceMock(nodeSettings, networkService);
         List<DiscoveryNode> discoveryNodes = buildDynamicNodes(mock, nodeSettings);
@@ -176,7 +176,7 @@ public class GceDiscoveryTests extends ESTestCase {
         Settings nodeSettings = Settings.builder()
                 .put(GceComputeService.PROJECT_SETTING.getKey(), projectName)
                 .put(GceComputeService.ZONE_SETTING.getKey(), "europe-west1-b")
-                .putArray(GceDiscovery.TAGS_SETTING.getKey(), "elasticsearch", "dev")
+                .putArray(GceUnicastHostsProvider.TAGS_SETTING.getKey(), "elasticsearch", "dev")
                 .build();
         mock = new GceComputeServiceMock(nodeSettings, networkService);
         List<DiscoveryNode> discoveryNodes = buildDynamicNodes(mock, nodeSettings);

+ 7 - 7
qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java

@@ -23,9 +23,9 @@ import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.service.InternalClusterService;
 import org.elasticsearch.common.SuppressForbidden;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.node.internal.InternalSettingsPreparer;
@@ -66,14 +66,14 @@ public class TribeUnitTests extends ESTestCase {
                 .put(baseSettings)
                 .put("cluster.name", "tribe1")
                 .put("node.name", "tribe1_node")
-                .put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), random().nextLong())
+                    .put(InternalClusterService.NODE_ID_SEED_SETTING.getKey(), random().nextLong())
                 .build()).start();
         tribe2 = new TribeClientNode(
             Settings.builder()
                 .put(baseSettings)
                 .put("cluster.name", "tribe2")
                 .put("node.name", "tribe2_node")
-                .put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), random().nextLong())
+                    .put(InternalClusterService.NODE_ID_SEED_SETTING.getKey(), random().nextLong())
                 .build()).start();
     }
 
@@ -88,8 +88,8 @@ public class TribeUnitTests extends ESTestCase {
         System.setProperty("es.cluster.name", "tribe_node_cluster");
         System.setProperty("es.tribe.t1.cluster.name", "tribe1");
         System.setProperty("es.tribe.t2.cluster.name", "tribe2");
-        System.setProperty("es.tribe.t1.discovery.id.seed", Long.toString(random().nextLong()));
-        System.setProperty("es.tribe.t2.discovery.id.seed", Long.toString(random().nextLong()));
+        System.setProperty("es.tribe.t1.node_id.seed", Long.toString(random().nextLong()));
+        System.setProperty("es.tribe.t2.node_id.seed", Long.toString(random().nextLong()));
 
         try {
             assertTribeNodeSuccessfullyCreated(Settings.EMPTY);
@@ -97,8 +97,8 @@ public class TribeUnitTests extends ESTestCase {
             System.clearProperty("es.cluster.name");
             System.clearProperty("es.tribe.t1.cluster.name");
             System.clearProperty("es.tribe.t2.cluster.name");
-            System.clearProperty("es.tribe.t1.discovery.id.seed");
-            System.clearProperty("es.tribe.t2.discovery.id.seed");
+            System.clearProperty("es.tribe.t1.node_id.seed");
+            System.clearProperty("es.tribe.t2.node_id.seed");
         }
     }
 

+ 2 - 2
qa/evil-tests/src/test/resources/org/elasticsearch/tribe/elasticsearch.yml

@@ -1,5 +1,5 @@
 cluster.name: tribe_node_cluster
 tribe.t1.cluster.name: tribe1
 tribe.t2.cluster.name: tribe2
-tribe.t1.discovery.id.seed: 1
-tribe.t2.discovery.id.seed: 2
+tribe.t1.node_id.seed: 1
+tribe.t2.node_id.seed: 2

+ 6 - 5
test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

@@ -45,6 +45,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.UnassignedInfo;
 import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
+import org.elasticsearch.cluster.service.InternalClusterService;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.breaker.CircuitBreaker;
@@ -61,7 +62,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
-import org.elasticsearch.discovery.DiscoveryService;
+import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.http.HttpServerTransport;
@@ -589,7 +590,7 @@ public final class InternalTestCluster extends TestCluster {
                 .put(Environment.PATH_HOME_SETTING.getKey(), baseDir) // allow overriding path.home
                 .put(settings)
                 .put("node.name", name)
-                .put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), seed)
+                .put(InternalClusterService.NODE_ID_SEED_SETTING.getKey(), seed)
                 .build();
         MockNode node = new MockNode(finalSettings, version, plugins);
         return new NodeAndClient(name, node);
@@ -670,7 +671,7 @@ public final class InternalTestCluster extends TestCluster {
         Builder builder = settingsBuilder().put(settings).put(Node.NODE_CLIENT_SETTING.getKey(), true);
         if (size() == 0) {
             // if we are the first node - don't wait for a state
-            builder.put(DiscoveryService.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
+            builder.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
         }
         String name = startNode(builder);
         return nodes.get(name).nodeClient();
@@ -836,8 +837,8 @@ public final class InternalTestCluster extends TestCluster {
                     IOUtils.rm(nodeEnv.nodeDataPaths());
                 }
             }
-            final long newIdSeed = DiscoveryService.DISCOVERY_SEED_SETTING.get(node.settings()) + 1; // use a new seed to make sure we have new node id
-            Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(DiscoveryService.DISCOVERY_SEED_SETTING.getKey(), newIdSeed).build();
+            final long newIdSeed = InternalClusterService.NODE_ID_SEED_SETTING.get(node.settings()) + 1; // use a new seed to make sure we have new node id
+            Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(InternalClusterService.NODE_ID_SEED_SETTING.getKey(), newIdSeed).build();
             Collection<Class<? extends Plugin>> plugins = node.getPlugins();
             Version version = node.getVersion();
             node = new MockNode(finalSettings, version, plugins);

+ 5 - 0
test/framework/src/main/java/org/elasticsearch/test/cluster/NoopClusterService.java

@@ -83,6 +83,11 @@ public class NoopClusterService implements ClusterService {
 
     }
 
+    @Override
+    public void removeInitialStateBlock(int blockId) throws IllegalStateException {
+
+    }
+
     @Override
     public OperationRouting operationRouting() {
         return null;

+ 4 - 0
test/framework/src/main/java/org/elasticsearch/test/cluster/TestClusterService.java

@@ -143,7 +143,11 @@ public class TestClusterService implements ClusterService {
     @Override
     public void removeInitialStateBlock(ClusterBlock block) throws IllegalStateException {
         throw new UnsupportedOperationException();
+    }
 
+    @Override
+    public void removeInitialStateBlock(int blockId) throws IllegalStateException {
+        throw new UnsupportedOperationException();
     }
 
     @Override