Browse Source

Rename no-master-block setting (#38350)

Replaces `discovery.zen.no_master_block` with `cluster.no_master_block`. Any
value set for the old setting is now ignored.
David Turner 6 years ago
parent
commit
3b2a0d7959
25 changed files with 268 additions and 112 deletions
  1. 8 0
      docs/reference/migration/migrate_7_0/discovery.asciidoc
  2. 6 2
      docs/reference/modules/discovery/discovery-settings.asciidoc
  3. 2 2
      modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java
  4. 5 6
      server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java
  5. 1 2
      server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java
  6. 73 0
      server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java
  7. 3 1
      server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
  8. 0 34
      server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java
  9. 8 5
      server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
  10. 5 5
      server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java
  11. 3 3
      server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
  12. 2 2
      server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java
  13. 3 3
      server/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java
  14. 13 13
      server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java
  15. 5 5
      server/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java
  16. 4 4
      server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java
  17. 5 5
      server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java
  18. 101 0
      server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java
  19. 2 2
      server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java
  20. 5 4
      server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java
  21. 6 6
      server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java
  22. 2 2
      test/framework/src/main/java/org/elasticsearch/test/RandomObjects.java
  23. 2 2
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/TransportMonitoringBulkActionTests.java
  24. 2 2
      x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherIndexingListenerTests.java
  25. 2 2
      x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java

+ 8 - 0
docs/reference/migration/migrate_7_0/discovery.asciidoc

@@ -38,3 +38,11 @@ file:
 - `discovery.seed_hosts`
 - `discovery.seed_providers`
 - `cluster.initial_master_nodes`
+
+[float]
+==== New name for `no_master_block` setting
+
+The `discovery.zen.no_master_block` setting is now known as
+`cluster.no_master_block`. Any value set for `discovery.zen.no_master_block` is
+now ignored. You should remove this setting and, if needed, set
+`cluster.no_master_block` appropriately after the upgrade.

+ 6 - 2
docs/reference/modules/discovery/discovery-settings.asciidoc

@@ -134,7 +134,7 @@ Discovery and cluster formation are affected by the following settings:
     the addresses of seed nodes. By default, it is the
     <<settings-based-hosts-provider,settings-based hosts provider>>.
 
-[[no-master-block]]`discovery.zen.no_master_block`::
+[[no-master-block]]`cluster.no_master_block`::
 Specifies which operations are rejected when there is no active master in a
 cluster. This setting has two valid values:
 +
@@ -150,12 +150,16 @@ cluster.
 
 [NOTE]
 ===============================
-* The `discovery.zen.no_master_block` setting doesn't apply to nodes-based APIs
+* The `cluster.no_master_block` setting doesn't apply to nodes-based APIs
 (for example, cluster stats, node info, and node stats APIs). Requests to these
 APIs are not be blocked and can run on any available node.
   
 * For the cluster to be fully operational, it must have an active master.
 ===============================
+
+WARNING: This setting replaces the `discovery.zen.no_master_block` setting in
+earlier versions. The `discovery.zen.no_master_block` setting is ignored.
+
 --
 
 `discovery.seed_hosts`::

+ 2 - 2
modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.action.OriginalIndices;
 import org.elasticsearch.action.search.SearchPhaseExecutionException;
 import org.elasticsearch.action.search.ShardSearchFailure;
 import org.elasticsearch.cluster.block.ClusterBlockException;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.breaker.CircuitBreakingException;
@@ -37,7 +38,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentLocation;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchParseException;
@@ -64,7 +64,7 @@ import static org.hamcrest.Matchers.instanceOf;
 public class RankEvalResponseTests extends ESTestCase {
 
     private static final Exception[] RANDOM_EXCEPTIONS = new Exception[] {
-            new ClusterBlockException(singleton(DiscoverySettings.NO_MASTER_BLOCK_WRITES)),
+            new ClusterBlockException(singleton(NoMasterBlockService.NO_MASTER_BLOCK_WRITES)),
             new CircuitBreakingException("Data too large", 123, 456, CircuitBreaker.Durability.PERMANENT),
             new SearchParseException(new TestSearchContext(null), "Parse failure", new XContentLocation(12, 98)),
             new IllegalArgumentException("Closed resource", new RuntimeException("Resource")),

+ 5 - 6
server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java

@@ -58,7 +58,6 @@ import org.elasticsearch.common.util.concurrent.ListenableFuture;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.discovery.Discovery;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.DiscoveryStats;
 import org.elasticsearch.discovery.HandshakingTransportAddressConnector;
 import org.elasticsearch.discovery.PeerFinder;
@@ -82,7 +81,7 @@ import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
-import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_ID;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ID;
 import static org.elasticsearch.gateway.ClusterStateUpdaters.hideStateIfNotRecovered;
 import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
 
@@ -103,7 +102,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
     private final JoinHelper joinHelper;
     private final NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor;
     private final Supplier<CoordinationState.PersistedState> persistedStateSupplier;
-    private final DiscoverySettings discoverySettings;
+    private final NoMasterBlockService noMasterBlockService;
     // TODO: the following field is package-private as some tests require access to it
     // These tests can be rewritten to use public methods once Coordinator is more feature-complete
     final Object mutex = new Object();
@@ -149,7 +148,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
         this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService,
             this::getCurrentTerm, this::getStateForMasterService, this::handleJoinRequest, this::joinLeaderInTerm, this.onJoinValidators);
         this.persistedStateSupplier = persistedStateSupplier;
-        this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
+        this.noMasterBlockService = new NoMasterBlockService(settings, clusterSettings);
         this.lastKnownLeader = Optional.empty();
         this.lastJoin = Optional.empty();
         this.joinAccumulator = new InitialJoinAccumulator();
@@ -632,7 +631,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
             ClusterState initialState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
                 .blocks(ClusterBlocks.builder()
                     .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
-                    .addGlobalBlock(discoverySettings.getNoMasterBlock()))
+                    .addGlobalBlock(noMasterBlockService.getNoMasterBlock()))
                 .nodes(DiscoveryNodes.builder().add(getLocalNode()).localNodeId(getLocalNode().getId()))
                 .build();
             applierState = initialState;
@@ -934,7 +933,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
             assert clusterState.blocks().hasGlobalBlockWithId(NO_MASTER_BLOCK_ID) == false :
                 "NO_MASTER_BLOCK should only be added by Coordinator";
             final ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(clusterState.blocks()).addGlobalBlock(
-                discoverySettings.getNoMasterBlock()).build();
+                noMasterBlockService.getNoMasterBlock()).build();
             final DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder(clusterState.nodes()).masterNodeId(null).build();
             return ClusterState.builder(clusterState).blocks(clusterBlocks).nodes(discoveryNodes).build();
         } else {

+ 1 - 2
server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java

@@ -30,7 +30,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
 import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.zen.ElectMasterService;
 import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
 
@@ -191,7 +190,7 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor<JoinTaskExecut
         // or removed by us above
         ClusterState tmpState = ClusterState.builder(currentState).nodes(nodesBuilder).blocks(ClusterBlocks.builder()
             .blocks(currentState.blocks())
-            .removeGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID))
+            .removeGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_ID))
             .minimumMasterNodesOnPublishingMaster(minimumMasterNodesOnLocalNode)
             .build();
         logger.trace("becomeMasterAndTrimConflictingNodes: {}", tmpState.nodes());

