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:
parent
2d114a02ff
commit
3b2a0d7959
|
@ -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.
|
||||
|
|
|
@ -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`::
|
||||
|
|
|
@ -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")),
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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,
|
||||
|
|
|
@ -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 + "]");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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 ");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
));
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)));
|
||||
|
|
|
@ -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"));
|
||||
|
|
Loading…
Reference in New Issue