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.
This commit is contained in:
David Turner 2019-02-05 08:47:56 +00:00 committed by GitHub
parent 2d114a02ff
commit 3b2a0d7959
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 268 additions and 112 deletions

View File

@ -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.

View File

@ -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`::

View File

@ -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")),

View File

@ -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 {

View File

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

View File

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

View File

@ -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,

View File

@ -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 + "]");
}
}
}

View File

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

View File

@ -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;

View File

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

View File

@ -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,

View File

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

View File

@ -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 ");

View File

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

View File

@ -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;

View File

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

View File

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

View File

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

View File

@ -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

View File

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

View File

@ -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;

View File

@ -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,

View File

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

View File

@ -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"));