+ 73 - 0
server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java

@@ -0,0 +1,73 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.cluster.coordination;
+
+import org.elasticsearch.cluster.block.ClusterBlock;
+import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.common.settings.Setting;
+import org.elasticsearch.common.settings.Setting.Property;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.rest.RestStatus;
+
+import java.util.EnumSet;
+
+public class NoMasterBlockService {
+    public static final int NO_MASTER_BLOCK_ID = 2;
+    public static final ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, false,
+        RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE));
+    public static final ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, false,
+        RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL);
+
+    public static final Setting<ClusterBlock> LEGACY_NO_MASTER_BLOCK_SETTING =
+        new Setting<>("discovery.zen.no_master_block", "write", NoMasterBlockService::parseNoMasterBlock,
+            Property.Dynamic, Property.NodeScope, Property.Deprecated);
+    public static final Setting<ClusterBlock> NO_MASTER_BLOCK_SETTING =
+        new Setting<>("cluster.no_master_block", "write", NoMasterBlockService::parseNoMasterBlock,
+            Property.Dynamic, Property.NodeScope);
+
+    private volatile ClusterBlock noMasterBlock;
+
+    public NoMasterBlockService(Settings settings, ClusterSettings clusterSettings) {
+        this.noMasterBlock = NO_MASTER_BLOCK_SETTING.get(settings);
+        clusterSettings.addSettingsUpdateConsumer(NO_MASTER_BLOCK_SETTING, this::setNoMasterBlock);
+
+        LEGACY_NO_MASTER_BLOCK_SETTING.get(settings); // for deprecation warnings
+        clusterSettings.addSettingsUpdateConsumer(LEGACY_NO_MASTER_BLOCK_SETTING, b -> {}); // for deprecation warnings
+    }
+
+    private static ClusterBlock parseNoMasterBlock(String value) {
+        switch (value) {
+            case "all":
+                return NO_MASTER_BLOCK_ALL;
+            case "write":
+                return NO_MASTER_BLOCK_WRITES;
+            default:
+                throw new IllegalArgumentException("invalid no-master block [" + value + "], must be one of [all, write]");
+        }
+    }
+
+    public ClusterBlock getNoMasterBlock() {
+        return noMasterBlock;
+    }
+
+    private void setNoMasterBlock(ClusterBlock noMasterBlock) {
+        this.noMasterBlock = noMasterBlock;
+    }
+}

+ 3 - 1
server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

@@ -41,6 +41,7 @@ import org.elasticsearch.cluster.coordination.FollowersChecker;
 import org.elasticsearch.cluster.coordination.JoinHelper;
 import org.elasticsearch.cluster.coordination.LagDetector;
 import org.elasticsearch.cluster.coordination.LeaderChecker;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.coordination.Reconfigurator;
 import org.elasticsearch.cluster.metadata.IndexGraveyard;
 import org.elasticsearch.cluster.metadata.MetaData;
