From aa905b7e52bc613e72622c8e19133d2251881cb8 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 4 Feb 2019 10:16:20 +0000 Subject: [PATCH 1/2] Rename no-master-block setting Replaces `discovery.zen.no_master_block` with `cluster.no_master_block`. Any value set for the old setting is now ignored. --- .../migration/migrate_7_0/discovery.asciidoc | 8 ++ .../discovery/discovery-settings.asciidoc | 8 +- .../index/rankeval/RankEvalResponseTests.java | 4 +- .../cluster/coordination/Coordinator.java | 11 +- .../coordination/JoinTaskExecutor.java | 3 +- .../coordination/NoMasterBlockService.java | 73 +++++++++++++ .../common/settings/ClusterSettings.java | 4 +- .../discovery/DiscoverySettings.java | 34 ------ .../discovery/zen/ZenDiscovery.java | 13 ++- .../ElasticsearchExceptionTests.java | 6 +- .../ExceptionSerializationTests.java | 6 +- ...TransportResyncReplicationActionTests.java | 4 +- .../cluster/ClusterStateDiffIT.java | 6 +- .../cluster/MinimumMasterNodesIT.java | 26 ++--- .../elasticsearch/cluster/NoMasterNodeIT.java | 10 +- .../coordination/CoordinatorTests.java | 8 +- .../ElasticsearchNodeCommandIT.java | 10 +- .../NoMasterBlockServiceTests.java | 101 ++++++++++++++++++ .../service/ClusterApplierServiceTests.java | 4 +- .../discovery/MasterDisruptionIT.java | 9 +- ...ClusterStateServiceRandomUpdatesTests.java | 12 +-- .../org/elasticsearch/test/RandomObjects.java | 4 +- .../TransportMonitoringBulkActionTests.java | 4 +- .../watcher/WatcherIndexingListenerTests.java | 4 +- .../watcher/WatcherLifeCycleServiceTests.java | 4 +- 25 files changed, 266 insertions(+), 110 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java diff --git a/docs/reference/migration/migrate_7_0/discovery.asciidoc b/docs/reference/migration/migrate_7_0/discovery.asciidoc index 193f6bdd86a6f..63dc384f3caf0 100644 --- a/docs/reference/migration/migrate_7_0/discovery.asciidoc +++ b/docs/reference/migration/migrate_7_0/discovery.asciidoc @@ -38,3 +38,11 @@ file: - `discovery.zen.ping.unicast.hosts` - `discovery.zen.hosts_provider` - `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. diff --git a/docs/reference/modules/discovery/discovery-settings.asciidoc b/docs/reference/modules/discovery/discovery-settings.asciidoc index 494c5ac225b87..f68d9a957f57a 100644 --- a/docs/reference/modules/discovery/discovery-settings.asciidoc +++ b/docs/reference/modules/discovery/discovery-settings.asciidoc @@ -133,7 +133,7 @@ Discovery and cluster formation are affected by the following settings: the list of seed nodes. By default, it is the <>. -[[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: + @@ -149,12 +149,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.zen.ping.unicast.hosts`:: diff --git a/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java b/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java index 070b2439e53cb..4c50ba44c990e 100644 --- a/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java +++ b/modules/rank-eval/src/test/java/org/elasticsearch/index/rankeval/RankEvalResponseTests.java @@ -24,6 +24,7 @@ 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.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 @@ 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")), diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index f6e2e1e958884..db094638ae6bd 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -58,7 +58,6 @@ 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.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 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 Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe 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 @@ protected void doStart() { 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 @@ private ClusterState clusterStateWithNoMasterBlock(ClusterState clusterState) { 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 { diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java index 2dcc1022f8d46..a360ea1ab60b8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java @@ -30,7 +30,6 @@ 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 @@ protected ClusterState.Builder becomeMasterAndTrimConflictingNodes(ClusterState // 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()); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java b/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java new file mode 100644 index 0000000000000..2944c3bb23283 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/NoMasterBlockService.java @@ -0,0 +1,73 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.cluster.block.ClusterBlock; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Setting.Property; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.RestStatus; + +import java.util.EnumSet; + +public class NoMasterBlockService { + public static final int NO_MASTER_BLOCK_ID = 2; + public static final ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, false, + RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE)); + public static final ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, false, + RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL); + + public static final Setting 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 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; + } +} diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index f0331ad2e30b2..0844e90c0bc19 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -41,6 +41,7 @@ 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 void apply(Settings value, Settings current, Settings previous) { 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, diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java b/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java index 1e99fdc3702df..e1a0c20864ea3 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoverySettings.java @@ -19,27 +19,17 @@ package org.elasticsearch.discovery; -import org.elasticsearch.cluster.block.ClusterBlock; -import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.rest.RestStatus; - -import java.util.EnumSet; /** * Exposes common discovery settings that may be supported by all the different discovery implementations */ public class DiscoverySettings { - public static final int NO_MASTER_BLOCK_ID = 2; - public static final ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, false, - RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL); - public static final ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, false, - RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE)); /** * sets the timeout for a complete publishing cycle, including both sending and committing. the master * will continue to process the next cluster state update after this time has elapsed @@ -56,26 +46,20 @@ public class DiscoverySettings { new Setting<>("discovery.zen.commit_timeout", PUBLISH_TIMEOUT_SETTING::getRaw, (s) -> TimeValue.parseTimeValue(s, TimeValue.timeValueSeconds(30), "discovery.zen.commit_timeout"), Property.Dynamic, Property.NodeScope, Property.Deprecated); - public static final Setting NO_MASTER_BLOCK_SETTING = - new Setting<>("discovery.zen.no_master_block", "write", DiscoverySettings::parseNoMasterBlock, - Property.Dynamic, Property.NodeScope); public static final Setting PUBLISH_DIFF_ENABLE_SETTING = Setting.boolSetting("discovery.zen.publish_diff.enable", true, Property.Dynamic, Property.NodeScope, Property.Deprecated); public static final Setting 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 TimeValue getCommitTimeout() { 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 @@ private void setCommitTimeout(TimeValue commitTimeout) { 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 + "]"); - } - } } diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 9c85217f17804..6d6b9b438f614 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -32,6 +32,7 @@ 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 ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService t 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 @@ protected void doStart() { 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 @@ boolean processNextCommittedClusterState(String reason) { } 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 @@ boolean processNextCommittedClusterState(String reason) { 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 @@ protected void rejoin(String reason) { 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(); diff --git a/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java b/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java index 21ee15d01cf81..98b9479fe901d 100644 --- a/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java +++ b/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java @@ -28,6 +28,7 @@ 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.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; @@ -380,7 +380,7 @@ 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 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 static Tuple randomExceptions() { 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; diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 901a4c3df018a..8ac056aa41c2a 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -34,6 +34,7 @@ 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.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 void testFailedNodeException() throws IOException { } 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()); } diff --git a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java index 0ca4be52504da..ed3663ed18d2c 100644 --- a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java @@ -25,6 +25,7 @@ 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.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 void testResyncDoesNotBlockOnPrimaryAction() throws Exception { 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, diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java b/server/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java index d795971ac125e..313bf1d47c771 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java @@ -24,6 +24,7 @@ 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.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 @@ private ClusterState.Builder randomBlocks(ClusterState clusterState) { 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; } diff --git a/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java b/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java index 03fa2407685d8..ad1a0516e4430 100644 --- a/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java @@ -26,12 +26,12 @@ 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 void testTwoNodesNoMasterBlock() throws Exception { 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 void testTwoNodesNoMasterBlock() throws Exception { 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 void testTwoNodesNoMasterBlock() throws Exception { 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 void testTwoNodesNoMasterBlock() throws Exception { 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 void testTwoNodesNoMasterBlock() throws Exception { 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 void testTwoNodesNoMasterBlock() throws Exception { 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 void testThreeNodesNoMasterBlock() throws Exception { 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 void testThreeNodesNoMasterBlock() throws Exception { // 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 "); diff --git a/server/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java b/server/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java index 60c3bbee87a7a..15b5b72205392 100644 --- a/server/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java @@ -28,11 +28,11 @@ 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 @@ protected Collection> nodePlugins() { 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 void testNoMasterActions() throws Exception { 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 @@ void checkWriteAction(ActionRequestBuilder builder) { 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 nodes = internalCluster().startNodes(3, settings); @@ -223,7 +223,7 @@ public void testNoMasterActionsWriteMasterBlock() throws Exception { 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); } )); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 333e6c5a3e7b1..6b344c93e1c7d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -120,10 +120,10 @@ 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; diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java index 8ff8cde653d16..df0419bb17378 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ElasticsearchNodeCommandIT.java @@ -154,7 +154,7 @@ public void testBootstrapNotBootstrappedCluster() throws Exception { 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 void testDetachNotBootstrappedCluster() throws Exception { 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 void test3MasterNodes2Failed() throws Exception { 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 void test3MasterNodes2Failed() throws Exception { 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 void testNoInitialBootstrapAfterDetach() throws Exception { 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)); } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java new file mode 100644 index 0000000000000..990fd8a01469d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NoMasterBlockServiceTests.java @@ -0,0 +1,101 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.cluster.coordination; + +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; + +import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ALL; +import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_SETTING; +import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_WRITES; +import static org.elasticsearch.cluster.coordination.NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING; +import static org.elasticsearch.common.settings.ClusterSettings.BUILT_IN_CLUSTER_SETTINGS; +import static org.hamcrest.Matchers.sameInstance; + +public class NoMasterBlockServiceTests extends ESTestCase { + + private NoMasterBlockService noMasterBlockService; + private ClusterSettings clusterSettings; + + private void createService(Settings settings) { + clusterSettings = new ClusterSettings(settings, BUILT_IN_CLUSTER_SETTINGS); + noMasterBlockService = new NoMasterBlockService(settings, clusterSettings); + } + + private void assertDeprecatedWarningEmitted() { + assertWarnings("[discovery.zen.no_master_block] setting was deprecated in Elasticsearch and will be removed in a future release! " + + "See the breaking changes documentation for the next major version."); + } + + public void testBlocksWritesByDefault() { + createService(Settings.EMPTY); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); + } + + public void testIgnoresLegacySettingBlockingWrites() { + createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); + assertDeprecatedWarningEmitted(); + } + + public void testBlocksWritesIfConfiguredBySetting() { + createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); + } + + public void testIgnoresLegacySettingBlockingAll() { + createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); + assertDeprecatedWarningEmitted(); + } + + public void testBlocksAllIfConfiguredBySetting() { + createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); + } + + public void testRejectsInvalidSetting() { + expectThrows(IllegalArgumentException.class, () -> + createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "unknown").build())); + } + + public void testRejectsInvalidLegacySetting() { + expectThrows(IllegalArgumentException.class, () -> + createService(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "unknown").build())); + assertDeprecatedWarningEmitted(); + } + + public void testSettingCanBeUpdated() { + createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); + + clusterSettings.applySettings(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_WRITES)); + } + + public void testIgnoresUpdatesToLegacySetting() { + createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); + + clusterSettings.applySettings(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build()); + assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL)); + assertDeprecatedWarningEmitted(); + } +} diff --git a/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java index 770ae68e1285f..c2b88979ee18c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java @@ -28,6 +28,7 @@ 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.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 String executorName() { 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)); diff --git a/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java index fc9450e982636..4ac924bb25569 100644 --- a/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java @@ -26,6 +26,7 @@ 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 void testVerifyApiBlocksDuringPartition() throws Exception { // 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 void testVerifyApiBlocksDuringPartition() throws Exception { // 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 void testVerifyApiBlocksDuringPartition() throws Exception { // 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 diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index e664cc87452fc..a1ad5ef3c97d4 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -34,6 +34,7 @@ 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.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.shard.PrimaryReplicaSyncer; import org.elasticsearch.index.shard.ShardId; @@ -292,19 +292,19 @@ public ClusterState randomlyUpdateClusterState(ClusterState state, Map clusterStateServiceMap, Supplier 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; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/RandomObjects.java b/test/framework/src/main/java/org/elasticsearch/test/RandomObjects.java index 4669284685c11..2af37bc238bdb 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/RandomObjects.java +++ b/test/framework/src/main/java/org/elasticsearch/test/RandomObjects.java @@ -25,6 +25,7 @@ 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.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 @@ private static Tuple randomShardInfoFailure(Random random) { 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; diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/TransportMonitoringBulkActionTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/TransportMonitoringBulkActionTests.java index bf6c77d30ea70..9df959ead1a45 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/TransportMonitoringBulkActionTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/TransportMonitoringBulkActionTests.java @@ -14,6 +14,7 @@ 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.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 void setUpMocks() { } 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, diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherIndexingListenerTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherIndexingListenerTests.java index 7a37f71e22f28..db8e3ffdc914e 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherIndexingListenerTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherIndexingListenerTests.java @@ -11,6 +11,7 @@ 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.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 void testThatIndexingListenerBecomesInactiveWithoutMasterNode() { 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))); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java index 467966e96fdbf..19e2a13487123 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java @@ -12,6 +12,7 @@ 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.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 void testWatcherStopsOnClusterLevelBlock() { .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")); From 071235cf936486f77da752e6aed2c394c7663eda Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 4 Feb 2019 18:00:44 +0000 Subject: [PATCH 2/2] Line length --- .../java/org/elasticsearch/ElasticsearchExceptionTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java b/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java index 98b9479fe901d..2706e14a361e1 100644 --- a/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java +++ b/server/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java @@ -378,9 +378,9 @@ public void testGenerateThrowableToXContent() throws IOException { public void testToXContentWithHeadersAndMetadata() throws IOException { ElasticsearchException e = new ElasticsearchException("foo", - new ElasticsearchException("bar", - new ElasticsearchException("baz", - new ClusterBlockException(singleton(NoMasterBlockService.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");