@@ -232,7 +233,8 @@ public final class ClusterSettings extends AbstractScopedSettings {
                     DiscoverySettings.PUBLISH_TIMEOUT_SETTING,
                     DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING,
                     DiscoverySettings.COMMIT_TIMEOUT_SETTING,
-                    DiscoverySettings.NO_MASTER_BLOCK_SETTING,
+                    NoMasterBlockService.NO_MASTER_BLOCK_SETTING,
+                    NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING,
                     GatewayService.EXPECTED_DATA_NODES_SETTING,
                     GatewayService.EXPECTED_MASTER_NODES_SETTING,
                     GatewayService.EXPECTED_NODES_SETTING,

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

@@ -19,27 +19,17 @@
 
 package org.elasticsearch.discovery;
 
-import org.elasticsearch.cluster.block.ClusterBlock;
-import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.rest.RestStatus;
-
-import java.util.EnumSet;
 
 /**
  * Exposes common discovery settings that may be supported by all the different discovery implementations
  */
 public class DiscoverySettings {
 
-    public static final int NO_MASTER_BLOCK_ID = 2;
-    public static final ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, false,
-        RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL);
-    public static final ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, false,
-        RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE));
     /**
      * sets the timeout for a complete publishing cycle, including both sending and committing. the master
      * will continue to process the next cluster state update after this time has elapsed
@@ -56,26 +46,20 @@ public class DiscoverySettings {
         new Setting<>("discovery.zen.commit_timeout", PUBLISH_TIMEOUT_SETTING::getRaw,
             (s) -> TimeValue.parseTimeValue(s, TimeValue.timeValueSeconds(30), "discovery.zen.commit_timeout"),
             Property.Dynamic, Property.NodeScope, Property.Deprecated);
-    public static final Setting<ClusterBlock> NO_MASTER_BLOCK_SETTING =
-        new Setting<>("discovery.zen.no_master_block", "write", DiscoverySettings::parseNoMasterBlock,
-            Property.Dynamic, Property.NodeScope);
     public static final Setting<Boolean> PUBLISH_DIFF_ENABLE_SETTING =
         Setting.boolSetting("discovery.zen.publish_diff.enable", true, Property.Dynamic, Property.NodeScope, Property.Deprecated);
     public static final Setting<TimeValue> INITIAL_STATE_TIMEOUT_SETTING =
         Setting.positiveTimeSetting("discovery.initial_state_timeout", TimeValue.timeValueSeconds(30), Property.NodeScope);
 
-    private volatile ClusterBlock noMasterBlock;
     private volatile TimeValue publishTimeout;
 
     private volatile TimeValue commitTimeout;
     private volatile boolean publishDiff;
 
     public DiscoverySettings(Settings settings, ClusterSettings clusterSettings) {
-        clusterSettings.addSettingsUpdateConsumer(NO_MASTER_BLOCK_SETTING, this::setNoMasterBlock);
         clusterSettings.addSettingsUpdateConsumer(PUBLISH_DIFF_ENABLE_SETTING, this::setPublishDiff);
         clusterSettings.addSettingsUpdateConsumer(COMMIT_TIMEOUT_SETTING, this::setCommitTimeout);
         clusterSettings.addSettingsUpdateConsumer(PUBLISH_TIMEOUT_SETTING, this::setPublishTimeout);
-        this.noMasterBlock = NO_MASTER_BLOCK_SETTING.get(settings);
         this.publishTimeout = PUBLISH_TIMEOUT_SETTING.get(settings);
         this.commitTimeout = COMMIT_TIMEOUT_SETTING.get(settings);
         this.publishDiff = PUBLISH_DIFF_ENABLE_SETTING.get(settings);
@@ -92,14 +76,6 @@ public class DiscoverySettings {
         return commitTimeout;
     }
 
-    public ClusterBlock getNoMasterBlock() {
-        return noMasterBlock;
-    }
-
-    private void setNoMasterBlock(ClusterBlock noMasterBlock) {
-        this.noMasterBlock = noMasterBlock;
-    }
-
     private void setPublishDiff(boolean publishDiff) {
         this.publishDiff = publishDiff;
     }
@@ -114,14 +90,4 @@ public class DiscoverySettings {
 
     public boolean getPublishDiff() { return publishDiff;}
 
-    private static ClusterBlock parseNoMasterBlock(String value) {
-        switch (value) {
-            case "all":
-                return NO_MASTER_BLOCK_ALL;
-            case "write":
-                return NO_MASTER_BLOCK_WRITES;
-            default:
-                throw new IllegalArgumentException("invalid master block [" + value + "]");
-        }
-    }
 }

+ 8 - 5
server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java

@@ -32,6 +32,7 @@ import org.elasticsearch.cluster.ClusterStateTaskConfig;
 import org.elasticsearch.cluster.NotMasterException;
 import org.elasticsearch.cluster.block.ClusterBlocks;
 import org.elasticsearch.cluster.coordination.JoinTaskExecutor;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
@@ -120,6 +121,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
     private final TransportService transportService;
     private final MasterService masterService;
     private final DiscoverySettings discoverySettings;
+    private final NoMasterBlockService noMasterBlockService;
     protected final ZenPing zenPing; // protected to allow tests access
     private final MasterFaultDetection masterFD;
     private final NodesFaultDetection nodesFD;
@@ -167,6 +169,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
         this.clusterApplier = clusterApplier;
         this.transportService = transportService;
         this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
+        this.noMasterBlockService = new NoMasterBlockService(settings, clusterSettings);
         this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider);
         this.electMaster = new ElectMasterService(settings);
         this.pingTimeout = PING_TIMEOUT_SETTING.get(settings);
@@ -252,7 +255,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
             ClusterState initialState = builder
                 .blocks(ClusterBlocks.builder()
                     .addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
-                    .addGlobalBlock(discoverySettings.getNoMasterBlock()))
+                    .addGlobalBlock(noMasterBlockService.getNoMasterBlock()))
                 .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()))
                 .build();
             committedState.set(initialState);
@@ -640,7 +643,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
         }
 
         assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master";
-        assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) :
+        assert !newClusterState.blocks().hasGlobalBlock(noMasterBlockService.getNoMasterBlock()) :
             "received a cluster state with a master block";
 
         if (currentState.nodes().isLocalNodeElectedMaster() && newClusterState.nodes().isLocalNodeElectedMaster() == false) {
@@ -670,7 +673,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
             return false;
         }
 
-        if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
+        if (currentState.blocks().hasGlobalBlock(noMasterBlockService.getNoMasterBlock())) {
             // its a fresh update from the master as we transition from a start of not having a master to having one
             logger.debug("got first state from fresh master [{}]", newClusterState.nodes().getMasterNodeId());
         }
@@ -898,10 +901,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
 
         if (clusterState.nodes().getMasterNodeId() != null) {
             // remove block if it already exists before adding new one
-            assert clusterState.blocks().hasGlobalBlockWithId(discoverySettings.getNoMasterBlock().id()) == false :
+            assert clusterState.blocks().hasGlobalBlockWithId(noMasterBlockService.getNoMasterBlock().id()) == false :
                 "NO_MASTER_BLOCK should only be added by ZenDiscovery";
             ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(clusterState.blocks())
-                .addGlobalBlock(discoverySettings.getNoMasterBlock())
+                .addGlobalBlock(noMasterBlockService.getNoMasterBlock())
                 .build();
 
             DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder(clusterState.nodes()).masterNodeId(null).build();

+ 5 - 5
server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java

@@ -28,6 +28,7 @@ import org.elasticsearch.action.search.ShardSearchFailure;
 import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
 import org.elasticsearch.client.transport.NoNodeAvailableException;
 import org.elasticsearch.cluster.block.ClusterBlockException;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.ParsingException;
 import org.elasticsearch.common.Strings;
@@ -43,7 +44,6 @@ import org.elasticsearch.common.xcontent.XContentLocation;
 import org.elasticsearch.common.xcontent.XContentParseException;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexNotFoundException;
 import org.elasticsearch.index.query.QueryShardException;
@@ -378,9 +378,9 @@ public class ElasticsearchExceptionTests extends ESTestCase {
 
     public void testToXContentWithHeadersAndMetadata() throws IOException {
         ElasticsearchException e = new ElasticsearchException("foo",
-                                        new ElasticsearchException("bar",
-                                                new ElasticsearchException("baz",
-                                                        new ClusterBlockException(singleton(DiscoverySettings.NO_MASTER_BLOCK_WRITES)))));
+            new ElasticsearchException("bar",
+                new ElasticsearchException("baz",
+                    new ClusterBlockException(singleton(NoMasterBlockService.NO_MASTER_BLOCK_WRITES)))));
         e.addHeader("foo_0", "0");
         e.addHeader("foo_1", "1");
         e.addMetadata("es.metadata_foo_0", "foo_0");
@@ -911,7 +911,7 @@ public class ElasticsearchExceptionTests extends ESTestCase {
         int type = randomIntBetween(0, 5);
         switch (type) {
             case 0:
-                actual = new ClusterBlockException(singleton(DiscoverySettings.NO_MASTER_BLOCK_WRITES));
+                actual = new ClusterBlockException(singleton(NoMasterBlockService.NO_MASTER_BLOCK_WRITES));
                 expected = new ElasticsearchException("Elasticsearch exception [type=cluster_block_exception, " +
                         "reason=blocked by: [SERVICE_UNAVAILABLE/2/no master];]");
                 break;

+ 3 - 3
server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java

@@ -34,6 +34,7 @@ import org.elasticsearch.client.AbstractClientHeadersTestCase;
 import org.elasticsearch.cluster.action.shard.ShardStateAction;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.coordination.CoordinationStateRejectedException;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.IllegalShardRoutingStateException;
 import org.elasticsearch.cluster.routing.ShardRouting;
@@ -56,7 +57,6 @@ import org.elasticsearch.common.util.CancellableThreadsTests;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.common.xcontent.UnknownNamedObjectException;
 import org.elasticsearch.common.xcontent.XContentLocation;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.env.ShardLockObtainFailedException;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.engine.RecoveryEngineException;
@@ -485,9 +485,9 @@ public class ExceptionSerializationTests extends ESTestCase {
     }
 
     public void testClusterBlockException() throws IOException {
-        ClusterBlockException ex = serialize(new ClusterBlockException(singleton(DiscoverySettings.NO_MASTER_BLOCK_WRITES)));
+        ClusterBlockException ex = serialize(new ClusterBlockException(singleton(NoMasterBlockService.NO_MASTER_BLOCK_WRITES)));
         assertEquals("blocked by: [SERVICE_UNAVAILABLE/2/no master];", ex.getMessage());
-        assertTrue(ex.blocks().contains(DiscoverySettings.NO_MASTER_BLOCK_WRITES));
+        assertTrue(ex.blocks().contains(NoMasterBlockService.NO_MASTER_BLOCK_WRITES));
         assertEquals(1, ex.blocks().size());
     }
 

+ 2 - 2
server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java

@@ -25,6 +25,7 @@ import org.elasticsearch.action.support.PlainActionFuture;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.action.shard.ShardStateAction;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@@ -36,7 +37,6 @@ import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.PageCacheRecycler;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexService;
 import org.elasticsearch.index.shard.IndexShard;
@@ -97,7 +97,7 @@ public class TransportResyncReplicationActionTests extends ESTestCase {
 
             setState(clusterService,
                 ClusterState.builder(clusterService.state()).blocks(ClusterBlocks.builder()
-                    .addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ALL)
+                    .addGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_ALL)
                     .addIndexBlock(indexName, IndexMetaData.INDEX_WRITE_BLOCK)));
 
             try (MockNioTransport transport = new MockNioTransport(Settings.EMPTY, Version.CURRENT, threadPool,

+ 3 - 3
server/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java

@@ -24,6 +24,7 @@ import org.elasticsearch.cluster.block.ClusterBlock;
 import org.elasticsearch.cluster.block.ClusterBlocks;
 import org.elasticsearch.cluster.coordination.CoordinationMetaData;
 import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.AliasMetaData;
 import org.elasticsearch.cluster.metadata.IndexGraveyard;
 import org.elasticsearch.cluster.metadata.IndexGraveyardTests;
@@ -50,7 +51,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.gateway.GatewayService;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.query.QueryBuilders;
@@ -347,9 +347,9 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
     private ClusterBlock randomGlobalBlock() {
         switch (randomInt(2)) {
             case 0:
-                return DiscoverySettings.NO_MASTER_BLOCK_ALL;
+                return NoMasterBlockService.NO_MASTER_BLOCK_ALL;
             case 1:
-                return DiscoverySettings.NO_MASTER_BLOCK_WRITES;
+                return NoMasterBlockService.NO_MASTER_BLOCK_WRITES;
             default:
                 return GatewayService.STATE_NOT_RECOVERED_BLOCK;
         }

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

@@ -26,12 +26,12 @@ import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExc
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -82,7 +82,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
 
         logger.info("--> should be blocked, no master...");
         ClusterState state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(true));
         assertThat(state.nodes().getSize(), equalTo(1)); // verify that we still see the local node in the cluster state
 
         logger.info("--> start second node, cluster should be formed");
@@ -93,9 +93,9 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
         assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
 
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(false));
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(false));
 
         state = client().admin().cluster().prepareState().execute().actionGet().getState();
         assertThat(state.nodes().getSize(), equalTo(2));
@@ -128,10 +128,10 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
 
         awaitBusy(() -> {
             ClusterState clusterState = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-            return clusterState.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID);
+            return clusterState.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID);
         });
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(true));
         // verify that both nodes are still in the cluster state but there is no master
         assertThat(state.nodes().getSize(), equalTo(2));
         assertThat(state.nodes().getMasterNode(), equalTo(null));
@@ -144,9 +144,9 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
         assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
 
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(false));
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(false));
 
         state = client().admin().cluster().prepareState().execute().actionGet().getState();
         assertThat(state.nodes().getSize(), equalTo(2));
@@ -173,7 +173,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
 
         assertBusy(() -> {
             ClusterState state1 = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-            assertThat(state1.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
+            assertThat(state1.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(true));
         });
 
         logger.info("--> starting the previous master node again...");
@@ -185,9 +185,9 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
         assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
 
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(false));
         state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-        assertThat(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
+        assertThat(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(false));
 
         state = client().admin().cluster().prepareState().execute().actionGet().getState();
         assertThat(state.nodes().getSize(), equalTo(2));
@@ -217,7 +217,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
         assertBusy(() -> {
             for (Client client : clients()) {
                 ClusterState state1 = client.admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-                assertThat(state1.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
+                assertThat(state1.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(true));
             }
         });
 
@@ -258,7 +258,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
         // spin here to wait till the state is set
         assertBusy(() -> {
             ClusterState st = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
-            assertThat(st.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
+            assertThat(st.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID), equalTo(true));
         });
 
         logger.info("--> start back the 2 nodes ");

+ 5 - 5
server/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java

@@ -28,11 +28,11 @@ import org.elasticsearch.action.support.AutoCreateIndex;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.client.Requests;
 import org.elasticsearch.cluster.block.ClusterBlockException;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.MasterNotDiscoveredException;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.rest.RestStatus;
@@ -73,7 +73,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
     public void testNoMasterActions() throws Exception {
         Settings settings = Settings.builder()
             .put(AutoCreateIndex.AUTO_CREATE_INDEX_SETTING.getKey(), true)
-            .put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all")
+            .put(NoMasterBlockService.NO_MASTER_BLOCK_SETTING.getKey(), "all")
             .build();
 
         final TimeValue timeout = TimeValue.timeValueMillis(10);
@@ -93,7 +93,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
         assertBusy(() -> {
             ClusterState state = clientToMasterlessNode.admin().cluster().prepareState().setLocal(true)
                 .execute().actionGet().getState();
-            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+            assertTrue(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID));
         });
 
         assertThrows(clientToMasterlessNode.prepareGet("test", "type1", "1"),
@@ -195,7 +195,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
     public void testNoMasterActionsWriteMasterBlock() throws Exception {
         Settings settings = Settings.builder()
             .put(AutoCreateIndex.AUTO_CREATE_INDEX_SETTING.getKey(), false)
-            .put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "write")
+            .put(NoMasterBlockService.NO_MASTER_BLOCK_SETTING.getKey(), "write")
             .build();
 
         final List<String> nodes = internalCluster().startNodes(3, settings);
@@ -223,7 +223,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
 
         assertTrue(awaitBusy(() -> {
                 ClusterState state = clientToMasterlessNode.admin().cluster().prepareState().setLocal(true).get().getState();
-                return state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID);
+                return state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID);
             }
         ));
 

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

@@ -120,10 +120,10 @@ import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_
 import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING;
 import static org.elasticsearch.cluster.coordination.LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING;
 import static org.elasticsearch.cluster.coordination.Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION;
-import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_ALL;
-import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_ID;
-import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_SETTING;
-import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_WRITES;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ALL;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ID;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_SETTING;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_WRITES;
 import static org.elasticsearch.discovery.PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING;
 import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
 import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR;

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

@@ -154,7 +154,7 @@ public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
         assertBusy(() -> {
             ClusterState state = client().admin().cluster().prepareState().setLocal(true)
                     .execute().actionGet().getState();
-            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+            assertTrue(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID));
         });
 
         internalCluster().stopRandomDataNode();
@@ -171,7 +171,7 @@ public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
         assertBusy(() -> {
             ClusterState state = client().admin().cluster().prepareState().setLocal(true)
                     .execute().actionGet().getState();
-            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+            assertTrue(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID));
         });
 
         internalCluster().stopRandomDataNode();
@@ -278,7 +278,7 @@ public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
         assertBusy(() -> {
             ClusterState state = internalCluster().client(dataNode).admin().cluster().prepareState().setLocal(true)
                     .execute().actionGet().getState();
-            assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+            assertTrue(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID));
         });
 
         logger.info("--> try to unsafely bootstrap 1st master-eligible node, while node lock is held");
@@ -310,7 +310,7 @@ public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
         assertBusy(() -> {
             ClusterState state = internalCluster().client(dataNode2).admin().cluster().prepareState().setLocal(true)
                     .execute().actionGet().getState();
-            assertFalse(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+            assertFalse(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID));
             assertTrue(state.metaData().persistentSettings().getAsBoolean(UnsafeBootstrapMasterCommand.UNSAFE_BOOTSTRAP.getKey(), false));
         });
 
@@ -386,7 +386,7 @@ public class ElasticsearchNodeCommandIT extends ESIntegTestCase {
 
         ClusterState state = internalCluster().client().admin().cluster().prepareState().setLocal(true)
                 .execute().actionGet().getState();
-        assertTrue(state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID));
+        assertTrue(state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID));
 
         internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node));
     }

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

@@ -0,0 +1,101 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.elasticsearch.cluster.coordination;
+
+import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.test.ESTestCase;
+
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ALL;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_SETTING;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_WRITES;
+import static org.elasticsearch.cluster.coordination.NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING;
+import static org.elasticsearch.common.settings.ClusterSettings.BUILT_IN_CLUSTER_SETTINGS;
+import static org.hamcrest.Matchers.sameInstance;
+
+public class NoMasterBlockServiceTests extends ESTestCase {
+
+    private NoMasterBlockService noMasterBlockService;
+    private ClusterSettings clusterSettings;
+
+    private void createService(Settings settings) {
+        clusterSettings = new ClusterSettings(settings, BUILT_IN_CLUSTER_SETTINGS);
+        noMasterBlockService = new NoMasterBlockService(settings, clusterSettings);
+    }
+
+    private void assertDeprecatedWarningEmitted() {
+        assertWarnings("[discovery.zen.no_master_block] setting was deprecated in Elasticsearch and will be removed in a future release! " +
+            "See the breaking changes documentation for the next major version.");
+    }
+
+    public void testBlocksWritesByDefault() {
+        createService(Settings.EMPTY);
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES));
+    }
+
+    public void testIgnoresLegacySettingBlockingWrites() {
+        createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES));
+        assertDeprecatedWarningEmitted();
+    }
+
+    public void testBlocksWritesIfConfiguredBySetting() {
+        createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "write").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES));
+    }
+
+    public void testIgnoresLegacySettingBlockingAll() {
+        createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "all").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES));
+        assertDeprecatedWarningEmitted();
+    }
+
+    public void testBlocksAllIfConfiguredBySetting() {
+        createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL));
+    }
+
+    public void testRejectsInvalidSetting() {
+        expectThrows(IllegalArgumentException.class, () ->
+            createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "unknown").build()));
+    }
+
+    public void testRejectsInvalidLegacySetting() {
+        expectThrows(IllegalArgumentException.class, () ->
+            createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "unknown").build()));
+        assertDeprecatedWarningEmitted();
+    }
+
+    public void testSettingCanBeUpdated() {
+        createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL));
+
+        clusterSettings.applySettings(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "write").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES));
+    }
+
+    public void testIgnoresUpdatesToLegacySetting() {
+        createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL));
+
+        clusterSettings.applySettings(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build());
+        assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL));
+        assertDeprecatedWarningEmitted();
+    }
+}

+ 2 - 2
server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java

@@ -28,6 +28,7 @@ import org.elasticsearch.cluster.ClusterStateObserver;
 import org.elasticsearch.cluster.LocalNodeMasterListener;
 import org.elasticsearch.cluster.NodeConnectionsService;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
@@ -37,7 +38,6 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.MockLogAppender;
 import org.elasticsearch.test.junit.annotations.TestLogging;
@@ -312,7 +312,7 @@ public class ClusterApplierServiceTests extends ESTestCase {
 
         nodes = state.nodes();
         nodesBuilder = DiscoveryNodes.builder(nodes).masterNodeId(null);
-        state = ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_WRITES))
+        state = ClusterState.builder(state).blocks(ClusterBlocks.builder().addGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_WRITES))
             .nodes(nodesBuilder).build();
         setState(timedClusterApplierService, state);
         assertThat(isMaster.get(), is(false));

+ 5 - 4
server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java

@@ -26,6 +26,7 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder;
 import org.elasticsearch.action.bulk.BulkResponse;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateUpdateTask;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
@@ -331,7 +332,7 @@ public class MasterDisruptionIT extends AbstractDisruptionTestCase {
         // continuously ping until network failures have been resolved. However
         // It may a take a bit before the node detects it has been cut off from the elected master
         logger.info("waiting for isolated node [{}] to have no master", isolatedNode);
-        assertNoMaster(isolatedNode, DiscoverySettings.NO_MASTER_BLOCK_WRITES, TimeValue.timeValueSeconds(10));
+        assertNoMaster(isolatedNode, NoMasterBlockService.NO_MASTER_BLOCK_WRITES, TimeValue.timeValueSeconds(10));
 
 
         logger.info("wait until elected master has been removed and a new 2 node cluster was from (via [{}])", isolatedNode);
@@ -358,9 +359,9 @@ public class MasterDisruptionIT extends AbstractDisruptionTestCase {
         // Wait until the master node sees al 3 nodes again.
         ensureStableCluster(3, new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + networkDisruption.expectedTimeToHeal().millis()));
 
-        logger.info("Verify no master block with {} set to {}", DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all");
+        logger.info("Verify no master block with {} set to {}", NoMasterBlockService.NO_MASTER_BLOCK_SETTING.getKey(), "all");
         client().admin().cluster().prepareUpdateSettings()
-                .setTransientSettings(Settings.builder().put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all"))
+                .setTransientSettings(Settings.builder().put(NoMasterBlockService.NO_MASTER_BLOCK_SETTING.getKey(), "all"))
                 .get();
 
         networkDisruption.startDisrupting();
@@ -370,7 +371,7 @@ public class MasterDisruptionIT extends AbstractDisruptionTestCase {
         // continuously ping until network failures have been resolved. However
         // It may a take a bit before the node detects it has been cut off from the elected master
         logger.info("waiting for isolated node [{}] to have no master", isolatedNode);
-        assertNoMaster(isolatedNode, DiscoverySettings.NO_MASTER_BLOCK_ALL, TimeValue.timeValueSeconds(10));
+        assertNoMaster(isolatedNode, NoMasterBlockService.NO_MASTER_BLOCK_ALL, TimeValue.timeValueSeconds(10));
 
         // make sure we have stable cluster & cross partition recoveries are canceled by the removal of the missing node
         // the unresponsive partition causes recoveries to only time out after 15m (default) and these will cause

+ 6 - 6
server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java

@@ -34,6 +34,7 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.action.shard.ShardStateAction;
 import org.elasticsearch.cluster.block.ClusterBlock;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -46,7 +47,6 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.set.Sets;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.seqno.RetentionLeaseSyncer;
 import org.elasticsearch.index.shard.PrimaryReplicaSyncer;
@@ -293,19 +293,19 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice
                                                    Map<DiscoveryNode, IndicesClusterStateService> clusterStateServiceMap,
                                                    Supplier<MockIndicesService> indicesServiceSupplier) {
         // randomly remove no_master blocks
-        if (randomBoolean() && state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID)) {
+        if (randomBoolean() && state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID)) {
             state = ClusterState.builder(state).blocks(
-                ClusterBlocks.builder().blocks(state.blocks()).removeGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID)).build();
+                ClusterBlocks.builder().blocks(state.blocks()).removeGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_ID)).build();
         }
 
         // randomly add no_master blocks
-        if (rarely() && state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID) == false) {
-            ClusterBlock block = randomBoolean() ? DiscoverySettings.NO_MASTER_BLOCK_ALL : DiscoverySettings.NO_MASTER_BLOCK_WRITES;
+        if (rarely() && state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID) == false) {
+            ClusterBlock block = randomBoolean() ? NoMasterBlockService.NO_MASTER_BLOCK_ALL : NoMasterBlockService.NO_MASTER_BLOCK_WRITES;
             state = ClusterState.builder(state).blocks(ClusterBlocks.builder().blocks(state.blocks()).addGlobalBlock(block)).build();
         }
 
         // if no_master block is in place, make no other cluster state changes
-        if (state.blocks().hasGlobalBlockWithId(DiscoverySettings.NO_MASTER_BLOCK_ID)) {
+        if (state.blocks().hasGlobalBlockWithId(NoMasterBlockService.NO_MASTER_BLOCK_ID)) {
             return state;
         }
 

+ 2 - 2
test/framework/src/main/java/org/elasticsearch/test/RandomObjects.java

@@ -25,6 +25,7 @@ import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
 import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo.Failure;
 import org.elasticsearch.cluster.block.ClusterBlockException;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.collect.Tuple;
@@ -33,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.shard.IndexShardRecoveringException;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.shard.ShardNotFoundException;
@@ -310,7 +310,7 @@ public final class RandomObjects {
         int type = randomIntBetween(random, 0, 3);
         switch (type) {
             case 0:
-                actualException = new ClusterBlockException(singleton(DiscoverySettings.NO_MASTER_BLOCK_WRITES));
+                actualException = new ClusterBlockException(singleton(NoMasterBlockService.NO_MASTER_BLOCK_WRITES));
                 expectedException = new ElasticsearchException("Elasticsearch exception [type=cluster_block_exception, " +
                         "reason=blocked by: [SERVICE_UNAVAILABLE/2/no master];]");
                 break;

+ 2 - 2
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/TransportMonitoringBulkActionTests.java

@@ -14,6 +14,7 @@ import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
@@ -24,7 +25,6 @@ import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskAwareRequest;
@@ -110,7 +110,7 @@ public class TransportMonitoringBulkActionTests extends ESTestCase {
     }
 
     public void testExecuteWithGlobalBlock() throws Exception {
-        final ClusterBlocks.Builder clusterBlock = ClusterBlocks.builder().addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ALL);
+        final ClusterBlocks.Builder clusterBlock = ClusterBlocks.builder().addGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_ALL);
         when(clusterService.state()).thenReturn(ClusterState.builder(ClusterName.DEFAULT).blocks(clusterBlock).build());
 
         final TransportMonitoringBulkAction action = new TransportMonitoringBulkAction(threadPool, clusterService,

+ 2 - 2
x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherIndexingListenerTests.java

@@ -11,6 +11,7 @@ import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.AliasMetaData;
 import org.elasticsearch.cluster.metadata.AliasOrIndex;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
@@ -26,7 +27,6 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.TestShardRouting;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.shard.ShardId;
@@ -670,7 +670,7 @@ public class WatcherIndexingListenerTests extends ESTestCase {
     public void testThatIndexingListenerBecomesInactiveOnClusterBlock() {
         ClusterState clusterState = mockClusterState(Watch.INDEX);
         ClusterState clusterStateWriteBlock = mockClusterState(Watch.INDEX);
-        ClusterBlocks clusterBlocks = ClusterBlocks.builder().addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_WRITES).build();
+        ClusterBlocks clusterBlocks = ClusterBlocks.builder().addGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_WRITES).build();
         when(clusterStateWriteBlock.getBlocks()).thenReturn(clusterBlocks);
 
         assertThat(listener.getConfiguration(), is(not(INACTIVE)));

+ 2 - 2
x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java

@@ -12,6 +12,7 @@ import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateUpdateTask;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.coordination.NoMasterBlockService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
@@ -24,7 +25,6 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.TestShardRouting;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.gateway.GatewayService;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.shard.ShardId;
@@ -458,7 +458,7 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
             .masterNodeId("node_1")
             .add(newNode("node_1"))
             .build();
-        ClusterBlocks clusterBlocks = ClusterBlocks.builder().addGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_WRITES).build();
+        ClusterBlocks clusterBlocks = ClusterBlocks.builder().addGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_WRITES).build();
         ClusterState state = ClusterState.builder(new ClusterName("my-cluster")).nodes(nodes).blocks(clusterBlocks).build();
         lifeCycleService.clusterChanged(new ClusterChangedEvent("any", state, state));
         verify(watcherService, times(1)).pauseExecution(eq("write level cluster block"));