[Remove] Deprecated Zen1 Discovery (#1216)

Zen1 discovery was deprecated in Legacy 7.x for eventual removal. OpenSearch 1.x
carries this deprecation. This commit completely removes all support for Zen1
discovery in favor of Zen2.

Signed-off-by: Nicholas Walter Knize <nknize@apache.org>
This commit is contained in:
Nick Knize 2021-12-03 16:27:10 -05:00 committed by GitHub
parent 70f07872f8
commit 4db97aa470
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
61 changed files with 52 additions and 11103 deletions

View File

@ -39,13 +39,13 @@ import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.DiscoverySettings;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.env.TestEnvironment;
import org.opensearch.gateway.GatewayMetaState;
import org.opensearch.gateway.PersistedClusterStateService;
import org.opensearch.indices.IndicesService;
import org.opensearch.node.Node.DiscoverySettings;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;

View File

@ -1,474 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.cluster.coordination;
import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction;
import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest;
import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction;
import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsRequest;
import org.opensearch.action.admin.cluster.health.ClusterHealthRequest;
import org.opensearch.action.admin.cluster.health.ClusterHealthRequestBuilder;
import org.opensearch.action.admin.cluster.health.ClusterHealthResponse;
import org.opensearch.action.admin.indices.create.CreateIndexRequest;
import org.opensearch.client.Client;
import org.opensearch.client.Requests;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.metadata.Manifest;
import org.opensearch.cluster.routing.UnassignedInfo;
import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.Allocation;
import org.opensearch.common.Priority;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.discovery.Discovery;
import org.opensearch.discovery.DiscoveryModule;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.gateway.MetaStateService;
import org.opensearch.indices.IndicesService;
import org.opensearch.plugins.Plugin;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.InternalTestCluster.RestartCallback;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.transport.TransportService;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.StreamSupport;
import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
import static org.opensearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM;
import static org.opensearch.cluster.coordination.FollowersChecker.FOLLOWER_CHECK_ACTION_NAME;
import static org.opensearch.cluster.coordination.JoinHelper.START_JOIN_ACTION_NAME;
import static org.opensearch.cluster.coordination.PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME;
import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING;
import static org.opensearch.cluster.routing.allocation.decider.FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING;
import static org.opensearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
import static org.opensearch.node.Node.NODE_NAME_SETTING;
import static org.opensearch.test.InternalTestCluster.REMOVED_MINIMUM_MASTER_NODES;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class Zen1IT extends OpenSearchIntegTestCase {
private static Settings ZEN1_SETTINGS = Coordinator.addZen1Attribute(
true,
Settings.builder()
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), DiscoveryModule.ZEN_DISCOVERY_TYPE)
.put(IndicesService.WRITE_DANGLING_INDICES_INFO_SETTING.getKey(), false)
).build();
private static Settings ZEN2_SETTINGS = Settings.builder()
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), DiscoveryModule.ZEN2_DISCOVERY_TYPE)
.build();
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Collections.singletonList(MockTransportService.TestPlugin.class);
}
@Override
protected void ensureClusterStateConsistency() {
// Zen1 does not properly handle the cluster_uuid_committed field
}
public void testZen2NodesJoiningZen1Cluster() {
internalCluster().startNodes(randomIntBetween(1, 3), ZEN1_SETTINGS);
internalCluster().startNodes(randomIntBetween(1, 3), ZEN2_SETTINGS);
createIndex("test");
}
public void testZen1NodesJoiningZen2Cluster() {
internalCluster().startNodes(randomIntBetween(1, 3), ZEN2_SETTINGS);
internalCluster().startNodes(randomIntBetween(1, 3), ZEN1_SETTINGS);
createIndex("test");
}
public void testMixedClusterDisruption() throws Exception {
final List<String> nodes = internalCluster().startNodes(
IntStream.range(0, 5).mapToObj(i -> i < 2 ? ZEN1_SETTINGS : ZEN2_SETTINGS).toArray(Settings[]::new)
);
final List<MockTransportService> transportServices = nodes.stream()
.map(n -> (MockTransportService) internalCluster().getInstance(TransportService.class, n))
.collect(Collectors.toList());
logger.info("--> disrupting communications");
// The idea here is to make some of the Zen2 nodes believe the Zen1 nodes have gone away by introducing a network partition, so that
// they bootstrap themselves, but keep the Zen1 side of the cluster alive.
// Set up a bridged network partition with the Zen1 nodes {0,1} on one side, Zen2 nodes {3,4} on the other, and node {2} in both
transportServices.get(0).addFailToSendNoConnectRule(transportServices.get(3));
transportServices.get(0).addFailToSendNoConnectRule(transportServices.get(4));
transportServices.get(1).addFailToSendNoConnectRule(transportServices.get(3));
transportServices.get(1).addFailToSendNoConnectRule(transportServices.get(4));
transportServices.get(3).addFailToSendNoConnectRule(transportServices.get(0));
transportServices.get(3).addFailToSendNoConnectRule(transportServices.get(1));
transportServices.get(4).addFailToSendNoConnectRule(transportServices.get(0));
transportServices.get(4).addFailToSendNoConnectRule(transportServices.get(1));
// Nodes 3 and 4 will bootstrap, but we want to keep node 2 as part of the Zen1 cluster, so prevent any messages that might switch
// its allegiance
transportServices.get(3)
.addFailToSendNoConnectRule(
transportServices.get(2),
PUBLISH_STATE_ACTION_NAME,
FOLLOWER_CHECK_ACTION_NAME,
START_JOIN_ACTION_NAME
);
transportServices.get(4)
.addFailToSendNoConnectRule(
transportServices.get(2),
PUBLISH_STATE_ACTION_NAME,
FOLLOWER_CHECK_ACTION_NAME,
START_JOIN_ACTION_NAME
);
logger.info("--> waiting for disconnected nodes to be removed");
ensureStableCluster(3, nodes.get(0));
logger.info("--> creating index on Zen1 side");
assertAcked(client(nodes.get(0)).admin().indices().create(new CreateIndexRequest("test")).get());
assertFalse(
client(nodes.get(0)).admin().cluster().health(new ClusterHealthRequest("test").waitForGreenStatus()).get().isTimedOut()
);
logger.info("--> waiting for disconnected nodes to bootstrap themselves");
assertBusy(
() -> assertTrue(
IntStream.range(3, 5)
.mapToObj(n -> (Coordinator) internalCluster().getInstance(Discovery.class, nodes.get(n)))
.anyMatch(Coordinator::isInitialConfigurationSet)
)
);
logger.info("--> clearing disruption and waiting for cluster to reform");
transportServices.forEach(MockTransportService::clearAllRules);
ensureStableCluster(5, nodes.get(0));
assertFalse(
client(nodes.get(0)).admin().cluster().health(new ClusterHealthRequest("test").waitForGreenStatus()).get().isTimedOut()
);
}
public void testMixedClusterFormation() throws Exception {
final int zen1NodeCount = randomIntBetween(1, 3);
final int zen2NodeCount = randomIntBetween(zen1NodeCount == 1 ? 2 : 1, 3);
logger.info("starting cluster of [{}] Zen1 nodes and [{}] Zen2 nodes", zen1NodeCount, zen2NodeCount);
final List<String> nodes = internalCluster().startNodes(
IntStream.range(0, zen1NodeCount + zen2NodeCount)
.mapToObj(i -> i < zen1NodeCount ? ZEN1_SETTINGS : ZEN2_SETTINGS)
.toArray(Settings[]::new)
);
createIndex(
"test",
Settings.builder()
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) // assign shards
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, zen1NodeCount + zen2NodeCount + randomIntBetween(0, 2)) // causes rebalancing
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
.build()
);
ensureGreen("test");
for (final String node : nodes) {
// With 1 Zen1 node when you stop the Zen1 node the Zen2 nodes might auto-bootstrap.
// But there are only 2 Zen2 nodes so you must do the right things with voting config exclusions to keep the cluster
// alive through the other two restarts.
final boolean masterNodeIsZen2 = zen1NodeCount <= nodes.indexOf(internalCluster().getMasterName());
final boolean thisNodeIsZen2 = zen1NodeCount <= nodes.indexOf(node);
final boolean requiresVotingConfigExclusions = zen1NodeCount == 1 && zen2NodeCount == 2 && masterNodeIsZen2 && thisNodeIsZen2;
if (requiresVotingConfigExclusions) {
client().execute(AddVotingConfigExclusionsAction.INSTANCE, new AddVotingConfigExclusionsRequest(new String[] { node }))
.get();
}
internalCluster().restartNode(node, new RestartCallback() {
@Override
public Settings onNodeStopped(String restartingNode) {
String viaNode = randomValueOtherThan(restartingNode, () -> randomFrom(nodes));
final ClusterHealthRequestBuilder clusterHealthRequestBuilder = client(viaNode).admin()
.cluster()
.prepareHealth()
.setWaitForEvents(Priority.LANGUID)
.setWaitForNodes(Integer.toString(zen1NodeCount + zen2NodeCount - 1))
.setTimeout(TimeValue.timeValueSeconds(30));
ClusterHealthResponse clusterHealthResponse = clusterHealthRequestBuilder.get();
assertFalse(restartingNode, clusterHealthResponse.isTimedOut());
return Settings.EMPTY;
}
});
ensureStableCluster(zen1NodeCount + zen2NodeCount);
ensureGreen("test");
if (requiresVotingConfigExclusions) {
final ClearVotingConfigExclusionsRequest clearVotingTombstonesRequest = new ClearVotingConfigExclusionsRequest();
clearVotingTombstonesRequest.setWaitForRemoval(false);
client().execute(ClearVotingConfigExclusionsAction.INSTANCE, clearVotingTombstonesRequest).get();
}
}
}
public void testRollingMigrationFromZen1ToZen2() throws Exception {
final int nodeCount = randomIntBetween(2, 5);
final List<String> zen1Nodes = internalCluster().startNodes(nodeCount, ZEN1_SETTINGS);
createIndex(
"test",
Settings.builder()
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) // assign shards
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, nodeCount) // causes rebalancing
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
.build()
);
ensureGreen("test");
for (final String zen1Node : zen1Nodes) {
logger.info("--> shutting down {}", zen1Node);
internalCluster().stopRandomNode(s -> NODE_NAME_SETTING.get(s).equals(zen1Node));
ensureStableCluster(nodeCount - 1);
if (nodeCount > 2) {
ensureGreen("test");
} else {
ensureYellow("test");
}
logger.info("--> starting replacement for {}", zen1Node);
final String newNode = internalCluster().startNode(ZEN2_SETTINGS);
ensureStableCluster(nodeCount);
ensureGreen("test");
logger.info("--> successfully replaced {} with {}", zen1Node, newNode);
}
assertThat(internalCluster().size(), equalTo(nodeCount));
}
public void testRollingUpgradeFromZen1ToZen2() throws Exception {
final int nodeCount = randomIntBetween(2, 5);
final List<String> nodes = internalCluster().startNodes(nodeCount, ZEN1_SETTINGS);
createIndex(
"test",
Settings.builder()
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) // assign shards
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, nodeCount) // causes rebalancing
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
.build()
);
ensureGreen("test");
internalCluster().rollingRestart(new RestartCallback() {
@Override
public void doAfterNodes(int n, Client client) {
ensureGreen("test");
}
@Override
public Settings onNodeStopped(String nodeName) {
String viaNode = randomValueOtherThan(nodeName, () -> randomFrom(nodes));
final ClusterHealthRequestBuilder clusterHealthRequestBuilder = client(viaNode).admin()
.cluster()
.prepareHealth()
.setWaitForEvents(Priority.LANGUID)
.setWaitForNodes(Integer.toString(nodeCount - 1))
.setTimeout(TimeValue.timeValueSeconds(30));
if (nodeCount == 2) {
clusterHealthRequestBuilder.setWaitForYellowStatus();
} else {
clusterHealthRequestBuilder.setWaitForGreenStatus();
}
ClusterHealthResponse clusterHealthResponse = clusterHealthRequestBuilder.get();
assertFalse(nodeName, clusterHealthResponse.isTimedOut());
return Coordinator.addZen1Attribute(
false,
Settings.builder()
.put(ZEN2_SETTINGS)
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), REMOVED_MINIMUM_MASTER_NODES)
).build();
}
});
ensureStableCluster(nodeCount);
ensureGreen("test");
assertThat(internalCluster().size(), equalTo(nodeCount));
}
private void testMultipleNodeMigrationFromZen1ToZen2(int nodeCount) throws Exception {
final List<String> oldNodes = internalCluster().startNodes(nodeCount, ZEN1_SETTINGS);
createIndex(
"test",
Settings.builder()
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) // assign shards
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, nodeCount) // causes rebalancing
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, nodeCount > 1 ? 1 : 0)
.build()
);
ensureGreen("test");
internalCluster().startNodes(nodeCount, ZEN2_SETTINGS);
logger.info("--> updating settings to exclude old nodes");
client().admin()
.cluster()
.prepareUpdateSettings()
.setPersistentSettings(
Settings.builder()
.put(CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING.getConcreteSettingForNamespace("_name").getKey(), String.join(",", oldNodes))
)
.get();
logger.info("--> waiting for old nodes to be vacated");
waitForRelocation();
while (internalCluster().size() > nodeCount) {
internalCluster().stopRandomNode(settings -> oldNodes.contains(NODE_NAME_SETTING.get(settings)));
}
ensureGreen("test");
}
public void testMultipleNodeMigrationFromZen1ToZen2WithOneNode() throws Exception {
testMultipleNodeMigrationFromZen1ToZen2(1);
}
public void testMultipleNodeMigrationFromZen1ToZen2WithTwoNodes() throws Exception {
testMultipleNodeMigrationFromZen1ToZen2(2);
}
public void testMultipleNodeMigrationFromZen1ToZen2WithThreeNodes() throws Exception {
testMultipleNodeMigrationFromZen1ToZen2(3);
}
public void testFreshestMasterElectedAfterFullClusterRestart() throws Exception {
final List<String> nodeNames = internalCluster().startNodes(3, ZEN1_SETTINGS);
// Set setting to a non-default value on all nodes.
assertTrue(
client().admin()
.cluster()
.prepareUpdateSettings()
.setPersistentSettings(Settings.builder().put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NEW_PRIMARIES))
.get()
.isAcknowledged()
);
final List<NodeEnvironment> nodeEnvironments = StreamSupport.stream(
internalCluster().getDataOrMasterNodeInstances(NodeEnvironment.class).spliterator(),
false
).collect(Collectors.toList());
final boolean randomiseVersions = rarely();
internalCluster().fullRestart(new RestartCallback() {
int nodesStopped;
@Override
public Settings onNodeStopped(String nodeName) throws Exception {
nodesStopped += 1;
if (nodesStopped == 1) {
final Client client = internalCluster().client(randomValueOtherThan(nodeName, () -> randomFrom(nodeNames)));
assertFalse(
client.admin()
.cluster()
.health(
Requests.clusterHealthRequest()
.waitForEvents(Priority.LANGUID)
.waitForNoRelocatingShards(true)
.waitForNodes("2")
)
.actionGet()
.isTimedOut()
);
// Set setting to a different non-default value on two of the three remaining nodes.
assertTrue(
client.admin()
.cluster()
.prepareUpdateSettings()
.setPersistentSettings(
Settings.builder().put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NONE)
)
.get()
.isAcknowledged()
);
}
if (nodesStopped == nodeNames.size()) {
for (final NodeEnvironment nodeEnvironment : nodeEnvironments) {
// The versions written by nodes following a Zen1 master cannot be trusted. Randomise them to demonstrate they are
// not important.
final MetaStateService metaStateService = new MetaStateService(nodeEnvironment, xContentRegistry());
final Manifest manifest = metaStateService.loadManifestOrEmpty();
assertThat(manifest.getCurrentTerm(), is(ZEN1_BWC_TERM));
final long newVersion = randomiseVersions ? randomNonNegativeLong() : 0L;
metaStateService.writeManifestAndCleanup(
"altering version to " + newVersion,
new Manifest(
manifest.getCurrentTerm(),
newVersion,
manifest.getGlobalGeneration(),
manifest.getIndexGenerations()
)
);
}
}
return Coordinator.addZen1Attribute(false, Settings.builder())
.put(ZEN2_SETTINGS)
.putList(INITIAL_MASTER_NODES_SETTING.getKey(), nodeNames)
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), REMOVED_MINIMUM_MASTER_NODES)
.build();
}
});
final AtomicReference<ClusterState> clusterState = new AtomicReference<>();
assertBusy(() -> {
clusterState.set(client().admin().cluster().prepareState().get().getState());
assertFalse(clusterState.get().blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK));
});
final Settings clusterSettings = clusterState.get().metadata().settings();
assertTrue(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.exists(clusterSettings));
assertThat(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.get(clusterSettings), equalTo(Allocation.NONE));
}
}

View File

@ -47,7 +47,6 @@ import org.opensearch.common.xcontent.XContentBuilder;
import org.opensearch.common.xcontent.XContentFactory;
import org.opensearch.discovery.Discovery;
import org.opensearch.discovery.DiscoveryStats;
import org.opensearch.discovery.zen.FaultDetection;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.TestCustomMetadata;
import org.opensearch.transport.RemoteTransportException;
@ -73,10 +72,6 @@ import static org.hamcrest.Matchers.notNullValue;
public class ZenDiscoveryIT extends OpenSearchIntegTestCase {
public void testNoShardRelocationsOccurWhenElectedMasterNodeFails() throws Exception {
Settings defaultSettings = Settings.builder()
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s")
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1")
.build();
Settings masterNodeSettings = masterOnlyNode();
internalCluster().startNodes(2, masterNodeSettings);

View File

@ -39,9 +39,6 @@ import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.zen.MembershipAction;
import org.opensearch.discovery.zen.PublishClusterStateAction;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.disruption.NetworkDisruption;
import org.opensearch.test.disruption.ServiceDisruptionScheme;
@ -56,7 +53,6 @@ import java.util.concurrent.CountDownLatch;
import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING;
import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
/**
* Tests for discovery during disruptions.
@ -97,23 +93,15 @@ public class DiscoveryDisruptionIT extends AbstractDisruptionTestCase {
discoveryNodes.getLocalNode().getName()
);
if (randomBoolean()) {
masterTransportService.addFailToSendNoConnectRule(
localTransportService,
PublishClusterStateAction.SEND_ACTION_NAME,
PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME
);
masterTransportService.addFailToSendNoConnectRule(localTransportService, PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME);
} else {
masterTransportService.addFailToSendNoConnectRule(
localTransportService,
PublishClusterStateAction.COMMIT_ACTION_NAME,
PublicationTransportHandler.COMMIT_STATE_ACTION_NAME
);
masterTransportService.addFailToSendNoConnectRule(localTransportService, PublicationTransportHandler.COMMIT_STATE_ACTION_NAME);
}
logger.info("allowing requests from non master [{}] to master [{}], waiting for two join request", nonMasterNode, masterNode);
final CountDownLatch countDownLatch = new CountDownLatch(2);
nonMasterTransportService.addSendBehavior(masterTransportService, (connection, requestId, action, request, options) -> {
if (action.equals(MembershipAction.DISCOVERY_JOIN_ACTION_NAME) || action.equals(JoinHelper.JOIN_ACTION_NAME)) {
if (action.equals(JoinHelper.JOIN_ACTION_NAME)) {
countDownLatch.countDown();
}
connection.sendRequest(requestId, action, request, options);
@ -139,7 +127,7 @@ public class DiscoveryDisruptionIT extends AbstractDisruptionTestCase {
SlowClusterStateProcessing disruption = new SlowClusterStateProcessing(random(), 0, 0, 1000, 2000);
// don't wait for initial state, we want to add the disruption while the cluster is forming
internalCluster().startNodes(3, Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "3s").build());
internalCluster().startNodes(3);
logger.info("applying disruption while cluster is forming ...");
@ -167,14 +155,6 @@ public class DiscoveryDisruptionIT extends AbstractDisruptionTestCase {
ensureStableCluster(3);
final String preferredMasterName = internalCluster().getMasterName();
final DiscoveryNode preferredMaster = internalCluster().clusterService(preferredMasterName).localNode();
final Discovery discovery = internalCluster().getInstance(Discovery.class);
// only Zen1 guarantees that node with lowest id is elected
if (discovery instanceof ZenDiscovery) {
for (String node : nodes) {
DiscoveryNode discoveryNode = internalCluster().clusterService(node).localNode();
assertThat(discoveryNode.getId(), greaterThanOrEqualTo(preferredMaster.getId()));
}
}
logger.info("--> preferred master is {}", preferredMaster);
final Set<String> nonPreferredNodes = new HashSet<>(nodes);

View File

@ -42,7 +42,6 @@ import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.disruption.BlockMasterServiceOnMaster;
import org.opensearch.test.disruption.IntermittentLongGCDisruption;
@ -55,7 +54,6 @@ import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
@ -92,9 +90,6 @@ public class MasterDisruptionIT extends AbstractDisruptionTestCase {
logger.info("waiting for nodes to elect a new master");
ensureStableCluster(2, oldNonMasterNodes.get(0));
logger.info("waiting for any pinging to stop");
assertDiscoveryCompleted(oldNonMasterNodes);
// restore GC
masterNodeDisruption.stopDisrupting();
final TimeValue waitTime = new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + masterNodeDisruption.expectedTimeToHeal().millis());
@ -323,15 +318,4 @@ public class MasterDisruptionIT extends AbstractDisruptionTestCase {
});
}
private void assertDiscoveryCompleted(List<String> nodes) throws Exception {
for (final String node : nodes) {
assertBusy(() -> {
final Discovery discovery = internalCluster().getInstance(Discovery.class, node);
if (discovery instanceof ZenDiscovery) {
assertFalse("node [" + node + "] is still joining master", ((ZenDiscovery) discovery).joiningCluster());
}
}, 30, TimeUnit.SECONDS);
}
}
}

View File

@ -41,7 +41,7 @@ import org.opensearch.cluster.coordination.JoinHelper;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.logging.Loggers;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.DiscoverySettings;
import org.opensearch.node.Node.DiscoverySettings;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.MockHttpTransport;

View File

@ -166,7 +166,6 @@ public class TransportClusterStateAction extends TransportMasterNodeReadAction<C
ClusterState.Builder builder = ClusterState.builder(currentState.getClusterName());
builder.version(currentState.version());
builder.stateUUID(currentState.stateUUID());
builder.minimumMasterNodesOnPublishingMaster(currentState.getMinimumMasterNodesOnPublishingMaster());
if (request.nodes()) {
builder.nodes(currentState.nodes());

View File

@ -762,11 +762,6 @@ final class BootstrapChecks {
static class DiscoveryConfiguredCheck implements BootstrapCheck {
@Override
public BootstrapCheckResult check(BootstrapContext context) {
if (DiscoveryModule.ZEN_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(context.settings()))) {
return BootstrapCheckResult.failure(
String.format(Locale.ROOT, "discovery type [%s] is unsuitable for production use", DiscoveryModule.ZEN_DISCOVERY_TYPE)
);
}
if (DiscoveryModule.ZEN2_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(context.settings())) == false) {
return BootstrapCheckResult.success();

View File

@ -323,23 +323,6 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
return coordinationMetadata().getVotingConfigExclusions();
}
/**
* The node-level `discovery.zen.minimum_master_nodes` setting on the master node that published this cluster state, for use in rolling
* upgrades from 6.x to 7.x. Once all the 6.x master-eligible nodes have left the cluster, the 7.x nodes use this value to determine how
* many master-eligible nodes must be discovered before the cluster can be bootstrapped. Note that this method returns the node-level
* value of this setting, and ignores any cluster-level override that was set via the API. Callers are expected to combine this value
* with any value set in the cluster-level settings. This should be removed once we no longer need support for
* {@link LegacyESVersion#V_6_7_0}.
*/
public int getMinimumMasterNodesOnPublishingMaster() {
return minimumMasterNodesOnPublishingMaster;
}
// Used for testing and logging to determine how this cluster state was send over the wire
public boolean wasReadFromDiff() {
return wasReadFromDiff;
}
/**
* Returns a built (on demand) routing nodes view of the routing table.
*/

View File

@ -82,7 +82,6 @@ import org.opensearch.discovery.HandshakingTransportAddressConnector;
import org.opensearch.discovery.PeerFinder;
import org.opensearch.discovery.SeedHostsProvider;
import org.opensearch.discovery.SeedHostsResolver;
import org.opensearch.discovery.zen.PendingClusterStateStats;
import org.opensearch.monitor.NodeHealthService;
import org.opensearch.monitor.StatusInfo;
import org.opensearch.threadpool.Scheduler;
@ -166,7 +165,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
private long maxTermSeen;
private final Reconfigurator reconfigurator;
private final ClusterBootstrapService clusterBootstrapService;
private final DiscoveryUpgradeService discoveryUpgradeService;
private final LagDetector lagDetector;
private final ClusterFormationFailureHelper clusterFormationFailureHelper;
@ -266,14 +264,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
this::isInitialConfigurationSet,
this::setInitialConfiguration
);
this.discoveryUpgradeService = new DiscoveryUpgradeService(
settings,
transportService,
this::isInitialConfigurationSet,
joinHelper,
peerFinder::getFoundPeers,
this::setInitialConfiguration
);
this.lagDetector = new LagDetector(
settings,
transportService.getThreadPool(),
@ -670,10 +660,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
peerFinder.activate(coordinationState.get().getLastAcceptedState().nodes());
clusterFormationFailureHelper.start();
if (getCurrentTerm() == ZEN1_BWC_TERM) {
discoveryUpgradeService.activate(lastKnownLeader, coordinationState.get().getLastAcceptedState());
}
leaderChecker.setCurrentNodes(DiscoveryNodes.EMPTY_NODES);
leaderChecker.updateLeader(null);
@ -713,7 +699,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
lastKnownLeader = Optional.of(getLocalNode());
peerFinder.deactivate(getLocalNode());
discoveryUpgradeService.deactivate();
clusterFormationFailureHelper.stop();
closePrevotingAndElectionScheduler();
preVoteCollector.update(getPreVoteResponse(), getLocalNode());
@ -751,7 +736,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
lastKnownLeader = Optional.of(leaderNode);
peerFinder.deactivate(leaderNode);
discoveryUpgradeService.deactivate();
clusterFormationFailureHelper.stop();
closePrevotingAndElectionScheduler();
cancelActivePublication("become follower: " + method);

View File

@ -1,403 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.cluster.coordination;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfiguration;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.Nullable;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.CountDown;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.discovery.zen.ElectMasterService.MasterCandidate;
import org.opensearch.discovery.zen.UnicastZenPing;
import org.opensearch.discovery.zen.UnicastZenPing.UnicastPingRequest;
import org.opensearch.discovery.zen.UnicastZenPing.UnicastPingResponse;
import org.opensearch.discovery.zen.ZenPing.PingResponse;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.threadpool.ThreadPool.Names;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponseHandler;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Optional;
import java.util.Set;
import java.util.function.BooleanSupplier;
import java.util.function.Consumer;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static java.lang.Math.max;
import static org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING;
import static org.opensearch.cluster.ClusterState.UNKNOWN_VERSION;
import static org.opensearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet;
import static org.opensearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING;
import static org.opensearch.discovery.zen.ZenDiscovery.PING_TIMEOUT_SETTING;
/**
* Deals with rolling upgrades of the cluster coordination layer. In mixed clusters we prefer to elect the older nodes, but
* when the last old node shuts down then as long as there are enough new nodes we can assume that they form the whole cluster and
* define them as the initial configuration.
*/
public class DiscoveryUpgradeService {
private static Logger logger = LogManager.getLogger(DiscoveryUpgradeService.class);
// how long to wait after activation before attempting to join a master or perform a bootstrap upgrade
public static final Setting<TimeValue> BWC_PING_TIMEOUT_SETTING = Setting.timeSetting(
"discovery.zen.bwc_ping_timeout",
PING_TIMEOUT_SETTING,
TimeValue.timeValueMillis(1),
Setting.Property.NodeScope,
Setting.Property.Deprecated
);
// whether to try and bootstrap all the discovered Zen2 nodes when the last Zen1 node leaves the cluster.
public static final Setting<Boolean> ENABLE_UNSAFE_BOOTSTRAPPING_ON_UPGRADE_SETTING = Setting.boolSetting(
"discovery.zen.unsafe_rolling_upgrades_enabled",
true,
Setting.Property.NodeScope,
Setting.Property.Deprecated
);
/**
* Dummy {@link ElectMasterService} that is only used to choose the best 6.x master from the discovered nodes, ignoring the
* `minimum_master_nodes` setting.
*/
private static final ElectMasterService electMasterService = new ElectMasterService(Settings.EMPTY);
private final TransportService transportService;
private final BooleanSupplier isBootstrappedSupplier;
private final JoinHelper joinHelper;
private final Supplier<Iterable<DiscoveryNode>> peersSupplier;
private final Consumer<VotingConfiguration> initialConfigurationConsumer;
private final TimeValue bwcPingTimeout;
private final boolean enableUnsafeBootstrappingOnUpgrade;
private final ClusterName clusterName;
@Nullable // null if no active joining round
private volatile JoiningRound joiningRound;
public DiscoveryUpgradeService(
Settings settings,
TransportService transportService,
BooleanSupplier isBootstrappedSupplier,
JoinHelper joinHelper,
Supplier<Iterable<DiscoveryNode>> peersSupplier,
Consumer<VotingConfiguration> initialConfigurationConsumer
) {
assert Version.CURRENT.major == 1 || Version.CURRENT.major == 2 : "remove this service once unsafe upgrades are no longer needed";
this.transportService = transportService;
this.isBootstrappedSupplier = isBootstrappedSupplier;
this.joinHelper = joinHelper;
this.peersSupplier = peersSupplier;
this.initialConfigurationConsumer = initialConfigurationConsumer;
this.bwcPingTimeout = BWC_PING_TIMEOUT_SETTING.get(settings);
this.enableUnsafeBootstrappingOnUpgrade = ENABLE_UNSAFE_BOOTSTRAPPING_ON_UPGRADE_SETTING.get(settings);
this.clusterName = CLUSTER_NAME_SETTING.get(settings);
}
public void activate(Optional<DiscoveryNode> lastKnownLeader, ClusterState lastAcceptedClusterState) {
// called under coordinator mutex
if (isBootstrappedSupplier.getAsBoolean()) {
return;
}
assert lastKnownLeader.isPresent() == false || Coordinator.isZen1Node(lastKnownLeader.get()) : lastKnownLeader;
// if there was a leader and it's not a old node then we must have been bootstrapped
final Settings dynamicSettings = lastAcceptedClusterState.metadata().settings();
final int minimumMasterNodes = DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(dynamicSettings)
? DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(dynamicSettings)
: lastAcceptedClusterState.getMinimumMasterNodesOnPublishingMaster();
assert joiningRound == null : joiningRound;
final Set<String> knownMasterNodeIds = new HashSet<>();
lastAcceptedClusterState.nodes().getMasterNodes().forEach(c -> knownMasterNodeIds.add(c.key));
joiningRound = new JoiningRound(
enableUnsafeBootstrappingOnUpgrade && lastKnownLeader.isPresent(),
minimumMasterNodes,
knownMasterNodeIds
);
joiningRound.scheduleNextAttempt();
}
public void deactivate() {
// called under coordinator mutex
joiningRound = null;
}
/**
* Waits for some number of calls to {@link ListenableCountDown#countDown()} and then notifies a listener. The listener
* is only ever notified once, whether successful or not.
*/
private static class ListenableCountDown {
private final CountDown countDown;
private final ActionListener<Void> listener;
ListenableCountDown(int count, ActionListener<Void> listener) {
this.countDown = new CountDown(count);
this.listener = listener;
}
void onFailure(Exception e) {
if (countDown.fastForward()) {
listener.onFailure(e);
}
}
void countDown() {
if (countDown.countDown()) {
listener.onResponse(null);
}
}
}
private class JoiningRound {
private final boolean upgrading;
private final int minimumMasterNodes;
private final Set<String> knownMasterNodeIds;
JoiningRound(boolean upgrading, int minimumMasterNodes, Set<String> knownMasterNodeIds) {
this.upgrading = upgrading;
this.minimumMasterNodes = minimumMasterNodes;
this.knownMasterNodeIds = knownMasterNodeIds;
}
private boolean isRunning() {
return joiningRound == this && isBootstrappedSupplier.getAsBoolean() == false;
}
private boolean canBootstrap(Set<DiscoveryNode> discoveryNodes) {
return upgrading && minimumMasterNodes <= discoveryNodes.stream().filter(DiscoveryNode::isMasterNode).count();
}
void scheduleNextAttempt() {
if (isRunning() == false) {
return;
}
final ThreadPool threadPool = transportService.getThreadPool();
threadPool.scheduleUnlessShuttingDown(bwcPingTimeout, Names.SAME, new Runnable() {
@Override
public void run() {
if (isRunning() == false) {
return;
}
final Set<DiscoveryNode> discoveryNodes = Stream.concat(
StreamSupport.stream(peersSupplier.get().spliterator(), false),
Stream.of(transportService.getLocalNode())
).filter(DiscoveryNode::isMasterNode).collect(Collectors.toSet());
// this set of nodes is reasonably fresh - the PeerFinder cleans up nodes to which the transport service is not
// connected each time it wakes up (every second by default)
logger.debug("upgrading={}, minimumMasterNodes={}, nodes={}", upgrading, minimumMasterNodes, discoveryNodes);
if (discoveryNodes.stream().anyMatch(Coordinator::isZen1Node)) {
electBestOldMaster(discoveryNodes);
} else if (canBootstrap(discoveryNodes)) {
// no Zen1 nodes found, but the last-known master was a Zen1 node, so this is a rolling upgrade
transportService.getThreadPool().generic().execute(() -> {
try {
Set<String> nodeIds = new HashSet<>();
discoveryNodes.forEach(n -> nodeIds.add(n.getId()));
final Iterator<String> knownNodeIdIterator = knownMasterNodeIds.iterator();
while (nodeIds.size() < 2 * minimumMasterNodes - 1 && knownNodeIdIterator.hasNext()) {
nodeIds.add(knownNodeIdIterator.next());
}
final VotingConfiguration votingConfiguration = new VotingConfiguration(nodeIds);
assert votingConfiguration.hasQuorum(
discoveryNodes.stream().map(DiscoveryNode::getId).collect(Collectors.toList())
);
assert 2 * minimumMasterNodes - 2 <= nodeIds.size() : nodeIds + " too small for " + minimumMasterNodes;
initialConfigurationConsumer.accept(votingConfiguration);
} catch (Exception e) {
logger.debug("exception during bootstrapping upgrade, retrying", e);
} finally {
scheduleNextAttempt();
}
});
} else {
scheduleNextAttempt();
}
}
/**
* Ping all the old discovered masters one more time to obtain their cluster state versions, and then vote for the best one.
* @param discoveryNodes The master nodes (old and new).
*/
private void electBestOldMaster(Set<DiscoveryNode> discoveryNodes) {
final Set<MasterCandidate> masterCandidates = newConcurrentSet();
final ListenableCountDown listenableCountDown = new ListenableCountDown(
discoveryNodes.size(),
new ActionListener<Void>() {
@Override
public void onResponse(Void value) {
assert masterCandidates.size() == discoveryNodes.size() : masterCandidates
+ " does not match "
+ discoveryNodes;
// TODO we shouldn't elect a master with a version that's older than ours
// If the only Zen1 nodes left are stale, and we can bootstrap, maybe we should bootstrap?
// Do we ever need to elect a freshly-started Zen1 node?
if (isRunning()) {
final MasterCandidate electedMaster = electMasterService.electMaster(masterCandidates);
logger.debug("elected {}, sending join", electedMaster);
joinHelper.sendJoinRequest(
electedMaster.getNode(),
0L,
Optional.empty(),
JoiningRound.this::scheduleNextAttempt
);
}
}
@Override
public void onFailure(Exception e) {
scheduleNextAttempt();
}
}
);
boolean foundOldMaster = false;
for (final DiscoveryNode discoveryNode : discoveryNodes) {
assert discoveryNode.isMasterNode() : discoveryNode;
if (Coordinator.isZen1Node(discoveryNode)) {
foundOldMaster = true;
transportService.sendRequest(
discoveryNode,
UnicastZenPing.ACTION_NAME,
new UnicastPingRequest(
0,
TimeValue.ZERO,
new PingResponse(
createDiscoveryNodeWithImpossiblyHighId(transportService.getLocalNode()),
null,
clusterName,
UNKNOWN_VERSION
)
),
TransportRequestOptions.builder().withTimeout(bwcPingTimeout).build(),
new TransportResponseHandler<UnicastPingResponse>() {
@Override
public void handleResponse(UnicastPingResponse response) {
long clusterStateVersion = UNKNOWN_VERSION;
for (PingResponse pingResponse : response.pingResponses) {
if (discoveryNode.equals(pingResponse.node())) {
clusterStateVersion = max(clusterStateVersion, pingResponse.getClusterStateVersion());
}
}
masterCandidates.add(new MasterCandidate(discoveryNode, clusterStateVersion));
listenableCountDown.countDown();
}
@Override
public void handleException(TransportException exp) {
logger.debug(new ParameterizedMessage("unexpected exception when pinging {}", discoveryNode), exp);
listenableCountDown.onFailure(exp);
}
@Override
public String executor() {
return Names.SAME;
}
@Override
public UnicastPingResponse read(StreamInput in) throws IOException {
return new UnicastPingResponse(in);
}
}
);
} else {
masterCandidates.add(
new MasterCandidate(createDiscoveryNodeWithImpossiblyHighId(discoveryNode), UNKNOWN_VERSION)
);
listenableCountDown.countDown();
}
}
assert foundOldMaster;
}
@Override
public String toString() {
return "discovery upgrade service retry";
}
});
}
}
/**
* Pre-7.0 nodes select the best master by comparing their IDs (as strings) and selecting the lowest one amongst those nodes with
* the best cluster state version. We want 7.0+ nodes to participate in these elections in a mixed cluster but never to win one, so
* we lie and claim to have an impossible ID that compares above all genuine IDs.
*/
public static DiscoveryNode createDiscoveryNodeWithImpossiblyHighId(DiscoveryNode node) {
// IDs are base-64-encoded UUIDs, which means they use the character set [0-9A-Za-z_-]. The highest character in this set is 'z',
// and 'z' < '{', so by starting the ID with '{' we can be sure it's greater. This is terrible.
return new DiscoveryNode(
node.getName(),
"{zen2}" + node.getId(),
node.getEphemeralId(),
node.getHostName(),
node.getHostAddress(),
node.getAddress(),
node.getAttributes(),
node.getRoles(),
node.getVersion()
);
}
}

View File

@ -35,8 +35,6 @@ package org.opensearch.cluster.coordination;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.Coordinator.Mode;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
@ -46,7 +44,6 @@ import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.discovery.zen.NodesFaultDetection;
import org.opensearch.monitor.NodeHealthService;
import org.opensearch.monitor.StatusInfo;
import org.opensearch.threadpool.ThreadPool.Names;
@ -152,15 +149,6 @@ public class FollowersChecker {
FollowerCheckRequest::new,
(request, transportChannel, task) -> handleFollowerCheck(request, transportChannel)
);
transportService.registerRequestHandler(
NodesFaultDetection.PING_ACTION_NAME,
Names.SAME,
false,
false,
NodesFaultDetection.PingRequest::new,
(request, channel, task) -> // TODO: check that we're a follower of the requesting node?
channel.sendResponse(new NodesFaultDetection.PingResponse())
);
transportService.addConnectionListener(new TransportConnectionListener() {
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
@ -348,24 +336,10 @@ public class FollowersChecker {
final FollowerCheckRequest request = new FollowerCheckRequest(fastResponseState.term, transportService.getLocalNode());
logger.trace("handleWakeUp: checking {} with {}", discoveryNode, request);
final String actionName;
final TransportRequest transportRequest;
if (Coordinator.isZen1Node(discoveryNode)) {
actionName = NodesFaultDetection.PING_ACTION_NAME;
transportRequest = new NodesFaultDetection.PingRequest(
discoveryNode,
ClusterName.CLUSTER_NAME_SETTING.get(settings),
transportService.getLocalNode(),
ClusterState.UNKNOWN_VERSION
);
} else {
actionName = FOLLOWER_CHECK_ACTION_NAME;
transportRequest = request;
}
transportService.sendRequest(
discoveryNode,
actionName,
transportRequest,
FOLLOWER_CHECK_ACTION_NAME,
request,
TransportRequestOptions.builder().withTimeout(followerCheckTimeout).withType(Type.PING).build(),
new TransportResponseHandler<Empty>() {
@Override

View File

@ -53,8 +53,6 @@ import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.discovery.zen.MembershipAction;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.monitor.NodeHealthService;
import org.opensearch.monitor.StatusInfo;
import org.opensearch.threadpool.ThreadPool;
@ -172,18 +170,6 @@ public class JoinHelper {
(request, channel, task) -> joinHandler.accept(request, transportJoinCallback(request, channel))
);
transportService.registerRequestHandler(
MembershipAction.DISCOVERY_JOIN_ACTION_NAME,
ThreadPool.Names.GENERIC,
false,
false,
MembershipAction.JoinRequest::new,
(request, channel, task) -> joinHandler.accept(
new JoinRequest(request.getNode(), 0L, Optional.empty()), // treat as non-voting join
transportJoinCallback(request, channel)
)
);
transportService.registerRequestHandler(
START_JOIN_ACTION_NAME,
Names.GENERIC,
@ -218,42 +204,6 @@ public class JoinHelper {
channel.sendResponse(Empty.INSTANCE);
}
);
transportService.registerRequestHandler(
MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME,
ThreadPool.Names.GENERIC,
ValidateJoinRequest::new,
(request, channel, task) -> {
final ClusterState localState = currentStateSupplier.get();
if (localState.metadata().clusterUUIDCommitted()
&& localState.metadata().clusterUUID().equals(request.getState().metadata().clusterUUID()) == false) {
throw new CoordinationStateRejectedException(
"mixed-version cluster join validation on cluster state"
+ " with a different cluster uuid "
+ request.getState().metadata().clusterUUID()
+ " than local cluster uuid "
+ localState.metadata().clusterUUID()
+ ", rejecting"
);
}
joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState()));
channel.sendResponse(Empty.INSTANCE);
}
);
transportService.registerRequestHandler(
ZenDiscovery.DISCOVERY_REJOIN_ACTION_NAME,
ThreadPool.Names.SAME,
ZenDiscovery.RejoinClusterRequest::new,
(request, channel, task) -> channel.sendResponse(Empty.INSTANCE)
); // TODO: do we need to implement anything here?
transportService.registerRequestHandler(
MembershipAction.DISCOVERY_LEAVE_ACTION_NAME,
ThreadPool.Names.SAME,
MembershipAction.LeaveRequest::new,
(request, channel, task) -> channel.sendResponse(Empty.INSTANCE)
); // TODO: do we need to implement anything here?
}
private JoinCallback transportJoinCallback(TransportRequest request, TransportChannel channel) {
@ -357,23 +307,11 @@ public class JoinHelper {
final Tuple<DiscoveryNode, JoinRequest> dedupKey = Tuple.tuple(destination, joinRequest);
if (pendingOutgoingJoins.add(dedupKey)) {
logger.debug("attempting to join {} with {}", destination, joinRequest);
final String actionName;
final TransportRequest transportRequest;
final TransportRequestOptions transportRequestOptions;
if (Coordinator.isZen1Node(destination)) {
actionName = MembershipAction.DISCOVERY_JOIN_ACTION_NAME;
transportRequest = new MembershipAction.JoinRequest(transportService.getLocalNode());
transportRequestOptions = TransportRequestOptions.builder().withTimeout(joinTimeout).build();
} else {
actionName = JOIN_ACTION_NAME;
transportRequest = joinRequest;
transportRequestOptions = TransportRequestOptions.EMPTY;
}
transportService.sendRequest(
destination,
actionName,
transportRequest,
transportRequestOptions,
JOIN_ACTION_NAME,
joinRequest,
TransportRequestOptions.EMPTY,
new TransportResponseHandler<Empty>() {
@Override
public Empty read(StreamInput in) {
@ -436,15 +374,9 @@ public class JoinHelper {
}
public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, ActionListener<TransportResponse.Empty> listener) {
final String actionName;
if (Coordinator.isZen1Node(node)) {
actionName = MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME;
} else {
actionName = VALIDATE_JOIN_ACTION_NAME;
}
transportService.sendRequest(
node,
actionName,
VALIDATE_JOIN_ACTION_NAME,
new ValidateJoinRequest(state),
new ActionListenerResponseHandler<>(listener, i -> Empty.INSTANCE, ThreadPool.Names.GENERIC)
);

View File

@ -47,7 +47,6 @@ import org.opensearch.cluster.routing.RerouteService;
import org.opensearch.cluster.routing.allocation.AllocationService;
import org.opensearch.common.Priority;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.persistent.PersistentTasksCustomMetadata;
import org.opensearch.transport.TransportService;
@ -71,8 +70,6 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor<JoinTaskExecut
private final RerouteService rerouteService;
private final TransportService transportService;
private final int minimumMasterNodesOnLocalNode;
public static class Task {
private final DiscoveryNode node;
@ -117,7 +114,6 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor<JoinTaskExecut
) {
this.allocationService = allocationService;
this.logger = logger;
minimumMasterNodesOnLocalNode = ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
this.rerouteService = rerouteService;
this.transportService = transportService;
}
@ -270,7 +266,6 @@ public class JoinTaskExecutor implements ClusterStateTaskExecutor<JoinTaskExecut
ClusterState tmpState = ClusterState.builder(currentState)
.nodes(nodesBuilder)
.blocks(ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_ID))
.minimumMasterNodesOnPublishingMaster(minimumMasterNodesOnLocalNode)
.build();
logger.trace("becomeMasterAndTrimConflictingNodes: {}", tmpState.nodes());
allocationService.cleanCaches();

View File

@ -36,7 +36,6 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.OpenSearchException;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.common.Nullable;
@ -46,7 +45,6 @@ import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.discovery.zen.MasterFaultDetection;
import org.opensearch.monitor.NodeHealthService;
import org.opensearch.monitor.StatusInfo;
import org.opensearch.threadpool.ThreadPool.Names;
@ -147,22 +145,6 @@ public class LeaderChecker {
}
);
transportService.registerRequestHandler(
MasterFaultDetection.MASTER_PING_ACTION_NAME,
Names.SAME,
false,
false,
MasterFaultDetection.MasterPingRequest::new,
(request, channel, task) -> {
try {
handleLeaderCheck(new LeaderCheckRequest(request.sourceNode));
} catch (CoordinationStateRejectedException e) {
throw new MasterFaultDetection.ThisIsNotTheMasterYouAreLookingForException(e.getMessage());
}
channel.sendResponse(new MasterFaultDetection.MasterPingResponseResponse());
}
);
transportService.addConnectionListener(new TransportConnectionListener() {
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
@ -276,27 +258,10 @@ public class LeaderChecker {
logger.trace("checking {} with [{}] = {}", leader, LEADER_CHECK_TIMEOUT_SETTING.getKey(), leaderCheckTimeout);
final String actionName;
final TransportRequest transportRequest;
if (Coordinator.isZen1Node(leader)) {
actionName = MasterFaultDetection.MASTER_PING_ACTION_NAME;
transportRequest = new MasterFaultDetection.MasterPingRequest(
transportService.getLocalNode(),
leader,
ClusterName.CLUSTER_NAME_SETTING.get(settings)
);
} else {
actionName = LEADER_CHECK_ACTION_NAME;
transportRequest = new LeaderCheckRequest(transportService.getLocalNode());
}
// TODO lag detection:
// In the PoC, the leader sent its current version to the follower in the response to a LeaderCheck, so the follower
// could detect if it was lagging. We'd prefer this to be implemented on the leader, so the response is just
// TransportResponse.Empty here.
transportService.sendRequest(
leader,
actionName,
transportRequest,
LEADER_CHECK_ACTION_NAME,
new LeaderCheckRequest(transportService.getLocalNode()),
TransportRequestOptions.builder().withTimeout(leaderCheckTimeout).withType(Type.PING).build(),
new TransportResponseHandler<TransportResponse.Empty>() {

View File

@ -71,14 +71,6 @@ public class NoMasterBlockService {
EnumSet.of(ClusterBlockLevel.METADATA_WRITE)
);
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",
@ -92,9 +84,6 @@ public class NoMasterBlockService {
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) {

View File

@ -30,7 +30,7 @@
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
package org.opensearch.cluster.coordination;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;

View File

@ -54,13 +54,10 @@ import org.opensearch.common.io.stream.OutputStreamStreamOutput;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.zen.PublishClusterStateAction;
import org.opensearch.discovery.zen.PublishClusterStateStats;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.BytesTransportRequest;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportResponseHandler;
@ -69,7 +66,6 @@ import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
@ -124,18 +120,6 @@ public class PublicationTransportHandler {
(request, channel, task) -> channel.sendResponse(handleIncomingPublishRequest(request))
);
transportService.registerRequestHandler(
PublishClusterStateAction.SEND_ACTION_NAME,
ThreadPool.Names.GENERIC,
false,
false,
BytesTransportRequest::new,
(request, channel, task) -> {
handleIncomingPublishRequest(request);
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}
);
transportService.registerRequestHandler(
COMMIT_STATE_ACTION_NAME,
ThreadPool.Names.GENERIC,
@ -144,27 +128,6 @@ public class PublicationTransportHandler {
ApplyCommitRequest::new,
(request, channel, task) -> handleApplyCommit.accept(request, transportCommitCallback(channel))
);
transportService.registerRequestHandler(
PublishClusterStateAction.COMMIT_ACTION_NAME,
ThreadPool.Names.GENERIC,
false,
false,
PublishClusterStateAction.CommitClusterStateRequest::new,
(request, channel, task) -> {
final Optional<ClusterState> matchingClusterState = Optional.ofNullable(lastSeenClusterState.get())
.filter(cs -> cs.stateUUID().equals(request.stateUUID));
if (matchingClusterState.isPresent() == false) {
throw new IllegalStateException("can't resolve cluster state with uuid" + " [" + request.stateUUID + "] to commit");
}
final ApplyCommitRequest applyCommitRequest = new ApplyCommitRequest(
matchingClusterState.get().getNodes().getMasterNode(),
matchingClusterState.get().term(),
matchingClusterState.get().version()
);
handleApplyCommit.accept(applyCommitRequest, transportCommitCallback(channel));
}
);
}
private ActionListener<Void> transportCommitCallback(TransportChannel channel) {
@ -408,19 +371,10 @@ public class PublicationTransportHandler {
ActionListener<TransportResponse.Empty> listener
) {
assert transportService.getThreadPool().getThreadContext().isSystemContext();
final String actionName;
final TransportRequest transportRequest;
if (Coordinator.isZen1Node(destination)) {
actionName = PublishClusterStateAction.COMMIT_ACTION_NAME;
transportRequest = new PublishClusterStateAction.CommitClusterStateRequest(newState.stateUUID());
} else {
actionName = COMMIT_STATE_ACTION_NAME;
transportRequest = applyCommitRequest;
}
transportService.sendRequest(
destination,
actionName,
transportRequest,
COMMIT_STATE_ACTION_NAME,
applyCommitRequest,
stateRequestOptions,
new TransportResponseHandler<TransportResponse.Empty>() {
@ -515,24 +469,7 @@ public class PublicationTransportHandler {
return ThreadPool.Names.GENERIC;
}
};
final String actionName;
final TransportResponseHandler<?> transportResponseHandler;
if (Coordinator.isZen1Node(destination)) {
actionName = PublishClusterStateAction.SEND_ACTION_NAME;
transportResponseHandler = responseHandler.wrap(
empty -> new PublishWithJoinResponse(
new PublishResponse(newState.term(), newState.version()),
Optional.of(
new Join(destination, transportService.getLocalNode(), newState.term(), newState.term(), newState.version())
)
),
in -> TransportResponse.Empty.INSTANCE
);
} else {
actionName = PUBLISH_STATE_ACTION_NAME;
transportResponseHandler = responseHandler;
}
transportService.sendRequest(destination, actionName, request, stateRequestOptions, transportResponseHandler);
transportService.sendRequest(destination, PUBLISH_STATE_ACTION_NAME, request, stateRequestOptions, responseHandler);
} catch (Exception e) {
logger.warn(() -> new ParameterizedMessage("error sending cluster state to {}", destination), e);
listener.onFailure(e);

View File

@ -30,7 +30,7 @@
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
package org.opensearch.cluster.coordination;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;

View File

@ -59,7 +59,6 @@ import org.opensearch.cluster.action.shard.ShardStateAction;
import org.opensearch.cluster.coordination.ClusterBootstrapService;
import org.opensearch.cluster.coordination.ClusterFormationFailureHelper;
import org.opensearch.cluster.coordination.Coordinator;
import org.opensearch.cluster.coordination.DiscoveryUpgradeService;
import org.opensearch.cluster.coordination.ElectionSchedulerFactory;
import org.opensearch.cluster.coordination.FollowersChecker;
import org.opensearch.cluster.coordination.JoinHelper;
@ -93,14 +92,10 @@ import org.opensearch.common.util.PageCacheRecycler;
import org.opensearch.common.util.concurrent.OpenSearchExecutors;
import org.opensearch.common.util.concurrent.ThreadContext;
import org.opensearch.discovery.DiscoveryModule;
import org.opensearch.discovery.DiscoverySettings;
import org.opensearch.discovery.HandshakingTransportAddressConnector;
import org.opensearch.discovery.PeerFinder;
import org.opensearch.discovery.SeedHostsResolver;
import org.opensearch.discovery.SettingsBasedSeedHostsProvider;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.discovery.zen.FaultDetection;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.gateway.DanglingIndicesState;
@ -125,6 +120,7 @@ import org.opensearch.monitor.jvm.JvmService;
import org.opensearch.monitor.os.OsService;
import org.opensearch.monitor.process.ProcessService;
import org.opensearch.node.Node;
import org.opensearch.node.Node.DiscoverySettings;
import org.opensearch.node.NodeRoleSettings;
import org.opensearch.persistent.PersistentTasksClusterService;
import org.opensearch.persistent.decider.EnableAssignmentDecider;
@ -280,11 +276,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING,
InternalSnapshotsInfoService.INTERNAL_SNAPSHOT_INFO_MAX_CONCURRENT_FETCHES_SETTING,
DestructiveOperations.REQUIRES_NAME_SETTING,
DiscoverySettings.PUBLISH_TIMEOUT_SETTING,
DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING,
DiscoverySettings.COMMIT_TIMEOUT_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,
@ -349,7 +341,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
MasterService.MASTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING,
SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING,
SearchService.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS,
ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING,
TransportSearchAction.SHARD_COUNT_LIMIT_SETTING,
TransportSearchAction.SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING,
RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE,
@ -472,20 +463,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
DiscoveryModule.DISCOVERY_TYPE_SETTING,
DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING,
DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING,
FaultDetection.PING_RETRIES_SETTING,
FaultDetection.PING_TIMEOUT_SETTING,
FaultDetection.REGISTER_CONNECTION_LISTENER_SETTING,
FaultDetection.PING_INTERVAL_SETTING,
FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING,
ZenDiscovery.PING_TIMEOUT_SETTING,
ZenDiscovery.JOIN_TIMEOUT_SETTING,
ZenDiscovery.JOIN_RETRY_ATTEMPTS_SETTING,
ZenDiscovery.JOIN_RETRY_DELAY_SETTING,
ZenDiscovery.MAX_PINGS_FROM_ANOTHER_MASTER_SETTING,
ZenDiscovery.SEND_LEAVE_REQUEST_SETTING,
ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING,
ZenDiscovery.MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING,
ZenDiscovery.MAX_PENDING_CLUSTER_STATES_SETTING,
DiscoveryModule.ELECTION_STRATEGY_SETTING,
SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING,
SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING,
@ -579,15 +556,12 @@ public final class ClusterSettings extends AbstractScopedSettings {
LagDetector.CLUSTER_FOLLOWER_LAG_TIMEOUT_SETTING,
HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING,
HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING,
DiscoveryUpgradeService.BWC_PING_TIMEOUT_SETTING,
DiscoveryUpgradeService.ENABLE_UNSAFE_BOOTSTRAPPING_ON_UPGRADE_SETTING,
SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING,
FsHealthService.ENABLED_SETTING,
FsHealthService.REFRESH_INTERVAL_SETTING,
FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING,
FsHealthService.HEALTHY_TIMEOUT_SETTING,
TransportMainAction.OVERRIDE_MAIN_RESPONSE_VERSION,
IndexingPressure.MAX_INDEXING_BYTES,
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_PROVISIONED_CAPACITY_SETTING,
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_SKEW_FACTOR_SETTING,
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_ALLOW_UNASSIGNED_PRIMARIES_SETTING,
@ -602,7 +576,8 @@ public final class ClusterSettings extends AbstractScopedSettings {
ShardIndexingPressureMemoryManager.NODE_SOFT_LIMIT,
ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS,
ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT,
ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS
ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS,
IndexingPressure.MAX_INDEXING_BYTES
)
)
);

View File

@ -34,7 +34,6 @@ package org.opensearch.discovery;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.Assertions;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.Coordinator;
import org.opensearch.cluster.coordination.ElectionStrategy;
@ -51,7 +50,6 @@ import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Setting.Property;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.transport.TransportAddress;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.gateway.GatewayMetaState;
import org.opensearch.monitor.NodeHealthService;
import org.opensearch.plugins.DiscoveryPlugin;
@ -81,9 +79,7 @@ import static org.opensearch.node.Node.NODE_NAME_SETTING;
public class DiscoveryModule {
private static final Logger logger = LogManager.getLogger(DiscoveryModule.class);
public static final String ZEN_DISCOVERY_TYPE = "legacy-zen-for-testing-only-do-not-use";
public static final String ZEN2_DISCOVERY_TYPE = "zen";
public static final String SINGLE_NODE_DISCOVERY_TYPE = "single-node";
public static final Setting<String> DISCOVERY_TYPE_SETTING = new Setting<>(
@ -209,20 +205,6 @@ public class DiscoveryModule {
electionStrategy,
nodeHealthService
);
} else if (Assertions.ENABLED && ZEN_DISCOVERY_TYPE.equals(discoveryType)) {
discovery = new ZenDiscovery(
settings,
threadPool,
transportService,
namedWriteableRegistry,
masterService,
clusterApplier,
clusterSettings,
seedHostsProvider,
allocationService,
joinValidators,
rerouteService
);
} else {
throw new IllegalArgumentException("Unknown discovery type [" + discoveryType + "]");
}

View File

@ -1,124 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Setting.Property;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
/**
* Exposes common discovery settings that may be supported by all the different discovery implementations
*/
public class DiscoverySettings {
/**
* 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
**/
public static final Setting<TimeValue> PUBLISH_TIMEOUT_SETTING = Setting.positiveTimeSetting(
"discovery.zen.publish_timeout",
TimeValue.timeValueSeconds(30),
Property.Dynamic,
Property.NodeScope,
Property.Deprecated
);
/**
* sets the timeout for receiving enough acks for a specific cluster state and committing it. failing
* to receive responses within this window will cause the cluster state change to be rejected.
*/
public static final Setting<TimeValue> COMMIT_TIMEOUT_SETTING = new Setting<>(
"discovery.zen.commit_timeout",
PUBLISH_TIMEOUT_SETTING,
(s) -> TimeValue.parseTimeValue(s, TimeValue.timeValueSeconds(30), "discovery.zen.commit_timeout"),
Property.Deprecated,
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 TimeValue publishTimeout;
private volatile TimeValue commitTimeout;
private volatile boolean publishDiff;
public DiscoverySettings(Settings settings, ClusterSettings clusterSettings) {
clusterSettings.addSettingsUpdateConsumer(PUBLISH_DIFF_ENABLE_SETTING, this::setPublishDiff);
clusterSettings.addSettingsUpdateConsumer(COMMIT_TIMEOUT_SETTING, this::setCommitTimeout);
clusterSettings.addSettingsUpdateConsumer(PUBLISH_TIMEOUT_SETTING, this::setPublishTimeout);
this.publishTimeout = PUBLISH_TIMEOUT_SETTING.get(settings);
this.commitTimeout = COMMIT_TIMEOUT_SETTING.get(settings);
this.publishDiff = PUBLISH_DIFF_ENABLE_SETTING.get(settings);
}
/**
* Returns the current publish timeout
*/
public TimeValue getPublishTimeout() {
return publishTimeout;
}
public TimeValue getCommitTimeout() {
return commitTimeout;
}
private void setPublishDiff(boolean publishDiff) {
this.publishDiff = publishDiff;
}
private void setPublishTimeout(TimeValue publishTimeout) {
this.publishTimeout = publishTimeout;
}
private void setCommitTimeout(TimeValue commitTimeout) {
this.commitTimeout = commitTimeout;
}
public boolean getPublishDiff() {
return publishDiff;
}
}

View File

@ -38,8 +38,8 @@ import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.io.stream.Writeable;
import org.opensearch.common.xcontent.ToXContentFragment;
import org.opensearch.common.xcontent.XContentBuilder;
import org.opensearch.discovery.zen.PendingClusterStateStats;
import org.opensearch.discovery.zen.PublishClusterStateStats;
import org.opensearch.cluster.coordination.PendingClusterStateStats;
import org.opensearch.cluster.coordination.PublishClusterStateStats;
import java.io.IOException;

View File

@ -38,8 +38,6 @@ import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.util.SetOnce;
import org.opensearch.action.ActionListener;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.PeersResponse;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
@ -50,24 +48,13 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.common.transport.TransportAddress;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.discovery.zen.UnicastZenPing;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.discovery.zen.ZenPing;
import org.opensearch.discovery.zen.ZenPing.PingResponse;
import org.opensearch.tasks.Task;
import org.opensearch.threadpool.ThreadPool.Names;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestHandler;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponseHandler;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@ -77,8 +64,6 @@ import java.util.function.Consumer;
import java.util.stream.Collectors;
import static java.util.Collections.emptyList;
import static org.opensearch.cluster.coordination.Coordinator.isZen1Node;
import static org.opensearch.cluster.coordination.DiscoveryUpgradeService.createDiscoveryNodeWithImpossiblyHighId;
public abstract class PeerFinder {
@ -101,8 +86,6 @@ public abstract class PeerFinder {
Setting.Property.NodeScope
);
private final Settings settings;
private final TimeValue findPeersInterval;
private final TimeValue requestPeersTimeout;
@ -124,7 +107,6 @@ public abstract class PeerFinder {
TransportAddressConnector transportAddressConnector,
ConfiguredHostsResolver configuredHostsResolver
) {
this.settings = settings;
findPeersInterval = DISCOVERY_FIND_PEERS_INTERVAL_SETTING.get(settings);
requestPeersTimeout = DISCOVERY_REQUEST_PEERS_TIMEOUT_SETTING.get(settings);
this.transportService = transportService;
@ -139,15 +121,6 @@ public abstract class PeerFinder {
PeersRequest::new,
(request, channel, task) -> channel.sendResponse(handlePeersRequest(request))
);
transportService.registerRequestHandler(
UnicastZenPing.ACTION_NAME,
Names.GENERIC,
false,
false,
UnicastZenPing.UnicastPingRequest::new,
new Zen1UnicastPingRequestHandler()
);
}
public void activate(final DiscoveryNodes lastAcceptedNodes) {
@ -491,47 +464,12 @@ public abstract class PeerFinder {
return Names.GENERIC;
}
};
final String actionName;
final TransportRequest transportRequest;
final TransportResponseHandler<?> transportResponseHandler;
if (isZen1Node(discoveryNode)) {
actionName = UnicastZenPing.ACTION_NAME;
transportRequest = new UnicastZenPing.UnicastPingRequest(
1,
ZenDiscovery.PING_TIMEOUT_SETTING.get(settings),
new ZenPing.PingResponse(
createDiscoveryNodeWithImpossiblyHighId(getLocalNode()),
null,
ClusterName.CLUSTER_NAME_SETTING.get(settings),
ClusterState.UNKNOWN_VERSION
)
);
transportResponseHandler = peersResponseHandler.wrap(ucResponse -> {
Optional<DiscoveryNode> optionalMasterNode = Arrays.stream(ucResponse.pingResponses)
.filter(pr -> discoveryNode.equals(pr.node()) && discoveryNode.equals(pr.master()))
.map(ZenPing.PingResponse::node)
.findFirst();
List<DiscoveryNode> discoveredNodes = new ArrayList<>();
if (optionalMasterNode.isPresent() == false) {
Arrays.stream(ucResponse.pingResponses)
.map(PingResponse::master)
.filter(Objects::nonNull)
.forEach(discoveredNodes::add);
Arrays.stream(ucResponse.pingResponses).map(PingResponse::node).forEach(discoveredNodes::add);
}
return new PeersResponse(optionalMasterNode, discoveredNodes, 0L);
}, UnicastZenPing.UnicastPingResponse::new);
} else {
actionName = REQUEST_PEERS_ACTION_NAME;
transportRequest = new PeersRequest(getLocalNode(), knownNodes);
transportResponseHandler = peersResponseHandler;
}
transportService.sendRequest(
discoveryNode,
actionName,
transportRequest,
REQUEST_PEERS_ACTION_NAME,
new PeersRequest(getLocalNode(), knownNodes),
TransportRequestOptions.builder().withTimeout(requestPeersTimeout).build(),
transportResponseHandler
peersResponseHandler
);
}
@ -547,38 +485,4 @@ public abstract class PeerFinder {
+ '}';
}
}
private class Zen1UnicastPingRequestHandler implements TransportRequestHandler<UnicastZenPing.UnicastPingRequest> {
@Override
public void messageReceived(UnicastZenPing.UnicastPingRequest request, TransportChannel channel, Task task) throws Exception {
final PeersRequest peersRequest = new PeersRequest(
request.pingResponse.node(),
Optional.ofNullable(request.pingResponse.master()).map(Collections::singletonList).orElse(emptyList())
);
final PeersResponse peersResponse = handlePeersRequest(peersRequest);
final List<ZenPing.PingResponse> pingResponses = new ArrayList<>();
final ClusterName clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
pingResponses.add(
new ZenPing.PingResponse(
createDiscoveryNodeWithImpossiblyHighId(transportService.getLocalNode()),
peersResponse.getMasterNode().orElse(null),
clusterName,
ClusterState.UNKNOWN_VERSION
)
);
peersResponse.getKnownPeers()
.forEach(
dn -> pingResponses.add(
new ZenPing.PingResponse(
ZenPing.PingResponse.FAKE_PING_ID,
isZen1Node(dn) ? dn : createDiscoveryNodeWithImpossiblyHighId(dn),
null,
clusterName,
ClusterState.UNKNOWN_VERSION
)
)
);
channel.sendResponse(new UnicastZenPing.UnicastPingResponse(request.id, pingResponses.toArray(new ZenPing.PingResponse[0])));
}
}
}

View File

@ -1,241 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import com.carrotsearch.hppc.ObjectContainer;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.CollectionUtil;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Setting.Property;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.CollectionUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
public class ElectMasterService {
private static final Logger logger = LogManager.getLogger(ElectMasterService.class);
public static final Setting<Integer> DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING = Setting.intSetting(
"discovery.zen.minimum_master_nodes",
-1,
Property.Dynamic,
Property.NodeScope,
Property.Deprecated
);
private volatile int minimumMasterNodes;
/**
* a class to encapsulate all the information about a candidate in a master election
* that is needed to decided which of the candidates should win
*/
public static class MasterCandidate {
public static final long UNRECOVERED_CLUSTER_VERSION = -1;
final DiscoveryNode node;
final long clusterStateVersion;
public MasterCandidate(DiscoveryNode node, long clusterStateVersion) {
Objects.requireNonNull(node);
assert clusterStateVersion >= -1 : "got: " + clusterStateVersion;
assert node.isMasterNode();
this.node = node;
this.clusterStateVersion = clusterStateVersion;
}
public DiscoveryNode getNode() {
return node;
}
public long getClusterStateVersion() {
return clusterStateVersion;
}
@Override
public String toString() {
return "Candidate{" + "node=" + node + ", clusterStateVersion=" + clusterStateVersion + '}';
}
/**
* compares two candidates to indicate which the a better master.
* A higher cluster state version is better
*
* @return -1 if c1 is a batter candidate, 1 if c2.
*/
public static int compare(MasterCandidate c1, MasterCandidate c2) {
// we explicitly swap c1 and c2 here. the code expects "better" is lower in a sorted
// list, so if c2 has a higher cluster state version, it needs to come first.
int ret = Long.compare(c2.clusterStateVersion, c1.clusterStateVersion);
if (ret == 0) {
ret = compareNodes(c1.getNode(), c2.getNode());
}
return ret;
}
}
public ElectMasterService(Settings settings) {
this.minimumMasterNodes = DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
logger.debug("using minimum_master_nodes [{}]", minimumMasterNodes);
}
public void minimumMasterNodes(int minimumMasterNodes) {
this.minimumMasterNodes = minimumMasterNodes;
}
public int minimumMasterNodes() {
return minimumMasterNodes;
}
public int countMasterNodes(Iterable<DiscoveryNode> nodes) {
int count = 0;
for (DiscoveryNode node : nodes) {
if (node.isMasterNode()) {
count++;
}
}
return count;
}
public boolean hasEnoughCandidates(Collection<MasterCandidate> candidates) {
if (candidates.isEmpty()) {
return false;
}
if (minimumMasterNodes < 1) {
return true;
}
assert candidates.stream().map(MasterCandidate::getNode).collect(Collectors.toSet()).size() == candidates
.size() : "duplicates ahead: " + candidates;
return candidates.size() >= minimumMasterNodes;
}
/**
* Elects a new master out of the possible nodes, returning it. Returns {@code null}
* if no master has been elected.
*/
public MasterCandidate electMaster(Collection<MasterCandidate> candidates) {
assert hasEnoughCandidates(candidates);
List<MasterCandidate> sortedCandidates = new ArrayList<>(candidates);
sortedCandidates.sort(MasterCandidate::compare);
return sortedCandidates.get(0);
}
/** selects the best active master to join, where multiple are discovered */
public DiscoveryNode tieBreakActiveMasters(Collection<DiscoveryNode> activeMasters) {
return activeMasters.stream().min(ElectMasterService::compareNodes).get();
}
public boolean hasEnoughMasterNodes(Iterable<DiscoveryNode> nodes) {
final int count = countMasterNodes(nodes);
return count > 0 && (minimumMasterNodes < 0 || count >= minimumMasterNodes);
}
public boolean hasTooManyMasterNodes(Iterable<DiscoveryNode> nodes) {
final int count = countMasterNodes(nodes);
return count > 1 && minimumMasterNodes <= count / 2;
}
public void logMinimumMasterNodesWarningIfNecessary(ClusterState oldState, ClusterState newState) {
// check if min_master_nodes setting is too low and log warning
if (hasTooManyMasterNodes(oldState.nodes()) == false && hasTooManyMasterNodes(newState.nodes())) {
logger.warn(
"value for setting \"{}\" is too low. This can result in data loss! Please set it to at least a quorum of master-"
+ "eligible nodes (current value: [{}], total number of master-eligible nodes used for publishing in this round: [{}])",
ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(),
minimumMasterNodes(),
newState.getNodes().getMasterNodes().size()
);
}
}
/**
* Returns the given nodes sorted by likelihood of being elected as master, most likely first.
* Non-master nodes are not removed but are rather put in the end
*/
static List<DiscoveryNode> sortByMasterLikelihood(Iterable<DiscoveryNode> nodes) {
ArrayList<DiscoveryNode> sortedNodes = CollectionUtils.iterableAsArrayList(nodes);
CollectionUtil.introSort(sortedNodes, ElectMasterService::compareNodes);
return sortedNodes;
}
/**
* Returns a list of the next possible masters.
*/
public DiscoveryNode[] nextPossibleMasters(ObjectContainer<DiscoveryNode> nodes, int numberOfPossibleMasters) {
List<DiscoveryNode> sortedNodes = sortedMasterNodes(Arrays.asList(nodes.toArray(DiscoveryNode.class)));
if (sortedNodes == null) {
return new DiscoveryNode[0];
}
List<DiscoveryNode> nextPossibleMasters = new ArrayList<>(numberOfPossibleMasters);
int counter = 0;
for (DiscoveryNode nextPossibleMaster : sortedNodes) {
if (++counter >= numberOfPossibleMasters) {
break;
}
nextPossibleMasters.add(nextPossibleMaster);
}
return nextPossibleMasters.toArray(new DiscoveryNode[nextPossibleMasters.size()]);
}
private List<DiscoveryNode> sortedMasterNodes(Iterable<DiscoveryNode> nodes) {
List<DiscoveryNode> possibleNodes = CollectionUtils.iterableAsArrayList(nodes);
if (possibleNodes.isEmpty()) {
return null;
}
// clean non master nodes
possibleNodes.removeIf(node -> !node.isMasterNode());
CollectionUtil.introSort(possibleNodes, ElectMasterService::compareNodes);
return possibleNodes;
}
/** master nodes go before other nodes, with a secondary sort by id **/
private static int compareNodes(DiscoveryNode o1, DiscoveryNode o2) {
if (o1.isMasterNode() && !o2.isMasterNode()) {
return -1;
}
if (!o1.isMasterNode() && o2.isMasterNode()) {
return 1;
}
return o1.getId().compareTo(o2.getId());
}
}

View File

@ -1,150 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import java.io.Closeable;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Setting.Property;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.Transport;
import org.opensearch.transport.TransportConnectionListener;
import org.opensearch.transport.TransportService;
import static org.opensearch.common.unit.TimeValue.timeValueSeconds;
/**
* A base class for {@link MasterFaultDetection} &amp; {@link NodesFaultDetection},
* making sure both use the same setting.
*/
public abstract class FaultDetection implements Closeable {
private static final Logger logger = LogManager.getLogger(FaultDetection.class);
public static final Setting<Boolean> CONNECT_ON_NETWORK_DISCONNECT_SETTING = Setting.boolSetting(
"discovery.zen.fd.connect_on_network_disconnect",
false,
Property.NodeScope,
Property.Deprecated
);
public static final Setting<TimeValue> PING_INTERVAL_SETTING = Setting.positiveTimeSetting(
"discovery.zen.fd.ping_interval",
timeValueSeconds(1),
Property.NodeScope,
Property.Deprecated
);
public static final Setting<TimeValue> PING_TIMEOUT_SETTING = Setting.timeSetting(
"discovery.zen.fd.ping_timeout",
timeValueSeconds(30),
Property.NodeScope,
Property.Deprecated
);
public static final Setting<Integer> PING_RETRIES_SETTING = Setting.intSetting(
"discovery.zen.fd.ping_retries",
3,
Property.NodeScope,
Property.Deprecated
);
public static final Setting<Boolean> REGISTER_CONNECTION_LISTENER_SETTING = Setting.boolSetting(
"discovery.zen.fd.register_connection_listener",
true,
Property.NodeScope,
Property.Deprecated
);
protected final ThreadPool threadPool;
protected final ClusterName clusterName;
protected final TransportService transportService;
// used mainly for testing, should always be true
protected final boolean registerConnectionListener;
protected final FDConnectionListener connectionListener;
protected final boolean connectOnNetworkDisconnect;
protected final TimeValue pingInterval;
protected final TimeValue pingRetryTimeout;
protected final int pingRetryCount;
public FaultDetection(Settings settings, ThreadPool threadPool, TransportService transportService, ClusterName clusterName) {
this.threadPool = threadPool;
this.transportService = transportService;
this.clusterName = clusterName;
this.connectOnNetworkDisconnect = CONNECT_ON_NETWORK_DISCONNECT_SETTING.get(settings);
this.pingInterval = PING_INTERVAL_SETTING.get(settings);
this.pingRetryTimeout = PING_TIMEOUT_SETTING.get(settings);
this.pingRetryCount = PING_RETRIES_SETTING.get(settings);
this.registerConnectionListener = REGISTER_CONNECTION_LISTENER_SETTING.get(settings);
this.connectionListener = new FDConnectionListener();
if (registerConnectionListener) {
transportService.addConnectionListener(connectionListener);
}
}
@Override
public void close() {
transportService.removeConnectionListener(connectionListener);
}
/**
* This method will be called when the {@link org.opensearch.transport.TransportService} raised a node disconnected event
*/
abstract void handleTransportDisconnect(DiscoveryNode node);
private class FDConnectionListener implements TransportConnectionListener {
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
AbstractRunnable runnable = new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
logger.warn("failed to handle transport disconnect for node: {}", node);
}
@Override
protected void doRun() {
handleTransportDisconnect(node);
}
};
threadPool.generic().execute(runnable);
}
}
}

View File

@ -1,504 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.OpenSearchException;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.ClusterStateUpdateTask;
import org.opensearch.cluster.NotMasterException;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.cluster.service.MasterService;
import org.opensearch.common.Nullable;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
import org.opensearch.tasks.Task;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.ConnectTransportException;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestHandler;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportResponseHandler;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* A fault detection that pings the master periodically to see if its alive.
*/
public class MasterFaultDetection extends FaultDetection {
private static final Logger logger = LogManager.getLogger(MasterFaultDetection.class);
public static final String MASTER_PING_ACTION_NAME = "internal:discovery/zen/fd/master_ping";
public interface Listener {
/** called when pinging the master failed, like a timeout, transport disconnects etc */
void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason);
}
private final MasterService masterService;
private final java.util.function.Supplier<ClusterState> clusterStateSupplier;
private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>();
private volatile MasterPinger masterPinger;
private final Object masterNodeMutex = new Object();
private volatile DiscoveryNode masterNode;
private volatile int retryCount;
private final AtomicBoolean notifiedMasterFailure = new AtomicBoolean();
public MasterFaultDetection(
Settings settings,
ThreadPool threadPool,
TransportService transportService,
java.util.function.Supplier<ClusterState> clusterStateSupplier,
MasterService masterService,
ClusterName clusterName
) {
super(settings, threadPool, transportService, clusterName);
this.clusterStateSupplier = clusterStateSupplier;
this.masterService = masterService;
logger.debug(
"[master] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]",
pingInterval,
pingRetryTimeout,
pingRetryCount
);
transportService.registerRequestHandler(
MASTER_PING_ACTION_NAME,
ThreadPool.Names.SAME,
false,
false,
MasterPingRequest::new,
new MasterPingRequestHandler()
);
}
public DiscoveryNode masterNode() {
return this.masterNode;
}
public void addListener(Listener listener) {
listeners.add(listener);
}
public void removeListener(Listener listener) {
listeners.remove(listener);
}
public void restart(DiscoveryNode masterNode, String reason) {
synchronized (masterNodeMutex) {
if (logger.isDebugEnabled()) {
logger.debug("[master] restarting fault detection against master [{}], reason [{}]", masterNode, reason);
}
innerStop();
innerStart(masterNode);
}
}
private void innerStart(final DiscoveryNode masterNode) {
this.masterNode = masterNode;
this.retryCount = 0;
this.notifiedMasterFailure.set(false);
if (masterPinger != null) {
masterPinger.stop();
}
this.masterPinger = new MasterPinger();
// we start pinging slightly later to allow the chosen master to complete it's own master election
threadPool.schedule(masterPinger, pingInterval, ThreadPool.Names.SAME);
}
public void stop(String reason) {
synchronized (masterNodeMutex) {
if (masterNode != null) {
if (logger.isDebugEnabled()) {
logger.debug("[master] stopping fault detection against master [{}], reason [{}]", masterNode, reason);
}
}
innerStop();
}
}
private void innerStop() {
// also will stop the next ping schedule
this.retryCount = 0;
if (masterPinger != null) {
masterPinger.stop();
masterPinger = null;
}
this.masterNode = null;
}
@Override
public void close() {
super.close();
stop("closing");
this.listeners.clear();
}
@Override
protected void handleTransportDisconnect(DiscoveryNode node) {
synchronized (masterNodeMutex) {
if (!node.equals(this.masterNode)) {
return;
}
if (connectOnNetworkDisconnect) {
try {
transportService.connectToNode(node);
// if all is well, make sure we restart the pinger
if (masterPinger != null) {
masterPinger.stop();
}
this.masterPinger = new MasterPinger();
// we use schedule with a 0 time value to run the pinger on the pool as it will run on later
threadPool.schedule(masterPinger, TimeValue.timeValueMillis(0), ThreadPool.Names.SAME);
} catch (Exception e) {
logger.trace("[master] [{}] transport disconnected (with verified connect)", masterNode);
notifyMasterFailure(masterNode, null, "transport disconnected (with verified connect)");
}
} else {
logger.trace("[master] [{}] transport disconnected", node);
notifyMasterFailure(node, null, "transport disconnected");
}
}
}
private void notifyMasterFailure(final DiscoveryNode masterNode, final Throwable cause, final String reason) {
if (notifiedMasterFailure.compareAndSet(false, true)) {
try {
threadPool.generic().execute(() -> {
for (Listener listener : listeners) {
listener.onMasterFailure(masterNode, cause, reason);
}
});
} catch (OpenSearchRejectedExecutionException e) {
logger.error("master failure notification was rejected, it's highly likely the node is shutting down", e);
}
stop("master failure, " + reason);
}
}
private class MasterPinger implements Runnable {
private volatile boolean running = true;
public void stop() {
this.running = false;
}
@Override
public void run() {
if (!running) {
// return and don't spawn...
return;
}
final DiscoveryNode masterToPing = masterNode;
if (masterToPing == null) {
// master is null, should not happen, but we are still running, so reschedule
threadPool.schedule(MasterPinger.this, pingInterval, ThreadPool.Names.SAME);
return;
}
final MasterPingRequest request = new MasterPingRequest(
clusterStateSupplier.get().nodes().getLocalNode(),
masterToPing,
clusterName
);
final TransportRequestOptions options = TransportRequestOptions.builder()
.withType(TransportRequestOptions.Type.PING)
.withTimeout(pingRetryTimeout)
.build();
transportService.sendRequest(
masterToPing,
MASTER_PING_ACTION_NAME,
request,
options,
new TransportResponseHandler<MasterPingResponseResponse>() {
@Override
public MasterPingResponseResponse read(StreamInput in) throws IOException {
return new MasterPingResponseResponse(in);
}
@Override
public void handleResponse(MasterPingResponseResponse response) {
if (!running) {
return;
}
// reset the counter, we got a good result
MasterFaultDetection.this.retryCount = 0;
// check if the master node did not get switched on us..., if it did, we simply return with no reschedule
if (masterToPing.equals(MasterFaultDetection.this.masterNode())) {
// we don't stop on disconnection from master, we keep pinging it
threadPool.schedule(MasterPinger.this, pingInterval, ThreadPool.Names.SAME);
}
}
@Override
public void handleException(TransportException exp) {
if (!running) {
return;
}
synchronized (masterNodeMutex) {
// check if the master node did not get switched on us...
if (masterToPing.equals(MasterFaultDetection.this.masterNode())) {
if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) {
handleTransportDisconnect(masterToPing);
return;
} else if (exp.getCause() instanceof NotMasterException) {
logger.debug("[master] pinging a master {} that is no longer a master", masterNode);
notifyMasterFailure(masterToPing, exp, "no longer master");
return;
} else if (exp.getCause() instanceof ThisIsNotTheMasterYouAreLookingForException) {
logger.debug("[master] pinging a master {} that is not the master", masterNode);
notifyMasterFailure(masterToPing, exp, "not master");
return;
} else if (exp.getCause() instanceof NodeDoesNotExistOnMasterException) {
logger.debug(
"[master] pinging a master {} but we do not exists on it, act as if its master failure",
masterNode
);
notifyMasterFailure(masterToPing, exp, "do not exists on master, act as master failure");
return;
}
int retryCount = ++MasterFaultDetection.this.retryCount;
logger.trace(
() -> new ParameterizedMessage(
"[master] failed to ping [{}], retry [{}] out of [{}]",
masterNode,
retryCount,
pingRetryCount
),
exp
);
if (retryCount >= pingRetryCount) {
logger.debug(
"[master] failed to ping [{}], tried [{}] times, each with maximum [{}] timeout",
masterNode,
pingRetryCount,
pingRetryTimeout
);
// not good, failure
notifyMasterFailure(
masterToPing,
null,
"failed to ping, tried ["
+ pingRetryCount
+ "] times, each with maximum ["
+ pingRetryTimeout
+ "] timeout"
);
} else {
// resend the request, not reschedule, rely on send timeout
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, this);
}
}
}
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
}
);
}
}
/** Thrown when a ping reaches the wrong node */
public static class ThisIsNotTheMasterYouAreLookingForException extends IllegalStateException {
public ThisIsNotTheMasterYouAreLookingForException(String msg) {
super(msg);
}
public ThisIsNotTheMasterYouAreLookingForException() {}
@Override
public Throwable fillInStackTrace() {
return null;
}
}
static class NodeDoesNotExistOnMasterException extends IllegalStateException {
@Override
public Throwable fillInStackTrace() {
return null;
}
}
private class MasterPingRequestHandler implements TransportRequestHandler<MasterPingRequest> {
@Override
public void messageReceived(final MasterPingRequest request, final TransportChannel channel, Task task) throws Exception {
final DiscoveryNodes nodes = clusterStateSupplier.get().nodes();
// check if we are really the same master as the one we seemed to be think we are
// this can happen if the master got "kill -9" and then another node started using the same port
if (!request.masterNode.equals(nodes.getLocalNode())) {
throw new ThisIsNotTheMasterYouAreLookingForException();
}
// ping from nodes of version < 1.4.0 will have the clustername set to null
if (request.clusterName != null && !request.clusterName.equals(clusterName)) {
logger.trace(
"master fault detection ping request is targeted for a different [{}] cluster then us [{}]",
request.clusterName,
clusterName
);
throw new ThisIsNotTheMasterYouAreLookingForException(
"master fault detection ping request is targeted for a different ["
+ request.clusterName
+ "] cluster then us ["
+ clusterName
+ "]"
);
}
// when we are elected as master or when a node joins, we use a cluster state update thread
// to incorporate that information in the cluster state. That cluster state is published
// before we make it available locally. This means that a master ping can come from a node
// that has already processed the new CS but it is not known locally.
// Therefore, if we fail we have to check again under a cluster state thread to make sure
// all processing is finished.
//
if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.sourceNode)) {
logger.trace("checking ping from {} under a cluster state thread", request.sourceNode);
masterService.submitStateUpdateTask("master ping (from: " + request.sourceNode + ")", new ClusterStateUpdateTask() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
// if we are no longer master, fail...
DiscoveryNodes nodes = currentState.nodes();
if (!nodes.nodeExists(request.sourceNode)) {
throw new NodeDoesNotExistOnMasterException();
}
return currentState;
}
@Override
public void onNoLongerMaster(String source) {
onFailure(source, new NotMasterException("local node is not master"));
}
@Override
public void onFailure(String source, @Nullable Exception e) {
if (e == null) {
e = new OpenSearchException("unknown error while processing ping");
}
try {
channel.sendResponse(e);
} catch (IOException inner) {
inner.addSuppressed(e);
logger.warn("error while sending ping response", inner);
}
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
try {
channel.sendResponse(new MasterPingResponseResponse());
} catch (IOException e) {
logger.warn("error while sending ping response", e);
}
}
});
} else {
// send a response, and note if we are connected to the master or not
channel.sendResponse(new MasterPingResponseResponse());
}
}
}
public static class MasterPingRequest extends TransportRequest {
public DiscoveryNode sourceNode;
private DiscoveryNode masterNode;
private ClusterName clusterName;
public MasterPingRequest(StreamInput in) throws IOException {
super(in);
sourceNode = new DiscoveryNode(in);
masterNode = new DiscoveryNode(in);
clusterName = new ClusterName(in);
}
public MasterPingRequest(DiscoveryNode sourceNode, DiscoveryNode masterNode, ClusterName clusterName) {
this.sourceNode = sourceNode;
this.masterNode = masterNode;
this.clusterName = clusterName;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
sourceNode.writeTo(out);
masterNode.writeTo(out);
clusterName.writeTo(out);
}
}
public static class MasterPingResponseResponse extends TransportResponse {
public MasterPingResponseResponse() {}
public MasterPingResponseResponse(StreamInput in) throws IOException {
super(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -1,249 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.ValidateJoinRequest;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.tasks.Task;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.EmptyTransportResponseHandler;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestHandler;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
import java.util.function.Supplier;
public class MembershipAction {
private static final Logger logger = LogManager.getLogger(MembershipAction.class);
public static final String DISCOVERY_JOIN_ACTION_NAME = "internal:discovery/zen/join";
public static final String DISCOVERY_JOIN_VALIDATE_ACTION_NAME = "internal:discovery/zen/join/validate";
public static final String DISCOVERY_LEAVE_ACTION_NAME = "internal:discovery/zen/leave";
public interface JoinCallback {
void onSuccess();
void onFailure(Exception e);
}
public interface MembershipListener {
void onJoin(DiscoveryNode node, JoinCallback callback);
void onLeave(DiscoveryNode node);
}
private final TransportService transportService;
private final MembershipListener listener;
public MembershipAction(
TransportService transportService,
MembershipListener listener,
Collection<BiConsumer<DiscoveryNode, ClusterState>> joinValidators
) {
this.transportService = transportService;
this.listener = listener;
transportService.registerRequestHandler(
DISCOVERY_JOIN_ACTION_NAME,
ThreadPool.Names.GENERIC,
JoinRequest::new,
new JoinRequestRequestHandler()
);
transportService.registerRequestHandler(
DISCOVERY_JOIN_VALIDATE_ACTION_NAME,
ThreadPool.Names.GENERIC,
ValidateJoinRequest::new,
new ValidateJoinRequestRequestHandler(transportService::getLocalNode, joinValidators)
);
transportService.registerRequestHandler(
DISCOVERY_LEAVE_ACTION_NAME,
ThreadPool.Names.GENERIC,
LeaveRequest::new,
new LeaveRequestRequestHandler()
);
}
public void sendLeaveRequest(DiscoveryNode masterNode, DiscoveryNode node) {
transportService.sendRequest(
node,
DISCOVERY_LEAVE_ACTION_NAME,
new LeaveRequest(masterNode),
EmptyTransportResponseHandler.INSTANCE_SAME
);
}
public void sendLeaveRequestBlocking(DiscoveryNode masterNode, DiscoveryNode node, TimeValue timeout) {
transportService.submitRequest(
masterNode,
DISCOVERY_LEAVE_ACTION_NAME,
new LeaveRequest(node),
EmptyTransportResponseHandler.INSTANCE_SAME
).txGet(timeout.millis(), TimeUnit.MILLISECONDS);
}
public void sendJoinRequestBlocking(DiscoveryNode masterNode, DiscoveryNode node, TimeValue timeout) {
transportService.submitRequest(
masterNode,
DISCOVERY_JOIN_ACTION_NAME,
new JoinRequest(node),
EmptyTransportResponseHandler.INSTANCE_SAME
).txGet(timeout.millis(), TimeUnit.MILLISECONDS);
}
/**
* Validates the join request, throwing a failure if it failed.
*/
public void sendValidateJoinRequestBlocking(DiscoveryNode node, ClusterState state, TimeValue timeout) {
transportService.submitRequest(
node,
DISCOVERY_JOIN_VALIDATE_ACTION_NAME,
new ValidateJoinRequest(state),
EmptyTransportResponseHandler.INSTANCE_SAME
).txGet(timeout.millis(), TimeUnit.MILLISECONDS);
}
public static class JoinRequest extends TransportRequest {
private DiscoveryNode node;
public DiscoveryNode getNode() {
return node;
}
public JoinRequest(StreamInput in) throws IOException {
super(in);
node = new DiscoveryNode(in);
}
public JoinRequest(DiscoveryNode node) {
this.node = node;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
node.writeTo(out);
}
}
private class JoinRequestRequestHandler implements TransportRequestHandler<JoinRequest> {
@Override
public void messageReceived(final JoinRequest request, final TransportChannel channel, Task task) throws Exception {
listener.onJoin(request.getNode(), new JoinCallback() {
@Override
public void onSuccess() {
try {
channel.sendResponse(TransportResponse.Empty.INSTANCE);
} catch (Exception e) {
onFailure(e);
}
}
@Override
public void onFailure(Exception e) {
try {
channel.sendResponse(e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.warn("failed to send back failure on join request", inner);
}
}
});
}
}
static class ValidateJoinRequestRequestHandler implements TransportRequestHandler<ValidateJoinRequest> {
private final Supplier<DiscoveryNode> localNodeSupplier;
private final Collection<BiConsumer<DiscoveryNode, ClusterState>> joinValidators;
ValidateJoinRequestRequestHandler(
Supplier<DiscoveryNode> localNodeSupplier,
Collection<BiConsumer<DiscoveryNode, ClusterState>> joinValidators
) {
this.localNodeSupplier = localNodeSupplier;
this.joinValidators = joinValidators;
}
@Override
public void messageReceived(ValidateJoinRequest request, TransportChannel channel, Task task) throws Exception {
DiscoveryNode node = localNodeSupplier.get();
assert node != null : "local node is null";
joinValidators.stream().forEach(action -> action.accept(node, request.getState()));
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}
}
public static class LeaveRequest extends TransportRequest {
private DiscoveryNode node;
public LeaveRequest(StreamInput in) throws IOException {
super(in);
node = new DiscoveryNode(in);
}
private LeaveRequest(DiscoveryNode node) {
this.node = node;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
node.writeTo(out);
}
}
private class LeaveRequestRequestHandler implements TransportRequestHandler<LeaveRequest> {
@Override
public void messageReceived(LeaveRequest request, TransportChannel channel, Task task) throws Exception {
listener.onLeave(request.node);
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}
}
}

View File

@ -1,412 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.store.AlreadyClosedException;
import org.opensearch.cluster.ClusterChangedEvent;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.ClusterStateTaskConfig;
import org.opensearch.cluster.ClusterStateTaskListener;
import org.opensearch.cluster.NotMasterException;
import org.opensearch.cluster.coordination.JoinTaskExecutor;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.routing.RerouteService;
import org.opensearch.cluster.routing.allocation.AllocationService;
import org.opensearch.cluster.service.MasterService;
import org.opensearch.common.Priority;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* This class processes incoming join request (passed zia {@link ZenDiscovery}). Incoming nodes
* are directly added to the cluster state or are accumulated during master election.
*/
public class NodeJoinController {
private static final Logger logger = LogManager.getLogger(NodeJoinController.class);
private final MasterService masterService;
private final JoinTaskExecutor joinTaskExecutor;
// this is set while trying to become a master
// mutation should be done under lock
private ElectionContext electionContext = null;
public NodeJoinController(
Settings settings,
MasterService masterService,
AllocationService allocationService,
ElectMasterService electMaster,
RerouteService rerouteService
) {
this.masterService = masterService;
joinTaskExecutor = new JoinTaskExecutor(settings, allocationService, logger, rerouteService, null) {
@Override
public void clusterStatePublished(ClusterChangedEvent event) {
electMaster.logMinimumMasterNodesWarningIfNecessary(event.previousState(), event.state());
}
};
}
/**
* waits for enough incoming joins from master eligible nodes to complete the master election
* <p>
* You must start accumulating joins before calling this method. See {@link #startElectionContext()}
* <p>
* The method will return once the local node has been elected as master or some failure/timeout has happened.
* The exact outcome is communicated via the callback parameter, which is guaranteed to be called.
*
* @param requiredMasterJoins the number of joins from master eligible needed to complete the election
* @param timeValue how long to wait before failing. a timeout is communicated via the callback's onFailure method.
* @param callback the result of the election (success or failure) will be communicated by calling methods on this
* object
**/
public void waitToBeElectedAsMaster(int requiredMasterJoins, TimeValue timeValue, final ElectionCallback callback) {
final CountDownLatch done = new CountDownLatch(1);
final ElectionCallback wrapperCallback = new ElectionCallback() {
@Override
public void onElectedAsMaster(ClusterState state) {
done.countDown();
callback.onElectedAsMaster(state);
}
@Override
public void onFailure(Throwable t) {
done.countDown();
callback.onFailure(t);
}
};
ElectionContext myElectionContext = null;
try {
// check what we have so far..
// capture the context we add the callback to make sure we fail our own
synchronized (this) {
assert electionContext != null : "waitToBeElectedAsMaster is called we are not accumulating joins";
myElectionContext = electionContext;
electionContext.onAttemptToBeElected(requiredMasterJoins, wrapperCallback);
checkPendingJoinsAndElectIfNeeded();
}
try {
if (done.await(timeValue.millis(), TimeUnit.MILLISECONDS)) {
// callback handles everything
return;
}
} catch (InterruptedException e) {
}
if (logger.isTraceEnabled()) {
final int pendingNodes = myElectionContext.getPendingMasterJoinsCount();
logger.trace("timed out waiting to be elected. waited [{}]. pending master node joins [{}]", timeValue, pendingNodes);
}
failContextIfNeeded(myElectionContext, "timed out waiting to be elected");
} catch (Exception e) {
logger.error("unexpected failure while waiting for incoming joins", e);
if (myElectionContext != null) {
failContextIfNeeded(myElectionContext, "unexpected failure while waiting for pending joins [" + e.getMessage() + "]");
}
}
}
/**
* utility method to fail the given election context under the cluster state thread
*/
private synchronized void failContextIfNeeded(final ElectionContext context, final String reason) {
if (electionContext == context) {
stopElectionContext(reason);
}
}
/**
* Accumulates any future incoming join request. Pending join requests will be processed in the final steps of becoming a
* master or when {@link #stopElectionContext(String)} is called.
*/
public synchronized void startElectionContext() {
logger.trace("starting an election context, will accumulate joins");
assert electionContext == null : "double startElectionContext() calls";
electionContext = new ElectionContext();
}
/**
* Stopped accumulating joins. All pending joins will be processed. Future joins will be processed immediately
*/
public void stopElectionContext(String reason) {
logger.trace("stopping election ([{}])", reason);
synchronized (this) {
assert electionContext != null : "stopElectionContext() called but not accumulating";
electionContext.closeAndProcessPending(reason);
electionContext = null;
}
}
/**
* processes or queues an incoming join request.
* <p>
* Note: doesn't do any validation. This should have been done before.
*/
public synchronized void handleJoinRequest(final DiscoveryNode node, final MembershipAction.JoinCallback callback) {
if (electionContext != null) {
electionContext.addIncomingJoin(node, callback);
checkPendingJoinsAndElectIfNeeded();
} else {
masterService.submitStateUpdateTask(
"zen-disco-node-join",
new JoinTaskExecutor.Task(node, "no election context"),
ClusterStateTaskConfig.build(Priority.URGENT),
joinTaskExecutor,
new JoinTaskListener(callback, logger)
);
}
}
/**
* checks if there is an on going request to become master and if it has enough pending joins. If so, the node will
* become master via a ClusterState update task.
*/
private synchronized void checkPendingJoinsAndElectIfNeeded() {
assert electionContext != null : "election check requested but no active context";
final int pendingMasterJoins = electionContext.getPendingMasterJoinsCount();
if (electionContext.isEnoughPendingJoins(pendingMasterJoins) == false) {
if (logger.isTraceEnabled()) {
logger.trace(
"not enough joins for election. Got [{}], required [{}]",
pendingMasterJoins,
electionContext.requiredMasterJoins
);
}
} else {
if (logger.isTraceEnabled()) {
logger.trace(
"have enough joins for election. Got [{}], required [{}]",
pendingMasterJoins,
electionContext.requiredMasterJoins
);
}
electionContext.closeAndBecomeMaster();
electionContext = null; // clear this out so future joins won't be accumulated
}
}
public interface ElectionCallback {
/**
* called when the local node is successfully elected as master
* Guaranteed to be called on the cluster state update thread
**/
void onElectedAsMaster(ClusterState state);
/**
* called when the local node failed to be elected as master
* Guaranteed to be called on the cluster state update thread
**/
void onFailure(Throwable t);
}
class ElectionContext {
private ElectionCallback callback = null;
private int requiredMasterJoins = -1;
private final Map<DiscoveryNode, List<MembershipAction.JoinCallback>> joinRequestAccumulator = new HashMap<>();
final AtomicBoolean closed = new AtomicBoolean();
public synchronized void onAttemptToBeElected(int requiredMasterJoins, ElectionCallback callback) {
ensureOpen();
assert this.requiredMasterJoins < 0;
assert this.callback == null;
this.requiredMasterJoins = requiredMasterJoins;
this.callback = callback;
}
public synchronized void addIncomingJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) {
ensureOpen();
joinRequestAccumulator.computeIfAbsent(node, n -> new ArrayList<>()).add(callback);
}
public synchronized boolean isEnoughPendingJoins(int pendingMasterJoins) {
final boolean hasEnough;
if (requiredMasterJoins < 0) {
// requiredMasterNodes is unknown yet, return false and keep on waiting
hasEnough = false;
} else {
assert callback != null : "requiredMasterJoins is set but not the callback";
hasEnough = pendingMasterJoins >= requiredMasterJoins;
}
return hasEnough;
}
private Map<JoinTaskExecutor.Task, ClusterStateTaskListener> getPendingAsTasks(String reason) {
Map<JoinTaskExecutor.Task, ClusterStateTaskListener> tasks = new HashMap<>();
joinRequestAccumulator.entrySet()
.stream()
.forEach(e -> tasks.put(new JoinTaskExecutor.Task(e.getKey(), reason), new JoinTaskListener(e.getValue(), logger)));
return tasks;
}
public synchronized int getPendingMasterJoinsCount() {
int pendingMasterJoins = 0;
for (DiscoveryNode node : joinRequestAccumulator.keySet()) {
if (node.isMasterNode()) {
pendingMasterJoins++;
}
}
return pendingMasterJoins;
}
public synchronized void closeAndBecomeMaster() {
assert callback != null : "becoming a master but the callback is not yet set";
assert isEnoughPendingJoins(getPendingMasterJoinsCount()) : "becoming a master but pending joins of "
+ getPendingMasterJoinsCount()
+ " are not enough. needs ["
+ requiredMasterJoins
+ "];";
innerClose();
Map<JoinTaskExecutor.Task, ClusterStateTaskListener> tasks = getPendingAsTasks("become master");
final String source = "zen-disco-elected-as-master ([" + tasks.size() + "] nodes joined)";
// noop listener, the election finished listener determines result
tasks.put(JoinTaskExecutor.newBecomeMasterTask(), (source1, e) -> {});
tasks.put(JoinTaskExecutor.newFinishElectionTask(), electionFinishedListener);
masterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor);
}
public synchronized void closeAndProcessPending(String reason) {
innerClose();
Map<JoinTaskExecutor.Task, ClusterStateTaskListener> tasks = getPendingAsTasks(reason);
final String source = "zen-disco-election-stop [" + reason + "]";
tasks.put(JoinTaskExecutor.newFinishElectionTask(), electionFinishedListener);
masterService.submitStateUpdateTasks(source, tasks, ClusterStateTaskConfig.build(Priority.URGENT), joinTaskExecutor);
}
private void innerClose() {
if (closed.getAndSet(true)) {
throw new AlreadyClosedException("election context is already closed");
}
}
private void ensureOpen() {
if (closed.get()) {
throw new AlreadyClosedException("election context is already closed");
}
}
private synchronized ElectionCallback getCallback() {
return callback;
}
private void onElectedAsMaster(ClusterState state) {
assert MasterService.assertMasterUpdateThread();
assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master";
ElectionCallback callback = getCallback(); // get under lock
if (callback != null) {
callback.onElectedAsMaster(state);
}
}
private void onFailure(Throwable t) {
assert MasterService.assertMasterUpdateThread();
ElectionCallback callback = getCallback(); // get under lock
if (callback != null) {
callback.onFailure(t);
}
}
private final ClusterStateTaskListener electionFinishedListener = new ClusterStateTaskListener() {
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
if (newState.nodes().isLocalNodeElectedMaster()) {
ElectionContext.this.onElectedAsMaster(newState);
} else {
onFailure(source, new NotMasterException("election stopped [" + source + "]"));
}
}
@Override
public void onFailure(String source, Exception e) {
ElectionContext.this.onFailure(e);
}
};
}
static class JoinTaskListener implements ClusterStateTaskListener {
final List<MembershipAction.JoinCallback> callbacks;
private final Logger logger;
JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) {
this(Collections.singletonList(callback), logger);
}
JoinTaskListener(List<MembershipAction.JoinCallback> callbacks, Logger logger) {
this.callbacks = callbacks;
this.logger = logger;
}
@Override
public void onFailure(String source, Exception e) {
for (MembershipAction.JoinCallback callback : callbacks) {
try {
callback.onFailure(e);
} catch (Exception inner) {
logger.error(() -> new ParameterizedMessage("error handling task failure [{}]", e), inner);
}
}
}
@Override
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
for (MembershipAction.JoinCallback callback : callbacks) {
try {
callback.onSuccess();
} catch (Exception e) {
logger.error(() -> new ParameterizedMessage("unexpected error during [{}]", source), e);
}
}
}
}
}

View File

@ -1,415 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
import org.opensearch.tasks.Task;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.ConnectTransportException;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestHandler;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportResponseHandler;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.Collections;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.Supplier;
import static org.opensearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
/**
* A fault detection of multiple nodes.
*/
public class NodesFaultDetection extends FaultDetection {
private static final Logger logger = LogManager.getLogger(NodesFaultDetection.class);
public static final String PING_ACTION_NAME = "internal:discovery/zen/fd/ping";
public abstract static class Listener {
public void onNodeFailure(DiscoveryNode node, String reason) {}
public void onPingReceived(PingRequest pingRequest) {}
}
private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>();
private final ConcurrentMap<DiscoveryNode, NodeFD> nodesFD = newConcurrentMap();
private final Supplier<ClusterState> clusterStateSupplier;
private volatile DiscoveryNode localNode;
public NodesFaultDetection(
Settings settings,
ThreadPool threadPool,
TransportService transportService,
Supplier<ClusterState> clusterStateSupplier,
ClusterName clusterName
) {
super(settings, threadPool, transportService, clusterName);
this.clusterStateSupplier = clusterStateSupplier;
logger.debug(
"[node ] uses ping_interval [{}], ping_timeout [{}], ping_retries [{}]",
pingInterval,
pingRetryTimeout,
pingRetryCount
);
transportService.registerRequestHandler(
PING_ACTION_NAME,
ThreadPool.Names.SAME,
false,
false,
PingRequest::new,
new PingRequestHandler()
);
}
public void setLocalNode(DiscoveryNode localNode) {
this.localNode = localNode;
}
public void addListener(Listener listener) {
listeners.add(listener);
}
public void removeListener(Listener listener) {
listeners.remove(listener);
}
/**
* Gets the current set of nodes involved in node fault detection.
* NB: For testing purposes.
*/
public Set<DiscoveryNode> getNodes() {
return Collections.unmodifiableSet(nodesFD.keySet());
}
/**
* make sure that nodes in clusterState are pinged. Any pinging to nodes which are not
* part of the cluster will be stopped
*/
public void updateNodesAndPing(ClusterState clusterState) {
// remove any nodes we don't need, this will cause their FD to stop
for (DiscoveryNode monitoredNode : nodesFD.keySet()) {
if (!clusterState.nodes().nodeExists(monitoredNode)) {
nodesFD.remove(monitoredNode);
}
}
// add any missing nodes
for (DiscoveryNode node : clusterState.nodes()) {
if (node.equals(localNode)) {
// no need to monitor the local node
continue;
}
if (!nodesFD.containsKey(node)) {
NodeFD fd = new NodeFD(node);
// it's OK to overwrite an existing nodeFD - it will just stop and the new one will pick things up.
nodesFD.put(node, fd);
// we use schedule with a 0 time value to run the pinger on the pool as it will run on later
threadPool.schedule(fd, TimeValue.timeValueMillis(0), ThreadPool.Names.SAME);
}
}
}
/** stops all pinging **/
public NodesFaultDetection stop() {
nodesFD.clear();
return this;
}
@Override
public void close() {
super.close();
stop();
}
@Override
protected void handleTransportDisconnect(DiscoveryNode node) {
NodeFD nodeFD = nodesFD.remove(node);
if (nodeFD == null) {
return;
}
if (connectOnNetworkDisconnect) {
NodeFD fd = new NodeFD(node);
try {
transportService.connectToNode(node);
nodesFD.put(node, fd);
// we use schedule with a 0 time value to run the pinger on the pool as it will run on later
threadPool.schedule(fd, TimeValue.timeValueMillis(0), ThreadPool.Names.SAME);
} catch (Exception e) {
logger.trace("[node ] [{}] transport disconnected (with verified connect)", node);
// clean up if needed, just to be safe..
nodesFD.remove(node, fd);
notifyNodeFailure(node, "transport disconnected (with verified connect)");
}
} else {
logger.trace("[node ] [{}] transport disconnected", node);
notifyNodeFailure(node, "transport disconnected");
}
}
private void notifyNodeFailure(final DiscoveryNode node, final String reason) {
try {
threadPool.generic().execute(new Runnable() {
@Override
public void run() {
for (Listener listener : listeners) {
listener.onNodeFailure(node, reason);
}
}
});
} catch (OpenSearchRejectedExecutionException ex) {
logger.trace(
() -> new ParameterizedMessage(
"[node ] [{}] ignoring node failure (reason [{}]). Local node is shutting down",
node,
reason
),
ex
);
}
}
private void notifyPingReceived(final PingRequest pingRequest) {
threadPool.generic().execute(new Runnable() {
@Override
public void run() {
for (Listener listener : listeners) {
listener.onPingReceived(pingRequest);
}
}
});
}
private class NodeFD implements Runnable {
volatile int retryCount;
private final DiscoveryNode node;
private NodeFD(DiscoveryNode node) {
this.node = node;
}
private boolean running() {
return NodeFD.this.equals(nodesFD.get(node));
}
private PingRequest newPingRequest() {
return new PingRequest(node, clusterName, localNode, clusterStateSupplier.get().version());
}
@Override
public void run() {
if (!running()) {
return;
}
final TransportRequestOptions options = TransportRequestOptions.builder()
.withType(TransportRequestOptions.Type.PING)
.withTimeout(pingRetryTimeout)
.build();
transportService.sendRequest(node, PING_ACTION_NAME, newPingRequest(), options, new TransportResponseHandler<PingResponse>() {
@Override
public PingResponse read(StreamInput in) throws IOException {
return new PingResponse(in);
}
@Override
public void handleResponse(PingResponse response) {
if (!running()) {
return;
}
retryCount = 0;
threadPool.schedule(NodeFD.this, pingInterval, ThreadPool.Names.SAME);
}
@Override
public void handleException(TransportException exp) {
if (!running()) {
return;
}
if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) {
handleTransportDisconnect(node);
return;
}
retryCount++;
logger.trace(
() -> new ParameterizedMessage(
"[node ] failed to ping [{}], retry [{}] out of [{}]",
node,
retryCount,
pingRetryCount
),
exp
);
if (retryCount >= pingRetryCount) {
logger.debug(
"[node ] failed to ping [{}], tried [{}] times, each with maximum [{}] timeout",
node,
pingRetryCount,
pingRetryTimeout
);
// not good, failure
if (nodesFD.remove(node, NodeFD.this)) {
notifyNodeFailure(
node,
"failed to ping, tried [" + pingRetryCount + "] times, each with maximum [" + pingRetryTimeout + "] timeout"
);
}
} else {
// resend the request, not reschedule, rely on send timeout
transportService.sendRequest(node, PING_ACTION_NAME, newPingRequest(), options, this);
}
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
});
}
}
class PingRequestHandler implements TransportRequestHandler<PingRequest> {
@Override
public void messageReceived(PingRequest request, TransportChannel channel, Task task) throws Exception {
// if we are not the node we are supposed to be pinged, send an exception
// this can happen when a kill -9 is sent, and another node is started using the same port
if (!localNode.equals(request.targetNode())) {
throw new IllegalStateException("Got pinged as node " + request.targetNode() + "], but I am node " + localNode);
}
// PingRequest will have clusterName set to null if it came from a node of version <1.4.0
if (request.clusterName != null && !request.clusterName.equals(clusterName)) {
// Don't introduce new exception for bwc reasons
throw new IllegalStateException(
"Got pinged with cluster name [" + request.clusterName + "], but I'm part of cluster [" + clusterName + "]"
);
}
notifyPingReceived(request);
channel.sendResponse(new PingResponse());
}
}
public static class PingRequest extends TransportRequest {
// the (assumed) node we are pinging
private DiscoveryNode targetNode;
private ClusterName clusterName;
private DiscoveryNode masterNode;
private long clusterStateVersion = ClusterState.UNKNOWN_VERSION;
public PingRequest(StreamInput in) throws IOException {
super(in);
targetNode = new DiscoveryNode(in);
clusterName = new ClusterName(in);
masterNode = new DiscoveryNode(in);
clusterStateVersion = in.readLong();
}
public PingRequest(DiscoveryNode targetNode, ClusterName clusterName, DiscoveryNode masterNode, long clusterStateVersion) {
this.targetNode = targetNode;
this.clusterName = clusterName;
this.masterNode = masterNode;
this.clusterStateVersion = clusterStateVersion;
}
public DiscoveryNode targetNode() {
return targetNode;
}
public ClusterName clusterName() {
return clusterName;
}
public DiscoveryNode masterNode() {
return masterNode;
}
public long clusterStateVersion() {
return clusterStateVersion;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
targetNode.writeTo(out);
clusterName.writeTo(out);
masterNode.writeTo(out);
out.writeLong(clusterStateVersion);
}
}
public static class PingResponse extends TransportResponse {
public PingResponse() {}
public PingResponse(StreamInput in) throws IOException {
super(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {}
}
}

View File

@ -1,357 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.apache.logging.log4j.Logger;
import org.opensearch.OpenSearchException;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.CoordinationMetadata;
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.cluster.node.DiscoveryNode;
import java.util.ArrayList;
import java.util.Locale;
import java.util.Objects;
/**
* A queue that holds all "in-flight" incoming cluster states from the master. Once a master commits a cluster
* state, it is made available via {@link #getNextClusterStateToProcess()}. The class also takes care of batching
* cluster states for processing and failures.
* <p>
* The queue is bound by {@link #maxQueueSize}. When the queue is at capacity and a new cluster state is inserted
* the oldest cluster state will be dropped. This is safe because:
* 1) Under normal operations, master will publish &amp; commit a cluster state before processing
* another change (i.e., the queue length is 1)
* 2) If the master fails to commit a change, it will step down, causing a master election, which will flush the queue.
* 3) In general it's safe to process the incoming cluster state as a replacement to the cluster state that's dropped.
* a) If the dropped cluster is from the same master as the incoming one is, it is likely to be superseded by the
* incoming state (or another state in the queue).
* This is only not true in very extreme cases of out of order delivery.
* b) If the dropping cluster state is not from the same master, it means that:
* i) we are no longer following the master of the dropped cluster state but follow the incoming one
* ii) we are no longer following any master, in which case it doesn't matter which cluster state will be processed first.
* <p>
* The class is fully thread safe and can be used concurrently.
*/
public class PendingClusterStatesQueue {
interface StateProcessedListener {
void onNewClusterStateProcessed();
void onNewClusterStateFailed(Exception e);
}
final ArrayList<ClusterStateContext> pendingStates = new ArrayList<>();
final Logger logger;
final int maxQueueSize;
public PendingClusterStatesQueue(Logger logger, int maxQueueSize) {
this.logger = logger;
this.maxQueueSize = maxQueueSize;
}
/** Add an incoming, not yet committed cluster state */
public synchronized void addPending(ClusterState state) {
pendingStates.add(new ClusterStateContext(state));
if (pendingStates.size() > maxQueueSize) {
ClusterStateContext context = pendingStates.remove(0);
logger.warn("dropping pending state [{}]. more than [{}] pending states.", context, maxQueueSize);
if (context.committed()) {
context.listener.onNewClusterStateFailed(new OpenSearchException("too many pending states ([{}] pending)", maxQueueSize));
}
}
}
/**
* Mark a previously added cluster state as committed. This will make it available via {@link #getNextClusterStateToProcess()}
* When the cluster state is processed (or failed), the supplied listener will be called
**/
public synchronized ClusterState markAsCommitted(String stateUUID, StateProcessedListener listener) {
final ClusterStateContext context = findState(stateUUID);
if (context == null) {
listener.onNewClusterStateFailed(
new IllegalStateException("can't resolve cluster state with uuid" + " [" + stateUUID + "] to commit")
);
return null;
}
if (context.committed()) {
listener.onNewClusterStateFailed(
new IllegalStateException("cluster state with uuid" + " [" + stateUUID + "] is already committed")
);
return null;
}
context.markAsCommitted(listener);
return context.state;
}
/**
* mark that the processing of the given state has failed. All committed states that are
* {@link ClusterState#supersedes(ClusterState)}-ed by this failed state, will be failed as well
*/
public synchronized void markAsFailed(ClusterState state, Exception reason) {
final ClusterStateContext failedContext = findState(state.stateUUID());
if (failedContext == null) {
throw new IllegalArgumentException(
"can't resolve failed cluster state with uuid [" + state.stateUUID() + "], version [" + state.version() + "]"
);
}
if (failedContext.committed() == false) {
throw new IllegalArgumentException("failed cluster state is not committed " + state);
}
// fail all committed states which are batch together with the failed state
ArrayList<ClusterStateContext> statesToRemove = new ArrayList<>();
for (int index = 0; index < pendingStates.size(); index++) {
final ClusterStateContext pendingContext = pendingStates.get(index);
if (pendingContext.committed() == false) {
continue;
}
final ClusterState pendingState = pendingContext.state;
if (pendingContext.equals(failedContext)) {
statesToRemove.add(pendingContext);
pendingContext.listener.onNewClusterStateFailed(reason);
} else if (state.supersedes(pendingState)) {
statesToRemove.add(pendingContext);
logger.debug("failing committed state {} together with state {}", pendingContext, failedContext);
pendingContext.listener.onNewClusterStateFailed(reason);
}
}
pendingStates.removeAll(statesToRemove);
assert findState(state.stateUUID()) == null : "state was marked as processed but can still be found in pending list " + state;
}
/**
* indicates that a cluster state was successfully processed. Any committed state that is
* {@link ClusterState#supersedes(ClusterState)}-ed by the processed state will be marked as processed as well.
* <p>
* NOTE: successfully processing a state indicates we are following the master it came from. Any committed state
* from another master will be failed by this method
*/
public synchronized void markAsProcessed(ClusterState state) {
if (findState(state.stateUUID()) == null) {
throw new IllegalStateException(
"can't resolve processed cluster state with uuid [" + state.stateUUID() + "], version [" + state.version() + "]"
);
}
final DiscoveryNode currentMaster = state.nodes().getMasterNode();
assert currentMaster != null : "processed cluster state mast have a master. " + state;
// fail or remove any incoming state from a different master
// respond to any committed state from the same master with same or lower version (we processed a higher version)
ArrayList<ClusterStateContext> contextsToRemove = new ArrayList<>();
for (int index = 0; index < pendingStates.size(); index++) {
final ClusterStateContext pendingContext = pendingStates.get(index);
final ClusterState pendingState = pendingContext.state;
final DiscoveryNode pendingMasterNode = pendingState.nodes().getMasterNode();
if (Objects.equals(currentMaster, pendingMasterNode) == false) {
contextsToRemove.add(pendingContext);
if (pendingContext.committed()) {
// this is a committed state , warn
logger.warn(
"received a cluster state (uuid[{}]/v[{}]) from a different master than the current one,"
+ " rejecting (received {}, current {})",
pendingState.stateUUID(),
pendingState.version(),
pendingMasterNode,
currentMaster
);
pendingContext.listener.onNewClusterStateFailed(
new IllegalStateException(
"cluster state from a different master than the current one,"
+ " rejecting (received "
+ pendingMasterNode
+ ", current "
+ currentMaster
+ ")"
)
);
} else {
logger.trace(
"removing non-committed state with uuid[{}]/v[{}] from [{}] - a state from" + " [{}] was successfully processed",
pendingState.stateUUID(),
pendingState.version(),
pendingMasterNode,
currentMaster
);
}
} else if (pendingState.stateUUID().equals(state.stateUUID())) {
assert pendingContext.committed() : "processed cluster state is not committed " + state;
contextsToRemove.add(pendingContext);
pendingContext.listener.onNewClusterStateProcessed();
} else if (state.version() >= pendingState.version()) {
logger.trace(
"processing pending state uuid[{}]/v[{}] together with state uuid[{}]/v[{}]",
pendingState.stateUUID(),
pendingState.version(),
state.stateUUID(),
state.version()
);
contextsToRemove.add(pendingContext);
if (pendingContext.committed()) {
pendingContext.listener.onNewClusterStateProcessed();
}
}
}
// now ack the processed state
pendingStates.removeAll(contextsToRemove);
assert findState(state.stateUUID()) == null : "state was marked as processed but can still be found in pending list " + state;
}
ClusterStateContext findState(String stateUUID) {
for (int i = 0; i < pendingStates.size(); i++) {
final ClusterStateContext context = pendingStates.get(i);
if (context.stateUUID().equals(stateUUID)) {
return context;
}
}
return null;
}
/** clear the incoming queue. any committed state will be failed
*/
public synchronized void failAllStatesAndClear(Exception reason) {
for (ClusterStateContext pendingState : pendingStates) {
if (pendingState.committed()) {
pendingState.listener.onNewClusterStateFailed(reason);
}
}
pendingStates.clear();
}
/**
* Gets the next committed state to process.
* <p>
* The method tries to batch operation by getting the cluster state the highest possible committed states
* which succeeds the first committed state in queue (i.e., it comes from the same master).
*/
public synchronized ClusterState getNextClusterStateToProcess() {
if (pendingStates.isEmpty()) {
return null;
}
ClusterStateContext stateToProcess = null;
int index = 0;
for (; index < pendingStates.size(); index++) {
ClusterStateContext potentialState = pendingStates.get(index);
if (potentialState.committed()) {
stateToProcess = potentialState;
break;
}
}
if (stateToProcess == null) {
return null;
}
// now try to find the highest committed state from the same master
for (; index < pendingStates.size(); index++) {
ClusterStateContext potentialState = pendingStates.get(index);
if (potentialState.state.supersedes(stateToProcess.state) && potentialState.committed()) {
// we found a new one
stateToProcess = potentialState;
}
}
assert stateToProcess.committed() : "should only return committed cluster state. found " + stateToProcess.state;
final ClusterState committedState = stateToProcess.state;
final CoordinationMetadata coordinationMetadata = committedState.coordinationMetadata();
if (coordinationMetadata.getLastAcceptedConfiguration().equals(coordinationMetadata.getLastCommittedConfiguration())) {
return committedState;
} else {
return ClusterState.builder(committedState)
.metadata(
Metadata.builder(committedState.metadata())
.coordinationMetadata(
CoordinationMetadata.builder(coordinationMetadata)
.lastCommittedConfiguration(committedState.getLastAcceptedConfiguration())
.build()
)
)
.build();
}
}
/** returns all pending states, committed or not */
public synchronized ClusterState[] pendingClusterStates() {
ArrayList<ClusterState> states = new ArrayList<>();
for (ClusterStateContext context : pendingStates) {
states.add(context.state);
}
return states.toArray(new ClusterState[states.size()]);
}
static class ClusterStateContext {
final ClusterState state;
StateProcessedListener listener;
ClusterStateContext(ClusterState clusterState) {
this.state = clusterState;
}
void markAsCommitted(StateProcessedListener listener) {
if (this.listener != null) {
throw new IllegalStateException(toString() + "is already committed");
}
this.listener = listener;
}
boolean committed() {
return listener != null;
}
public String stateUUID() {
return state.stateUUID();
}
@Override
public String toString() {
return String.format(Locale.ROOT, "[uuid[%s], v[%d], m[%s]]", stateUUID(), state.version(), state.nodes().getMasterNodeId());
}
}
public synchronized PendingClusterStateStats stats() {
// calculate committed cluster state
int committed = 0;
for (ClusterStateContext clusterStatsContext : pendingStates) {
if (clusterStatsContext.committed()) {
committed += 1;
}
}
return new PendingClusterStateStats(pendingStates.size(), pendingStates.size() - committed, committed);
}
}

View File

@ -1,41 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.opensearch.cluster.ClusterState;
public interface PingContextProvider {
/** return the current cluster state of the node */
ClusterState clusterState();
}

View File

@ -1,790 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.OpenSearchException;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.cluster.ClusterChangedEvent;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.Diff;
import org.opensearch.cluster.IncompatibleClusterStateVersionException;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.common.bytes.BytesReference;
import org.opensearch.common.compress.Compressor;
import org.opensearch.common.compress.CompressorFactory;
import org.opensearch.common.io.stream.BytesStreamOutput;
import org.opensearch.common.io.stream.InputStreamStreamInput;
import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.opensearch.common.io.stream.NamedWriteableRegistry;
import org.opensearch.common.io.stream.OutputStreamStreamOutput;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.AckClusterStatePublishResponseHandler;
import org.opensearch.discovery.BlockingClusterStatePublishResponseHandler;
import org.opensearch.discovery.Discovery;
import org.opensearch.discovery.DiscoverySettings;
import org.opensearch.cluster.coordination.FailedToCommitClusterStateException;
import org.opensearch.tasks.Task;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.BytesTransportRequest;
import org.opensearch.transport.EmptyTransportResponseHandler;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestHandler;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
public class PublishClusterStateAction {
private static final Logger logger = LogManager.getLogger(PublishClusterStateAction.class);
public static final String SEND_ACTION_NAME = "internal:discovery/zen/publish/send";
public static final String COMMIT_ACTION_NAME = "internal:discovery/zen/publish/commit";
// -> no need to put a timeout on the options, because we want the state response to eventually be received
// and not log an error if it arrives after the timeout
private final TransportRequestOptions stateRequestOptions = TransportRequestOptions.builder()
.withType(TransportRequestOptions.Type.STATE)
.build();
public interface IncomingClusterStateListener {
/**
* called when a new incoming cluster state has been received.
* Should validate the incoming state and throw an exception if it's not a valid successor state.
*/
void onIncomingClusterState(ClusterState incomingState);
/**
* called when a cluster state has been committed and is ready to be processed
*/
void onClusterStateCommitted(String stateUUID, ActionListener<Void> processedListener);
}
private final TransportService transportService;
private final NamedWriteableRegistry namedWriteableRegistry;
private final IncomingClusterStateListener incomingClusterStateListener;
private final DiscoverySettings discoverySettings;
private final AtomicLong fullClusterStateReceivedCount = new AtomicLong();
private final AtomicLong incompatibleClusterStateDiffReceivedCount = new AtomicLong();
private final AtomicLong compatibleClusterStateDiffReceivedCount = new AtomicLong();
public PublishClusterStateAction(
TransportService transportService,
NamedWriteableRegistry namedWriteableRegistry,
IncomingClusterStateListener incomingClusterStateListener,
DiscoverySettings discoverySettings
) {
this.transportService = transportService;
this.namedWriteableRegistry = namedWriteableRegistry;
this.incomingClusterStateListener = incomingClusterStateListener;
this.discoverySettings = discoverySettings;
transportService.registerRequestHandler(
SEND_ACTION_NAME,
ThreadPool.Names.SAME,
false,
false,
BytesTransportRequest::new,
new SendClusterStateRequestHandler()
);
transportService.registerRequestHandler(
COMMIT_ACTION_NAME,
ThreadPool.Names.SAME,
false,
false,
CommitClusterStateRequest::new,
new CommitClusterStateRequestHandler()
);
}
/**
* publishes a cluster change event to other nodes. if at least minMasterNodes acknowledge the change it is committed and will
* be processed by the master and the other nodes.
* <p>
* The method is guaranteed to throw a {@link FailedToCommitClusterStateException}
* if the change is not committed and should be rejected.
* Any other exception signals the something wrong happened but the change is committed.
*/
public void publish(final ClusterChangedEvent clusterChangedEvent, final int minMasterNodes, final Discovery.AckListener ackListener)
throws FailedToCommitClusterStateException {
final DiscoveryNodes nodes;
final SendingController sendingController;
final Set<DiscoveryNode> nodesToPublishTo;
final Map<Version, BytesReference> serializedStates;
final Map<Version, BytesReference> serializedDiffs;
final boolean sendFullVersion;
try {
nodes = clusterChangedEvent.state().nodes();
nodesToPublishTo = new HashSet<>(nodes.getSize());
DiscoveryNode localNode = nodes.getLocalNode();
final int totalMasterNodes = nodes.getMasterNodes().size();
for (final DiscoveryNode node : nodes) {
if (node.equals(localNode) == false) {
nodesToPublishTo.add(node);
}
}
sendFullVersion = !discoverySettings.getPublishDiff() || clusterChangedEvent.previousState() == null;
serializedStates = new HashMap<>();
serializedDiffs = new HashMap<>();
// we build these early as a best effort not to commit in the case of error.
// sadly this is not water tight as it may that a failed diff based publishing to a node
// will cause a full serialization based on an older version, which may fail after the
// change has been committed.
buildDiffAndSerializeStates(
clusterChangedEvent.state(),
clusterChangedEvent.previousState(),
nodesToPublishTo,
sendFullVersion,
serializedStates,
serializedDiffs
);
final BlockingClusterStatePublishResponseHandler publishResponseHandler = new AckClusterStatePublishResponseHandler(
nodesToPublishTo,
ackListener
);
sendingController = new SendingController(
clusterChangedEvent.state(),
minMasterNodes,
totalMasterNodes,
publishResponseHandler
);
} catch (Exception e) {
throw new FailedToCommitClusterStateException("unexpected error while preparing to publish", e);
}
try {
innerPublish(
clusterChangedEvent,
nodesToPublishTo,
sendingController,
ackListener,
sendFullVersion,
serializedStates,
serializedDiffs
);
} catch (FailedToCommitClusterStateException t) {
throw t;
} catch (Exception e) {
// try to fail committing, in cause it's still on going
if (sendingController.markAsFailed("unexpected error", e)) {
// signal the change should be rejected
throw new FailedToCommitClusterStateException("unexpected error", e);
} else {
throw e;
}
}
}
private void innerPublish(
final ClusterChangedEvent clusterChangedEvent,
final Set<DiscoveryNode> nodesToPublishTo,
final SendingController sendingController,
final Discovery.AckListener ackListener,
final boolean sendFullVersion,
final Map<Version, BytesReference> serializedStates,
final Map<Version, BytesReference> serializedDiffs
) {
final ClusterState clusterState = clusterChangedEvent.state();
final ClusterState previousState = clusterChangedEvent.previousState();
final TimeValue publishTimeout = discoverySettings.getPublishTimeout();
final long publishingStartInNanos = System.nanoTime();
for (final DiscoveryNode node : nodesToPublishTo) {
// try and serialize the cluster state once (or per version), so we don't serialize it
// per node when we send it over the wire, compress it while we are at it...
// we don't send full version if node didn't exist in the previous version of cluster state
if (sendFullVersion || !previousState.nodes().nodeExists(node)) {
sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController);
} else {
sendClusterStateDiff(clusterState, serializedDiffs, serializedStates, node, publishTimeout, sendingController);
}
}
sendingController.waitForCommit(discoverySettings.getCommitTimeout());
final long commitTime = System.nanoTime() - publishingStartInNanos;
ackListener.onCommit(TimeValue.timeValueNanos(commitTime));
try {
long timeLeftInNanos = Math.max(0, publishTimeout.nanos() - commitTime);
final BlockingClusterStatePublishResponseHandler publishResponseHandler = sendingController.getPublishResponseHandler();
sendingController.setPublishingTimedOut(!publishResponseHandler.awaitAllNodes(TimeValue.timeValueNanos(timeLeftInNanos)));
if (sendingController.getPublishingTimedOut()) {
DiscoveryNode[] pendingNodes = publishResponseHandler.pendingNodes();
// everyone may have just responded
if (pendingNodes.length > 0) {
logger.warn(
"timed out waiting for all nodes to process published state [{}] (timeout [{}], pending nodes: {})",
clusterState.version(),
publishTimeout,
pendingNodes
);
}
}
// The failure is logged under debug when a sending failed. we now log a summary.
Set<DiscoveryNode> failedNodes = publishResponseHandler.getFailedNodes();
if (failedNodes.isEmpty() == false) {
logger.warn(
"publishing cluster state with version [{}] failed for the following nodes: [{}]",
clusterChangedEvent.state().version(),
failedNodes
);
}
} catch (InterruptedException e) {
// ignore & restore interrupt
Thread.currentThread().interrupt();
}
}
private void buildDiffAndSerializeStates(
ClusterState clusterState,
ClusterState previousState,
Set<DiscoveryNode> nodesToPublishTo,
boolean sendFullVersion,
Map<Version, BytesReference> serializedStates,
Map<Version, BytesReference> serializedDiffs
) {
Diff<ClusterState> diff = null;
for (final DiscoveryNode node : nodesToPublishTo) {
try {
if (sendFullVersion || !previousState.nodes().nodeExists(node)) {
// will send a full reference
if (serializedStates.containsKey(node.getVersion()) == false) {
serializedStates.put(node.getVersion(), serializeFullClusterState(clusterState, node.getVersion()));
}
} else {
// will send a diff
if (diff == null) {
diff = clusterState.diff(previousState);
}
if (serializedDiffs.containsKey(node.getVersion()) == false) {
serializedDiffs.put(node.getVersion(), serializeDiffClusterState(diff, node.getVersion()));
}
}
} catch (IOException e) {
throw new OpenSearchException("failed to serialize cluster_state for publishing to node {}", e, node);
}
}
}
private void sendFullClusterState(
ClusterState clusterState,
Map<Version, BytesReference> serializedStates,
DiscoveryNode node,
TimeValue publishTimeout,
SendingController sendingController
) {
BytesReference bytes = serializedStates.get(node.getVersion());
if (bytes == null) {
try {
bytes = serializeFullClusterState(clusterState, node.getVersion());
serializedStates.put(node.getVersion(), bytes);
} catch (Exception e) {
logger.warn(() -> new ParameterizedMessage("failed to serialize cluster_state before publishing it to node {}", node), e);
sendingController.onNodeSendFailed(node, e);
return;
}
}
sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, false, serializedStates);
}
private void sendClusterStateDiff(
ClusterState clusterState,
Map<Version, BytesReference> serializedDiffs,
Map<Version, BytesReference> serializedStates,
DiscoveryNode node,
TimeValue publishTimeout,
SendingController sendingController
) {
BytesReference bytes = serializedDiffs.get(node.getVersion());
assert bytes != null : "failed to find serialized diff for node " + node + " of version [" + node.getVersion() + "]";
sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, true, serializedStates);
}
private void sendClusterStateToNode(
final ClusterState clusterState,
BytesReference bytes,
final DiscoveryNode node,
final TimeValue publishTimeout,
final SendingController sendingController,
final boolean sendDiffs,
final Map<Version, BytesReference> serializedStates
) {
try {
transportService.sendRequest(
node,
SEND_ACTION_NAME,
new BytesTransportRequest(bytes, node.getVersion()),
stateRequestOptions,
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override
public void handleResponse(TransportResponse.Empty response) {
if (sendingController.getPublishingTimedOut()) {
logger.debug(
"node {} responded for cluster state [{}] (took longer than [{}])",
node,
clusterState.version(),
publishTimeout
);
}
sendingController.onNodeSendAck(node);
}
@Override
public void handleException(TransportException exp) {
if (sendDiffs && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) {
logger.debug("resending full cluster state to node {} reason {}", node, exp.getDetailedMessage());
sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController);
} else {
logger.debug(() -> new ParameterizedMessage("failed to send cluster state to {}", node), exp);
sendingController.onNodeSendFailed(node, exp);
}
}
}
);
} catch (Exception e) {
logger.warn(() -> new ParameterizedMessage("error sending cluster state to {}", node), e);
sendingController.onNodeSendFailed(node, e);
}
}
private void sendCommitToNode(final DiscoveryNode node, final ClusterState clusterState, final SendingController sendingController) {
try {
logger.trace(
"sending commit for cluster state (uuid: [{}], version [{}]) to [{}]",
clusterState.stateUUID(),
clusterState.version(),
node
);
transportService.sendRequest(
node,
COMMIT_ACTION_NAME,
new CommitClusterStateRequest(clusterState.stateUUID()),
stateRequestOptions,
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override
public void handleResponse(TransportResponse.Empty response) {
if (sendingController.getPublishingTimedOut()) {
logger.debug("node {} responded to cluster state commit [{}]", node, clusterState.version());
}
sendingController.getPublishResponseHandler().onResponse(node);
}
@Override
public void handleException(TransportException exp) {
logger.debug(
() -> new ParameterizedMessage(
"failed to commit cluster state (uuid [{}], version [{}]) to {}",
clusterState.stateUUID(),
clusterState.version(),
node
),
exp
);
sendingController.getPublishResponseHandler().onFailure(node, exp);
}
}
);
} catch (Exception t) {
logger.warn(
() -> new ParameterizedMessage(
"error sending cluster state commit (uuid [{}], version [{}]) to {}",
clusterState.stateUUID(),
clusterState.version(),
node
),
t
);
sendingController.getPublishResponseHandler().onFailure(node, t);
}
}
public static BytesReference serializeFullClusterState(ClusterState clusterState, Version nodeVersion) throws IOException {
BytesStreamOutput bStream = new BytesStreamOutput();
try (StreamOutput stream = new OutputStreamStreamOutput(CompressorFactory.COMPRESSOR.threadLocalOutputStream(bStream))) {
stream.setVersion(nodeVersion);
stream.writeBoolean(true);
clusterState.writeTo(stream);
}
return bStream.bytes();
}
public static BytesReference serializeDiffClusterState(Diff diff, Version nodeVersion) throws IOException {
BytesStreamOutput bStream = new BytesStreamOutput();
try (StreamOutput stream = new OutputStreamStreamOutput(CompressorFactory.COMPRESSOR.threadLocalOutputStream(bStream))) {
stream.setVersion(nodeVersion);
stream.writeBoolean(false);
diff.writeTo(stream);
}
return bStream.bytes();
}
private Object lastSeenClusterStateMutex = new Object();
private ClusterState lastSeenClusterState;
protected void handleIncomingClusterStateRequest(BytesTransportRequest request, TransportChannel channel) throws IOException {
Compressor compressor = CompressorFactory.compressor(request.bytes());
StreamInput in = request.bytes().streamInput();
final ClusterState incomingState;
synchronized (lastSeenClusterStateMutex) {
try {
if (compressor != null) {
in = new InputStreamStreamInput(compressor.threadLocalInputStream(in));
}
in = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry);
in.setVersion(request.version());
// If true we received full cluster state - otherwise diffs
if (in.readBoolean()) {
incomingState = ClusterState.readFrom(in, transportService.getLocalNode());
fullClusterStateReceivedCount.incrementAndGet();
logger.debug(
"received full cluster state version [{}] with size [{}]",
incomingState.version(),
request.bytes().length()
);
} else if (lastSeenClusterState != null) {
Diff<ClusterState> diff = ClusterState.readDiffFrom(in, lastSeenClusterState.nodes().getLocalNode());
incomingState = diff.apply(lastSeenClusterState);
compatibleClusterStateDiffReceivedCount.incrementAndGet();
logger.debug(
"received diff cluster state version [{}] with uuid [{}], diff size [{}]",
incomingState.version(),
incomingState.stateUUID(),
request.bytes().length()
);
} else {
logger.debug("received diff for but don't have any local cluster state - requesting full state");
throw new IncompatibleClusterStateVersionException("have no local cluster state");
}
} catch (IncompatibleClusterStateVersionException e) {
incompatibleClusterStateDiffReceivedCount.incrementAndGet();
throw e;
} catch (Exception e) {
logger.warn("unexpected error while deserializing an incoming cluster state", e);
throw e;
} finally {
IOUtils.close(in);
}
incomingClusterStateListener.onIncomingClusterState(incomingState);
lastSeenClusterState = incomingState;
}
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}
protected void handleCommitRequest(CommitClusterStateRequest request, final TransportChannel channel) {
incomingClusterStateListener.onClusterStateCommitted(request.stateUUID, new ActionListener<Void>() {
@Override
public void onResponse(Void ignore) {
try {
// send a response to the master to indicate that this cluster state has been processed post committing it.
channel.sendResponse(TransportResponse.Empty.INSTANCE);
} catch (Exception e) {
logger.debug("failed to send response on cluster state processed", e);
onFailure(e);
}
}
@Override
public void onFailure(Exception e) {
try {
channel.sendResponse(e);
} catch (Exception inner) {
inner.addSuppressed(e);
logger.debug("failed to send response on cluster state processed", inner);
}
}
});
}
private class SendClusterStateRequestHandler implements TransportRequestHandler<BytesTransportRequest> {
@Override
public void messageReceived(BytesTransportRequest request, final TransportChannel channel, Task task) throws Exception {
handleIncomingClusterStateRequest(request, channel);
}
}
private class CommitClusterStateRequestHandler implements TransportRequestHandler<CommitClusterStateRequest> {
@Override
public void messageReceived(CommitClusterStateRequest request, final TransportChannel channel, Task task) throws Exception {
handleCommitRequest(request, channel);
}
}
public static class CommitClusterStateRequest extends TransportRequest {
public String stateUUID;
public CommitClusterStateRequest(StreamInput in) throws IOException {
super(in);
stateUUID = in.readString();
}
public CommitClusterStateRequest(String stateUUID) {
this.stateUUID = stateUUID;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(stateUUID);
}
}
/**
* Coordinates acknowledgments of the sent cluster state from the different nodes. Commits the change
* after `minimum_master_nodes` have successfully responded or fails the entire change. After committing
* the cluster state, will trigger a commit message to all nodes that responded previously and responds immediately
* to all future acknowledgments.
*/
class SendingController {
private final ClusterState clusterState;
public BlockingClusterStatePublishResponseHandler getPublishResponseHandler() {
return publishResponseHandler;
}
private final BlockingClusterStatePublishResponseHandler publishResponseHandler;
final ArrayList<DiscoveryNode> sendAckedBeforeCommit = new ArrayList<>();
// writes and reads of these are protected under synchronization
final CountDownLatch committedOrFailedLatch; // 0 count indicates that a decision was made w.r.t committing or failing
boolean committed; // true if cluster state was committed
int neededMastersToCommit; // number of master nodes acks still needed before committing
int pendingMasterNodes; // how many master node still need to respond
// an external marker to note that the publishing process is timed out. This is useful for proper logging.
final AtomicBoolean publishingTimedOut = new AtomicBoolean();
private SendingController(
ClusterState clusterState,
int minMasterNodes,
int totalMasterNodes,
BlockingClusterStatePublishResponseHandler publishResponseHandler
) {
this.clusterState = clusterState;
this.publishResponseHandler = publishResponseHandler;
this.neededMastersToCommit = Math.max(0, minMasterNodes - 1); // we are one of the master nodes
this.pendingMasterNodes = totalMasterNodes - 1;
if (this.neededMastersToCommit > this.pendingMasterNodes) {
throw new FailedToCommitClusterStateException(
"not enough masters to ack sent cluster state." + "[{}] needed , have [{}]",
neededMastersToCommit,
pendingMasterNodes
);
}
this.committed = neededMastersToCommit == 0;
this.committedOrFailedLatch = new CountDownLatch(committed ? 0 : 1);
}
public void waitForCommit(TimeValue commitTimeout) {
boolean timedout = false;
try {
timedout = committedOrFailedLatch.await(commitTimeout.millis(), TimeUnit.MILLISECONDS) == false;
} catch (InterruptedException e) {
// the commit check bellow will either translate to an exception or we are committed and we can safely continue
}
if (timedout) {
markAsFailed("timed out waiting for commit (commit timeout [" + commitTimeout + "])");
}
if (isCommitted() == false) {
throw new FailedToCommitClusterStateException(
"{} enough masters to ack sent cluster state. [{}] left",
timedout ? "timed out while waiting for" : "failed to get",
neededMastersToCommit
);
}
}
public synchronized boolean isCommitted() {
return committed;
}
public synchronized void onNodeSendAck(DiscoveryNode node) {
if (committed) {
assert sendAckedBeforeCommit.isEmpty();
sendCommitToNode(node, clusterState, this);
} else if (committedOrFailed()) {
logger.trace("ignoring ack from [{}] for cluster state version [{}]. already failed", node, clusterState.version());
} else {
// we're still waiting
sendAckedBeforeCommit.add(node);
if (node.isMasterNode()) {
checkForCommitOrFailIfNoPending(node);
}
}
}
private synchronized boolean committedOrFailed() {
return committedOrFailedLatch.getCount() == 0;
}
/**
* check if enough master node responded to commit the change. fails the commit
* if there are no more pending master nodes but not enough acks to commit.
*/
private synchronized void checkForCommitOrFailIfNoPending(DiscoveryNode masterNode) {
logger.trace(
"master node {} acked cluster state version [{}]. processing ... (current pending [{}], needed [{}])",
masterNode,
clusterState.version(),
pendingMasterNodes,
neededMastersToCommit
);
neededMastersToCommit--;
if (neededMastersToCommit == 0) {
if (markAsCommitted()) {
for (DiscoveryNode nodeToCommit : sendAckedBeforeCommit) {
sendCommitToNode(nodeToCommit, clusterState, this);
}
sendAckedBeforeCommit.clear();
}
}
decrementPendingMasterAcksAndChangeForFailure();
}
private synchronized void decrementPendingMasterAcksAndChangeForFailure() {
pendingMasterNodes--;
if (pendingMasterNodes == 0 && neededMastersToCommit > 0) {
markAsFailed("no more pending master nodes, but failed to reach needed acks ([" + neededMastersToCommit + "] left)");
}
}
public synchronized void onNodeSendFailed(DiscoveryNode node, Exception e) {
if (node.isMasterNode()) {
logger.trace(
"master node {} failed to ack cluster state version [{}]. " + "processing ... (current pending [{}], needed [{}])",
node,
clusterState.version(),
pendingMasterNodes,
neededMastersToCommit
);
decrementPendingMasterAcksAndChangeForFailure();
}
publishResponseHandler.onFailure(node, e);
}
/**
* tries and commit the current state, if a decision wasn't made yet
*
* @return true if successful
*/
private synchronized boolean markAsCommitted() {
if (committedOrFailed()) {
return committed;
}
logger.trace("committing version [{}]", clusterState.version());
committed = true;
committedOrFailedLatch.countDown();
return true;
}
/**
* tries marking the publishing as failed, if a decision wasn't made yet
*
* @return true if the publishing was failed and the cluster state is *not* committed
**/
private synchronized boolean markAsFailed(String details, Exception reason) {
if (committedOrFailed()) {
return committed == false;
}
logger.trace(() -> new ParameterizedMessage("failed to commit version [{}]. {}", clusterState.version(), details), reason);
committed = false;
committedOrFailedLatch.countDown();
return true;
}
/**
* tries marking the publishing as failed, if a decision wasn't made yet
*
* @return true if the publishing was failed and the cluster state is *not* committed
**/
private synchronized boolean markAsFailed(String reason) {
if (committedOrFailed()) {
return committed == false;
}
logger.trace("failed to commit version [{}]. {}", clusterState.version(), reason);
committed = false;
committedOrFailedLatch.countDown();
return true;
}
public boolean getPublishingTimedOut() {
return publishingTimedOut.get();
}
public void setPublishingTimedOut(boolean isTimedOut) {
publishingTimedOut.set(isTimedOut);
}
}
public PublishClusterStateStats stats() {
return new PublishClusterStateStats(
fullClusterStateReceivedCount.get(),
incompatibleClusterStateDiffReceivedCount.get(),
compatibleClusterStateDiffReceivedCount.get()
);
}
}

View File

@ -1,637 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.store.AlreadyClosedException;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.lease.Releasables;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.transport.TransportAddress;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.CancellableThreads;
import org.opensearch.common.util.CollectionUtils;
import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.common.util.concurrent.ConcurrentCollections;
import org.opensearch.common.util.concurrent.OpenSearchExecutors;
import org.opensearch.common.util.concurrent.OpenSearchThreadPoolExecutor;
import org.opensearch.common.util.concurrent.KeyedLock;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.SeedHostsProvider;
import org.opensearch.discovery.SeedHostsResolver;
import org.opensearch.node.Node;
import org.opensearch.tasks.Task;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.ConnectTransportException;
import org.opensearch.transport.ConnectionProfile;
import org.opensearch.transport.NodeNotConnectedException;
import org.opensearch.transport.RemoteTransportException;
import org.opensearch.transport.Transport.Connection;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportException;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestHandler;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportResponseHandler;
import org.opensearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.opensearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
public class UnicastZenPing implements ZenPing {
private static final Logger logger = LogManager.getLogger(UnicastZenPing.class);
public static final String ACTION_NAME = "internal:discovery/zen/unicast";
private final ThreadPool threadPool;
private final TransportService transportService;
private final ClusterName clusterName;
private final PingContextProvider contextProvider;
private final AtomicInteger pingingRoundIdGenerator = new AtomicInteger();
private final Map<Integer, PingingRound> activePingingRounds = newConcurrentMap();
// a list of temporal responses a node will return for a request (holds responses from other nodes)
private final Queue<PingResponse> temporalResponses = ConcurrentCollections.newQueue();
private final SeedHostsProvider hostsProvider;
protected final OpenSearchThreadPoolExecutor unicastZenPingExecutorService;
private final TimeValue resolveTimeout;
private final String nodeName;
private volatile boolean closed = false;
public UnicastZenPing(
Settings settings,
ThreadPool threadPool,
TransportService transportService,
SeedHostsProvider seedHostsProvider,
PingContextProvider contextProvider
) {
this.threadPool = threadPool;
this.transportService = transportService;
this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
this.hostsProvider = seedHostsProvider;
this.contextProvider = contextProvider;
final int concurrentConnects = SeedHostsResolver.getMaxConcurrentResolvers(settings);
resolveTimeout = SeedHostsResolver.getResolveTimeout(settings);
nodeName = Node.NODE_NAME_SETTING.get(settings);
logger.debug("using max_concurrent_resolvers [{}], resolver timeout [{}]", concurrentConnects, resolveTimeout);
transportService.registerRequestHandler(
ACTION_NAME,
ThreadPool.Names.SAME,
UnicastPingRequest::new,
new UnicastPingRequestHandler()
);
final ThreadFactory threadFactory = OpenSearchExecutors.daemonThreadFactory(settings, "[unicast_connect]");
unicastZenPingExecutorService = OpenSearchExecutors.newScaling(
nodeName + "/" + "unicast_connect",
0,
concurrentConnects,
60,
TimeUnit.SECONDS,
threadFactory,
threadPool.getThreadContext()
);
}
private SeedHostsProvider.HostsResolver createHostsResolver() {
return hosts -> SeedHostsResolver.resolveHostsLists(new CancellableThreads() {
public void execute(Interruptible interruptible) {
try {
interruptible.run();
} catch (InterruptedException e) {
throw new CancellableThreads.ExecutionCancelledException("interrupted by " + e);
}
}
}, unicastZenPingExecutorService, logger, hosts, transportService, resolveTimeout);
}
@Override
public void close() {
ThreadPool.terminate(unicastZenPingExecutorService, 10, TimeUnit.SECONDS);
Releasables.close(activePingingRounds.values());
closed = true;
}
@Override
public void start() {}
/**
* Clears the list of cached ping responses.
*/
public void clearTemporalResponses() {
temporalResponses.clear();
}
/**
* Sends three rounds of pings notifying the specified {@link Consumer} when pinging is complete. Pings are sent after resolving
* configured unicast hosts to their IP address (subject to DNS caching within the JVM). A batch of pings is sent, then another batch
* of pings is sent at half the specified {@link TimeValue}, and then another batch of pings is sent at the specified {@link TimeValue}.
* The pings that are sent carry a timeout of 1.25 times the specified {@link TimeValue}. When pinging each node, a connection and
* handshake is performed, with a connection timeout of the specified {@link TimeValue}.
*
* @param resultsConsumer the callback when pinging is complete
* @param duration the timeout for various components of the pings
*/
@Override
public void ping(final Consumer<PingCollection> resultsConsumer, final TimeValue duration) {
ping(resultsConsumer, duration, duration);
}
/**
* a variant of {@link #ping(Consumer, TimeValue)}, but allows separating the scheduling duration
* from the duration used for request level time outs. This is useful for testing
*/
protected void ping(final Consumer<PingCollection> resultsConsumer, final TimeValue scheduleDuration, final TimeValue requestDuration) {
final List<TransportAddress> seedAddresses = new ArrayList<>();
seedAddresses.addAll(hostsProvider.getSeedAddresses(createHostsResolver()));
final DiscoveryNodes nodes = contextProvider.clusterState().nodes();
// add all possible master nodes that were active in the last known cluster configuration
for (ObjectCursor<DiscoveryNode> masterNode : nodes.getMasterNodes().values()) {
seedAddresses.add(masterNode.value.getAddress());
}
final ConnectionProfile connectionProfile = ConnectionProfile.buildSingleChannelProfile(
TransportRequestOptions.Type.REG,
requestDuration,
requestDuration,
TimeValue.MINUS_ONE,
null
);
final PingingRound pingingRound = new PingingRound(
pingingRoundIdGenerator.incrementAndGet(),
seedAddresses,
resultsConsumer,
nodes.getLocalNode(),
connectionProfile
);
activePingingRounds.put(pingingRound.id(), pingingRound);
final AbstractRunnable pingSender = new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
if (e instanceof AlreadyClosedException == false) {
logger.warn("unexpected error while pinging", e);
}
}
@Override
protected void doRun() throws Exception {
sendPings(requestDuration, pingingRound);
}
};
threadPool.generic().execute(pingSender);
threadPool.schedule(pingSender, TimeValue.timeValueMillis(scheduleDuration.millis() / 3), ThreadPool.Names.GENERIC);
threadPool.schedule(pingSender, TimeValue.timeValueMillis(scheduleDuration.millis() / 3 * 2), ThreadPool.Names.GENERIC);
threadPool.schedule(new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
finishPingingRound(pingingRound);
}
@Override
public void onFailure(Exception e) {
logger.warn("unexpected error while finishing pinging round", e);
}
}, scheduleDuration, ThreadPool.Names.GENERIC);
}
// for testing
protected void finishPingingRound(PingingRound pingingRound) {
pingingRound.close();
}
protected class PingingRound implements Releasable {
private final int id;
private final Map<TransportAddress, Connection> tempConnections = new HashMap<>();
private final KeyedLock<TransportAddress> connectionLock = new KeyedLock<>(true);
private final PingCollection pingCollection;
private final List<TransportAddress> seedAddresses;
private final Consumer<PingCollection> pingListener;
private final DiscoveryNode localNode;
private final ConnectionProfile connectionProfile;
private AtomicBoolean closed = new AtomicBoolean(false);
PingingRound(
int id,
List<TransportAddress> seedAddresses,
Consumer<PingCollection> resultsConsumer,
DiscoveryNode localNode,
ConnectionProfile connectionProfile
) {
this.id = id;
this.seedAddresses = Collections.unmodifiableList(seedAddresses.stream().distinct().collect(Collectors.toList()));
this.pingListener = resultsConsumer;
this.localNode = localNode;
this.connectionProfile = connectionProfile;
this.pingCollection = new PingCollection();
}
public int id() {
return this.id;
}
public boolean isClosed() {
return this.closed.get();
}
public List<TransportAddress> getSeedAddresses() {
ensureOpen();
return seedAddresses;
}
public Connection getOrConnect(DiscoveryNode node) throws IOException {
Connection result;
try (Releasable ignore = connectionLock.acquire(node.getAddress())) {
result = tempConnections.get(node.getAddress());
if (result == null) {
ensureOpen();
boolean success = false;
logger.trace("[{}] opening connection to [{}]", id(), node);
result = transportService.openConnection(node, connectionProfile);
try {
Connection finalResult = result;
PlainActionFuture.get(
fut -> transportService.handshake(
finalResult,
connectionProfile.getHandshakeTimeout().millis(),
ActionListener.map(fut, x -> null)
)
);
synchronized (this) {
// acquire lock and check if closed, to prevent leaving an open connection after closing
ensureOpen();
Connection existing = tempConnections.put(node.getAddress(), result);
assert existing == null;
success = true;
}
} finally {
if (success == false) {
logger.trace("[{}] closing connection to [{}] due to failure", id(), node);
IOUtils.closeWhileHandlingException(result);
}
}
}
}
return result;
}
private void ensureOpen() {
if (isClosed()) {
throw new AlreadyClosedException("pinging round [" + id + "] is finished");
}
}
public void addPingResponseToCollection(PingResponse pingResponse) {
if (localNode.equals(pingResponse.node()) == false) {
pingCollection.addPing(pingResponse);
}
}
@Override
public void close() {
List<Connection> toClose = null;
synchronized (this) {
if (closed.compareAndSet(false, true)) {
activePingingRounds.remove(id);
toClose = new ArrayList<>(tempConnections.values());
tempConnections.clear();
}
}
if (toClose != null) {
// we actually closed
try {
pingListener.accept(pingCollection);
} finally {
IOUtils.closeWhileHandlingException(toClose);
}
}
}
public ConnectionProfile getConnectionProfile() {
return connectionProfile;
}
}
protected void sendPings(final TimeValue timeout, final PingingRound pingingRound) {
final ClusterState lastState = contextProvider.clusterState();
final UnicastPingRequest pingRequest = new UnicastPingRequest(pingingRound.id(), timeout, createPingResponse(lastState));
List<TransportAddress> temporalAddresses = temporalResponses.stream().map(pingResponse -> {
assert clusterName.equals(pingResponse.clusterName()) : "got a ping request from a different cluster. expected "
+ clusterName
+ " got "
+ pingResponse.clusterName();
return pingResponse.node().getAddress();
}).collect(Collectors.toList());
final Stream<TransportAddress> uniqueAddresses = Stream.concat(pingingRound.getSeedAddresses().stream(), temporalAddresses.stream())
.distinct();
// resolve what we can via the latest cluster state
final Set<DiscoveryNode> nodesToPing = uniqueAddresses.map(address -> {
DiscoveryNode foundNode = lastState.nodes().findByAddress(address);
if (foundNode != null && transportService.nodeConnected(foundNode)) {
return foundNode;
} else {
return new DiscoveryNode(
address.toString(),
address,
emptyMap(),
emptySet(),
Version.CURRENT.minimumCompatibilityVersion()
);
}
}).collect(Collectors.toSet());
nodesToPing.forEach(node -> sendPingRequestToNode(node, timeout, pingingRound, pingRequest));
}
private void sendPingRequestToNode(
final DiscoveryNode node,
TimeValue timeout,
final PingingRound pingingRound,
final UnicastPingRequest pingRequest
) {
submitToExecutor(new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
Connection connection = null;
if (transportService.nodeConnected(node)) {
try {
// concurrency can still cause disconnects
connection = transportService.getConnection(node);
} catch (NodeNotConnectedException e) {
logger.trace("[{}] node [{}] just disconnected, will create a temp connection", pingingRound.id(), node);
}
}
if (connection == null) {
connection = pingingRound.getOrConnect(node);
}
logger.trace("[{}] sending to {}", pingingRound.id(), node);
transportService.sendRequest(
connection,
ACTION_NAME,
pingRequest,
TransportRequestOptions.builder().withTimeout((long) (timeout.millis() * 1.25)).build(),
getPingResponseHandler(pingingRound, node)
);
}
@Override
public void onFailure(Exception e) {
if (e instanceof ConnectTransportException || e instanceof AlreadyClosedException) {
// can't connect to the node - this is more common path!
logger.trace(() -> new ParameterizedMessage("[{}] failed to ping {}", pingingRound.id(), node), e);
} else if (e instanceof RemoteTransportException) {
// something went wrong on the other side
logger.debug(
() -> new ParameterizedMessage("[{}] received a remote error as a response to ping {}", pingingRound.id(), node),
e
);
} else {
logger.warn(() -> new ParameterizedMessage("[{}] failed send ping to {}", pingingRound.id(), node), e);
}
}
@Override
public void onRejection(Exception e) {
// The RejectedExecutionException can come from the fact unicastZenPingExecutorService is at its max down in sendPings
// But don't bail here, we can retry later on after the send ping has been scheduled.
logger.debug("Ping execution rejected", e);
}
});
}
// for testing
protected void submitToExecutor(AbstractRunnable abstractRunnable) {
unicastZenPingExecutorService.execute(abstractRunnable);
}
// for testing
protected TransportResponseHandler<UnicastPingResponse> getPingResponseHandler(
final PingingRound pingingRound,
final DiscoveryNode node
) {
return new TransportResponseHandler<UnicastPingResponse>() {
@Override
public UnicastPingResponse read(StreamInput in) throws IOException {
return new UnicastPingResponse(in);
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
@Override
public void handleResponse(UnicastPingResponse response) {
logger.trace("[{}] received response from {}: {}", pingingRound.id(), node, Arrays.toString(response.pingResponses));
if (pingingRound.isClosed()) {
if (logger.isTraceEnabled()) {
logger.trace("[{}] skipping received response from {}. already closed", pingingRound.id(), node);
}
} else {
Stream.of(response.pingResponses).forEach(pingingRound::addPingResponseToCollection);
}
}
@Override
public void handleException(TransportException exp) {
if (exp instanceof ConnectTransportException
|| exp.getCause() instanceof ConnectTransportException
|| exp.getCause() instanceof AlreadyClosedException) {
// ok, not connected...
logger.trace(() -> new ParameterizedMessage("failed to connect to {}", node), exp);
} else if (closed == false) {
logger.warn(() -> new ParameterizedMessage("failed to send ping to [{}]", node), exp);
}
}
};
}
private UnicastPingResponse handlePingRequest(final UnicastPingRequest request) {
assert clusterName.equals(request.pingResponse.clusterName()) : "got a ping request from a different cluster. expected "
+ clusterName
+ " got "
+ request.pingResponse.clusterName();
temporalResponses.add(request.pingResponse);
// add to any ongoing pinging
activePingingRounds.values().forEach(p -> p.addPingResponseToCollection(request.pingResponse));
threadPool.schedule(
() -> temporalResponses.remove(request.pingResponse),
TimeValue.timeValueMillis(request.timeout.millis() * 2),
ThreadPool.Names.SAME
);
List<PingResponse> pingResponses = CollectionUtils.iterableAsArrayList(temporalResponses);
pingResponses.add(createPingResponse(contextProvider.clusterState()));
return new UnicastPingResponse(request.id, pingResponses.toArray(new PingResponse[pingResponses.size()]));
}
class UnicastPingRequestHandler implements TransportRequestHandler<UnicastPingRequest> {
@Override
public void messageReceived(UnicastPingRequest request, TransportChannel channel, Task task) throws Exception {
if (closed) {
throw new AlreadyClosedException("node is shutting down");
}
if (request.pingResponse.clusterName().equals(clusterName)) {
channel.sendResponse(handlePingRequest(request));
} else {
throw new IllegalStateException(
String.format(
Locale.ROOT,
"mismatched cluster names; request: [%s], local: [%s]",
request.pingResponse.clusterName().value(),
clusterName.value()
)
);
}
}
}
public static class UnicastPingRequest extends TransportRequest {
public final int id;
public final TimeValue timeout;
public final PingResponse pingResponse;
public UnicastPingRequest(int id, TimeValue timeout, PingResponse pingResponse) {
this.id = id;
this.timeout = timeout;
this.pingResponse = pingResponse;
}
public UnicastPingRequest(StreamInput in) throws IOException {
super(in);
id = in.readInt();
timeout = in.readTimeValue();
pingResponse = new PingResponse(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeInt(id);
out.writeTimeValue(timeout);
pingResponse.writeTo(out);
}
}
private PingResponse createPingResponse(ClusterState clusterState) {
DiscoveryNodes discoNodes = clusterState.nodes();
return new PingResponse(discoNodes.getLocalNode(), discoNodes.getMasterNode(), clusterState);
}
public static class UnicastPingResponse extends TransportResponse {
final int id;
public final PingResponse[] pingResponses;
public UnicastPingResponse(int id, PingResponse[] pingResponses) {
this.id = id;
this.pingResponses = pingResponses;
}
public UnicastPingResponse(StreamInput in) throws IOException {
id = in.readInt();
pingResponses = new PingResponse[in.readVInt()];
for (int i = 0; i < pingResponses.length; i++) {
pingResponses[i] = new PingResponse(in);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeInt(id);
out.writeVInt(pingResponses.length);
for (PingResponse pingResponse : pingResponses) {
pingResponse.writeTo(out);
}
}
}
protected Version getVersion() {
return Version.CURRENT; // for tests
}
}

View File

@ -1,223 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.io.stream.Writeable;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.unit.TimeValue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import static org.opensearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
public interface ZenPing extends Releasable {
void start();
void ping(Consumer<PingCollection> resultsConsumer, TimeValue timeout);
class PingResponse implements Writeable {
/**
* An ID of a ping response that was generated on behalf of another node. Needs to be less than all other ping IDs so that fake ping
* responses don't override real ones.
*/
public static long FAKE_PING_ID = -1;
private static final AtomicLong idGenerator = new AtomicLong();
// an always increasing unique identifier for this ping response.
// lower values means older pings.
private final long id;
private final ClusterName clusterName;
private final DiscoveryNode node;
private final DiscoveryNode master;
private final long clusterStateVersion;
/**
* @param node the node which this ping describes
* @param master the current master of the node
* @param clusterName the cluster name of the node
* @param clusterStateVersion the current cluster state version of that node
* ({@link ElectMasterService.MasterCandidate#UNRECOVERED_CLUSTER_VERSION} for not recovered)
*/
public PingResponse(DiscoveryNode node, DiscoveryNode master, ClusterName clusterName, long clusterStateVersion) {
this(idGenerator.incrementAndGet(), node, master, clusterName, clusterStateVersion);
}
/**
* @param id the ping's ID
* @param node the node which this ping describes
* @param master the current master of the node
* @param clusterName the cluster name of the node
* @param clusterStateVersion the current cluster state version of that node
* ({@link ElectMasterService.MasterCandidate#UNRECOVERED_CLUSTER_VERSION} for not recovered)
*/
public PingResponse(long id, DiscoveryNode node, DiscoveryNode master, ClusterName clusterName, long clusterStateVersion) {
this.id = id;
this.node = node;
this.master = master;
this.clusterName = clusterName;
this.clusterStateVersion = clusterStateVersion;
}
public PingResponse(DiscoveryNode node, DiscoveryNode master, ClusterState state) {
this(
node,
master,
state.getClusterName(),
state.blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK)
? ElectMasterService.MasterCandidate.UNRECOVERED_CLUSTER_VERSION
: state.version()
);
}
PingResponse(StreamInput in) throws IOException {
this.clusterName = new ClusterName(in);
this.node = new DiscoveryNode(in);
this.master = in.readOptionalWriteable(DiscoveryNode::new);
this.clusterStateVersion = in.readLong();
this.id = in.readLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
clusterName.writeTo(out);
node.writeTo(out);
out.writeOptionalWriteable(master);
out.writeLong(clusterStateVersion);
out.writeLong(id);
}
/**
* an always increasing unique identifier for this ping response.
* lower values means older pings.
*/
public long id() {
return this.id;
}
/**
* the name of the cluster this node belongs to
*/
public ClusterName clusterName() {
return this.clusterName;
}
/** the node which this ping describes */
public DiscoveryNode node() {
return node;
}
/** the current master of the node */
public DiscoveryNode master() {
return master;
}
/**
* the current cluster state version of that node ({@link ElectMasterService.MasterCandidate#UNRECOVERED_CLUSTER_VERSION}
* for not recovered) */
public long getClusterStateVersion() {
return clusterStateVersion;
}
@Override
public String toString() {
return "ping_response{node ["
+ node
+ "], id["
+ id
+ "], master ["
+ master
+ "],"
+ "cluster_state_version ["
+ clusterStateVersion
+ "], cluster_name["
+ clusterName.value()
+ "]}";
}
}
/**
* a utility collection of pings where only the most recent ping is stored per node
*/
class PingCollection {
Map<DiscoveryNode, PingResponse> pings;
public PingCollection() {
pings = new HashMap<>();
}
/**
* adds a ping if newer than previous pings from the same node
*
* @return true if added, false o.w.
*/
public synchronized boolean addPing(PingResponse ping) {
PingResponse existingResponse = pings.get(ping.node());
// in case both existing and new ping have the same id (probably because they come
// from nodes from version <1.4.0) we prefer to use the last added one.
if (existingResponse == null || existingResponse.id() <= ping.id()) {
pings.put(ping.node(), ping);
return true;
}
return false;
}
/** serialize current pings to a list. It is guaranteed that the list contains one ping response per node */
public synchronized List<PingResponse> toList() {
return new ArrayList<>(pings.values());
}
/** the number of nodes for which there are known pings */
public synchronized int size() {
return pings.size();
}
}
}

View File

@ -42,7 +42,6 @@ import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.index.Index;
import java.util.Arrays;
@ -56,8 +55,6 @@ public class Gateway {
private final TransportNodesListGatewayMetaState listGatewayMetaState;
private final int minimumMasterNodes;
public Gateway(
final Settings settings,
final ClusterService clusterService,
@ -65,7 +62,6 @@ public class Gateway {
) {
this.clusterService = clusterService;
this.listGatewayMetaState = listGatewayMetaState;
this.minimumMasterNodes = ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
}
public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException {
@ -73,7 +69,7 @@ public class Gateway {
logger.trace("performing state recovery from {}", Arrays.toString(nodesIds));
final TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds, null).actionGet();
final int requiredAllocation = Math.max(1, minimumMasterNodes);
final int requiredAllocation = 1;
if (nodesState.hasFailures()) {
for (final FailedNodeException failedNodeException : nodesState.failures()) {

View File

@ -62,7 +62,6 @@ import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.common.util.concurrent.OpenSearchExecutors;
import org.opensearch.common.util.concurrent.OpenSearchThreadPoolExecutor;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.DiscoveryModule;
import org.opensearch.env.NodeMetadata;
import org.opensearch.node.Node;
import org.opensearch.plugins.MetadataUpgrader;
@ -126,39 +125,6 @@ public class GatewayMetaState implements Closeable {
) {
assert persistedState.get() == null : "should only start once, but already have " + persistedState.get();
if (DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings).equals(DiscoveryModule.ZEN_DISCOVERY_TYPE)) {
// only for tests that simulate mixed Zen1/Zen2 clusters, see Zen1IT
final Tuple<Manifest, Metadata> manifestClusterStateTuple;
try {
NodeMetadata.FORMAT.writeAndCleanup(
new NodeMetadata(persistedClusterStateService.getNodeId(), Version.CURRENT),
persistedClusterStateService.getDataPaths()
);
manifestClusterStateTuple = metaStateService.loadFullState();
} catch (IOException e) {
throw new UncheckedIOException(e);
}
final ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.get(settings))
.version(manifestClusterStateTuple.v1().getClusterStateVersion())
.metadata(manifestClusterStateTuple.v2())
.build();
final IncrementalClusterStateWriter incrementalClusterStateWriter = new IncrementalClusterStateWriter(
settings,
clusterService.getClusterSettings(),
metaStateService,
manifestClusterStateTuple.v1(),
prepareInitialClusterState(transportService, clusterService, clusterState),
transportService.getThreadPool()::relativeTimeInMillis
);
if (DiscoveryNode.isMasterNode(settings) || DiscoveryNode.isDataNode(settings)) {
clusterService.addLowPriorityApplier(new GatewayClusterApplier(incrementalClusterStateWriter));
}
persistedState.set(new InMemoryPersistedState(manifestClusterStateTuple.v1().getCurrentTerm(), clusterState));
return;
}
if (DiscoveryNode.isMasterNode(settings) || DiscoveryNode.isDataNode(settings)) {
try {
final PersistedClusterStateService.OnDiskState onDiskState = persistedClusterStateService.loadBestOnDiskState();

View File

@ -54,7 +54,6 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.AbstractRunnable;
import org.opensearch.discovery.Discovery;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.rest.RestStatus;
import org.opensearch.threadpool.ThreadPool;
@ -170,8 +169,6 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste
// default the recover after master nodes to the minimum master nodes in the discovery
if (RECOVER_AFTER_MASTER_NODES_SETTING.exists(settings)) {
recoverAfterMasterNodes = RECOVER_AFTER_MASTER_NODES_SETTING.get(settings);
} else if (discovery instanceof ZenDiscovery) {
recoverAfterMasterNodes = settings.getAsInt("discovery.zen.minimum_master_nodes", -1);
} else {
recoverAfterMasterNodes = -1;
}

View File

@ -109,7 +109,6 @@ import org.opensearch.common.xcontent.NamedXContentRegistry;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.Discovery;
import org.opensearch.discovery.DiscoveryModule;
import org.opensearch.discovery.DiscoverySettings;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.env.NodeMetadata;
@ -294,6 +293,14 @@ public class Node implements Closeable {
private static final String CLIENT_TYPE = "node";
public static class DiscoverySettings {
public static final Setting<TimeValue> INITIAL_STATE_TIMEOUT_SETTING = Setting.positiveTimeSetting(
"discovery.initial_state_timeout",
TimeValue.timeValueSeconds(30),
Property.NodeScope
);
}
private final Lifecycle lifecycle = new Lifecycle();
/**
@ -1068,10 +1075,7 @@ public class Node implements Closeable {
);
if (Assertions.ENABLED) {
try {
if (DiscoveryModule.DISCOVERY_TYPE_SETTING.get(environment.settings())
.equals(DiscoveryModule.ZEN_DISCOVERY_TYPE) == false) {
assert injector.getInstance(MetaStateService.class).loadFullState().v1().isEmpty();
}
assert injector.getInstance(MetaStateService.class).loadFullState().v1().isEmpty();
final NodeMetadata nodeMetadata = NodeMetadata.FORMAT.loadLatestState(
logger,
NamedXContentRegistry.EMPTY,

View File

@ -306,7 +306,7 @@ public final class TransportSettings {
);
public static final Setting<List<String>> TRACE_LOG_EXCLUDE_SETTING = listSetting(
"transport.tracer.exclude",
Arrays.asList("internal:discovery/zen/fd*", "internal:coordination/fault_detection/*", TransportLivenessAction.NAME),
Arrays.asList("internal:coordination/fault_detection/*", TransportLivenessAction.NAME),
Function.identity(),
Setting.Property.Dynamic,
Setting.Property.NodeScope

View File

@ -36,8 +36,8 @@ import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.io.stream.BytesStreamOutput;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.discovery.DiscoveryStats;
import org.opensearch.discovery.zen.PendingClusterStateStats;
import org.opensearch.discovery.zen.PublishClusterStateStats;
import org.opensearch.cluster.coordination.PendingClusterStateStats;
import org.opensearch.cluster.coordination.PublishClusterStateStats;
import org.opensearch.http.HttpStats;
import org.opensearch.indices.breaker.AllCircuitBreakerStats;
import org.opensearch.indices.breaker.CircuitBreakerStats;

View File

@ -59,7 +59,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import static org.opensearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE;
import static org.opensearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE;
import static org.hamcrest.CoreMatchers.allOf;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.equalTo;
@ -122,7 +121,7 @@ public class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
when(boundTransportAddress.boundAddresses()).thenReturn(transportAddresses.toArray(new TransportAddress[0]));
when(boundTransportAddress.publishAddress()).thenReturn(publishAddress);
final String discoveryType = randomFrom(ZEN_DISCOVERY_TYPE, ZEN2_DISCOVERY_TYPE, "single-node");
final String discoveryType = randomFrom(ZEN2_DISCOVERY_TYPE, "single-node");
assertEquals(BootstrapChecks.enforceLimits(boundTransportAddress, discoveryType), !"single-node".equals(discoveryType));
}
@ -140,7 +139,7 @@ public class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
when(boundTransportAddress.boundAddresses()).thenReturn(transportAddresses.toArray(new TransportAddress[0]));
when(boundTransportAddress.publishAddress()).thenReturn(publishAddress);
final String discoveryType = randomFrom(ZEN_DISCOVERY_TYPE, ZEN2_DISCOVERY_TYPE, "single-node");
final String discoveryType = randomFrom(ZEN2_DISCOVERY_TYPE, "single-node");
assertEquals(BootstrapChecks.enforceLimits(boundTransportAddress, discoveryType), !"single-node".equals(discoveryType));
}
@ -774,11 +773,6 @@ public class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
public void testDiscoveryConfiguredCheck() throws NodeValidationException {
final List<BootstrapCheck> checks = Collections.singletonList(new BootstrapChecks.DiscoveryConfiguredCheck());
final BootstrapContext zen1Context = createTestContext(
Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), ZEN_DISCOVERY_TYPE).build(),
Metadata.EMPTY_METADATA
);
final BootstrapContext zen2Context = createTestContext(
Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), ZEN2_DISCOVERY_TYPE).build(),
Metadata.EMPTY_METADATA
@ -799,11 +793,6 @@ public class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
checks
);
assertThat(
expectThrows(NodeValidationException.class, () -> BootstrapChecks.check(zen1Context, true, checks)),
hasToString(containsString("discovery type [legacy-zen-for-testing-only-do-not-use] is unsuitable for production use"))
);
final NodeValidationException e = expectThrows(
NodeValidationException.class,
() -> BootstrapChecks.check(zen2Context, true, checks)
@ -829,9 +818,5 @@ public class BootstrapChecksTests extends AbstractBootstrapCheckTestCase {
ensureChecksPass.accept(Settings.builder().putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey()));
ensureChecksPass.accept(Settings.builder().putList(DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING.getKey()));
ensureChecksPass.accept(Settings.builder().putList(SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING.getKey()));
ensureChecksPass.accept(Settings.builder().putList(DiscoveryModule.LEGACY_DISCOVERY_HOSTS_PROVIDER_SETTING.getKey()));
ensureChecksPass.accept(
Settings.builder().putList(SettingsBasedSeedHostsProvider.LEGACY_DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey())
);
}
}

View File

@ -56,7 +56,6 @@ import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static java.util.Collections.singletonList;
import static java.util.Collections.singletonMap;
import static org.opensearch.cluster.coordination.ClusterBootstrapService.BOOTSTRAP_PLACEHOLDER_PREFIX;
import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
import static org.opensearch.cluster.coordination.ClusterBootstrapService.UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING;
@ -195,26 +194,6 @@ public class ClusterBootstrapServiceTests extends OpenSearchTestCase {
deterministicTaskQueue.runAllTasks();
}
public void testDoesNothingByDefaultIfZen1NodesDiscovered() {
final DiscoveryNode zen1Node = new DiscoveryNode(
"zen1",
buildNewFakeTransportAddress(),
singletonMap("zen1", "true"),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT
);
ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(
Settings.EMPTY,
transportService,
() -> Stream.of(localNode, zen1Node).collect(Collectors.toSet()),
() -> false,
vc -> { throw new AssertionError("should not be called"); }
);
transportService.start();
clusterBootstrapService.scheduleUnconfiguredBootstrap();
deterministicTaskQueue.runAllTasks();
}
public void testThrowsExceptionOnDuplicates() {
final IllegalArgumentException illegalArgumentException = expectThrows(IllegalArgumentException.class, () -> {
new ClusterBootstrapService(
@ -472,32 +451,7 @@ public class ClusterBootstrapServiceTests extends OpenSearchTestCase {
deterministicTaskQueue.runAllTasks();
}
public void testDoesNotBootstrapsIfZen1NodesDiscovered() {
final DiscoveryNode zen1Node = new DiscoveryNode(
"zen1",
buildNewFakeTransportAddress(),
singletonMap("zen1", "true"),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE),
Version.CURRENT
);
ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(
Settings.builder()
.putList(INITIAL_MASTER_NODES_SETTING.getKey(), localNode.getName(), otherNode1.getName(), otherNode2.getName())
.build(),
transportService,
() -> Stream.of(otherNode1, otherNode2, zen1Node).collect(Collectors.toList()),
() -> false,
vc -> { throw new AssertionError("should not be called"); }
);
transportService.start();
clusterBootstrapService.onFoundPeersUpdated();
deterministicTaskQueue.runAllTasks();
}
public void testRetriesBootstrappingOnException() {
final AtomicLong bootstrappingAttempts = new AtomicLong();
ClusterBootstrapService clusterBootstrapService = new ClusterBootstrapService(
Settings.builder()

View File

@ -56,7 +56,6 @@ import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.set.Sets;
import org.opensearch.common.xcontent.XContentBuilder;
import org.opensearch.discovery.DiscoveryModule;
import org.opensearch.discovery.zen.PublishClusterStateStats;
import org.opensearch.gateway.GatewayService;
import org.opensearch.monitor.StatusInfo;
import org.opensearch.test.MockLogAppender;

View File

@ -1,54 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.cluster.coordination;
import org.opensearch.Version;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.UUIDs;
import org.opensearch.test.OpenSearchTestCase;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.lessThan;
public class DiscoveryUpgradeServiceTests extends OpenSearchTestCase {
public void testCreateDiscoveryNodeWithImpossiblyHighId() {
final DiscoveryNode discoveryNode = new DiscoveryNode(
UUIDs.randomBase64UUID(random()),
buildNewFakeTransportAddress(),
Version.CURRENT
);
final DiscoveryNode fakeNode = DiscoveryUpgradeService.createDiscoveryNodeWithImpossiblyHighId(discoveryNode);
assertThat(discoveryNode.getId(), lessThan(fakeNode.getId()));
assertThat(UUIDs.randomBase64UUID(random()), lessThan(fakeNode.getId()));
assertThat(fakeNode.getId(), containsString(discoveryNode.getId()));
}
}

View File

@ -41,7 +41,6 @@ import org.opensearch.cluster.NotMasterException;
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.zen.MembershipAction;
import org.opensearch.monitor.StatusInfo;
import org.opensearch.test.OpenSearchTestCase;
import org.opensearch.test.transport.CapturingTransport;
@ -193,13 +192,6 @@ public class JoinHelperTests extends OpenSearchTestCase {
);
}
public void testZen1JoinValidationRejectsMismatchedClusterUUID() {
assertJoinValidationRejectsMismatchedClusterUUID(
MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME,
"mixed-version cluster join validation on cluster state with a different cluster uuid"
);
}
public void testJoinValidationRejectsMismatchedClusterUUID() {
assertJoinValidationRejectsMismatchedClusterUUID(
JoinHelper.VALIDATE_JOIN_ACTION_NAME,

View File

@ -39,7 +39,6 @@ import static org.opensearch.cluster.coordination.NoMasterBlockService.NO_MASTER
import static org.opensearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_METADATA_WRITES;
import static org.opensearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_SETTING;
import static org.opensearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_WRITES;
import static org.opensearch.cluster.coordination.NoMasterBlockService.LEGACY_NO_MASTER_BLOCK_SETTING;
import static org.opensearch.common.settings.ClusterSettings.BUILT_IN_CLUSTER_SETTINGS;
import static org.hamcrest.Matchers.sameInstance;
@ -65,23 +64,11 @@ public class NoMasterBlockServiceTests extends OpenSearchTestCase {
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));
@ -99,14 +86,6 @@ public class NoMasterBlockServiceTests extends OpenSearchTestCase {
);
}
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));
@ -117,13 +96,4 @@ public class NoMasterBlockServiceTests extends OpenSearchTestCase {
clusterSettings.applySettings(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "metadata_write").build());
assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_METADATA_WRITES));
}
public void testIgnoresUpdatesToLegacySetting() {
createService(Settings.builder().put(NO_MASTER_BLOCK_SETTING.getKey(), "all").build());
assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL));
clusterSettings.applySettings(Settings.builder().put(LEGACY_NO_MASTER_BLOCK_SETTING.getKey(), "write").build());
assertThat(noMasterBlockService.getNoMasterBlock(), sameInstance(NO_MASTER_BLOCK_ALL));
assertDeprecatedWarningEmitted();
}
}

View File

@ -126,7 +126,11 @@ public class MetadataIndexUpgradeServiceTests extends OpenSearchTestCase {
public void testFailUpgrade() {
MetadataIndexUpgradeService service = getMetadataIndexUpgradeService();
Version minCompat = Version.CURRENT.minimumIndexCompatibilityVersion();
Version indexUpgraded = VersionUtils.randomVersionBetween(random(), minCompat, VersionUtils.getPreviousVersion(Version.CURRENT));
Version indexUpgraded = VersionUtils.randomVersionBetween(
random(),
minCompat,
Version.max(minCompat, VersionUtils.getPreviousVersion(Version.CURRENT))
);
Version indexCreated = LegacyESVersion.fromString((minCompat.major - 1) + "." + randomInt(5) + "." + randomInt(5));
final IndexMetadata metadata = newIndexMeta(
"foo",

View File

@ -92,7 +92,6 @@ public class ClusterSerializationTests extends OpenSearchAllocationTestCase {
.nodes(nodes)
.metadata(metadata)
.routingTable(routingTable)
.minimumMasterNodesOnPublishingMaster(randomIntBetween(-1, 10))
.build();
AllocationService strategy = createAllocationService();
@ -107,11 +106,6 @@ public class ClusterSerializationTests extends OpenSearchAllocationTestCase {
assertThat(serializedClusterState.getClusterName().value(), equalTo(clusterState.getClusterName().value()));
assertThat(serializedClusterState.routingTable().toString(), equalTo(clusterState.routingTable().toString()));
assertThat(
serializedClusterState.getMinimumMasterNodesOnPublishingMaster(),
equalTo(clusterState.getMinimumMasterNodesOnPublishingMaster())
);
}
public void testRoutingTableSerialization() throws Exception {

View File

@ -1,429 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery;
import org.opensearch.Version;
import org.opensearch.action.admin.cluster.node.liveness.TransportLivenessAction;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.common.breaker.CircuitBreaker;
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.io.stream.NamedWriteableRegistry;
import org.opensearch.common.network.NetworkService;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.ByteSizeValue;
import org.opensearch.common.util.PageCacheRecycler;
import org.opensearch.discovery.zen.FaultDetection;
import org.opensearch.discovery.zen.MasterFaultDetection;
import org.opensearch.discovery.zen.NodesFaultDetection;
import org.opensearch.indices.breaker.CircuitBreakerService;
import org.opensearch.indices.breaker.HierarchyCircuitBreakerService;
import org.opensearch.node.Node;
import org.opensearch.test.OpenSearchTestCase;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.threadpool.TestThreadPool;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.Transport;
import org.opensearch.transport.TransportConnectionListener;
import org.opensearch.transport.TransportMessageListener;
import org.opensearch.transport.TransportRequest;
import org.opensearch.transport.TransportRequestOptions;
import org.opensearch.transport.TransportService;
import org.opensearch.transport.TransportSettings;
import org.opensearch.transport.nio.MockNioTransport;
import org.hamcrest.Matcher;
import org.hamcrest.Matchers;
import org.junit.After;
import org.junit.Before;
import java.util.Collections;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
public class ZenFaultDetectionTests extends OpenSearchTestCase {
protected ThreadPool threadPool;
private CircuitBreakerService circuitBreakerService;
protected static final Version version0 = Version.fromId(6080099);
protected DiscoveryNode nodeA;
protected MockTransportService serviceA;
private Settings settingsA;
protected static final Version version1 = Version.fromId(7100099);
protected DiscoveryNode nodeB;
protected MockTransportService serviceB;
private Settings settingsB;
@Override
@Before
public void setUp() throws Exception {
super.setUp();
Settings settings = Settings.builder()
.put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), new ByteSizeValue(0))
.build();
ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
threadPool = new TestThreadPool(getClass().getName());
circuitBreakerService = new HierarchyCircuitBreakerService(settings, Collections.emptyList(), clusterSettings);
settingsA = Settings.builder().put("node.name", "TS_A").put(settings).build();
serviceA = build(settingsA, version0);
nodeA = serviceA.getLocalDiscoNode();
settingsB = Settings.builder().put("node.name", "TS_B").put(settings).build();
serviceB = build(settingsB, version1);
nodeB = serviceB.getLocalDiscoNode();
// wait till all nodes are properly connected and the event has been sent, so tests in this class
// will not get this callback called on the connections done in this setup
final CountDownLatch latch = new CountDownLatch(2);
TransportConnectionListener waitForConnection = new TransportConnectionListener() {
@Override
public void onNodeConnected(DiscoveryNode node, Transport.Connection connection) {
latch.countDown();
}
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
fail("disconnect should not be called " + node);
}
};
serviceA.addConnectionListener(waitForConnection);
serviceB.addConnectionListener(waitForConnection);
serviceA.connectToNode(nodeB);
serviceA.connectToNode(nodeA);
serviceB.connectToNode(nodeA);
serviceB.connectToNode(nodeB);
assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true));
serviceA.removeConnectionListener(waitForConnection);
serviceB.removeConnectionListener(waitForConnection);
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
serviceA.close();
serviceB.close();
terminate(threadPool);
}
protected MockTransportService build(Settings settings, Version version) {
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
MockTransportService transportService = new MockTransportService(
Settings.builder()
.put(settings)
// trace zenfd actions but keep the default otherwise
.putList(TransportSettings.TRACE_LOG_EXCLUDE_SETTING.getKey(), TransportLivenessAction.NAME)
.build(),
new MockNioTransport(
settings,
version,
threadPool,
new NetworkService(Collections.emptyList()),
PageCacheRecycler.NON_RECYCLING_INSTANCE,
namedWriteableRegistry,
circuitBreakerService
),
threadPool,
TransportService.NOOP_TRANSPORT_INTERCEPTOR,
(boundAddress) -> new DiscoveryNode(
Node.NODE_NAME_SETTING.get(settings),
boundAddress.publishAddress(),
Node.NODE_ATTRIBUTES.getAsMap(settings),
DiscoveryNode.getRolesFromSettings(settings),
version
),
null,
Collections.emptySet()
);
transportService.start();
transportService.acceptIncomingRequests();
return transportService;
}
private DiscoveryNodes buildNodesForA(boolean master) {
DiscoveryNodes.Builder builder = DiscoveryNodes.builder();
builder.add(nodeA);
builder.add(nodeB);
builder.localNodeId(nodeA.getId());
builder.masterNodeId(master ? nodeA.getId() : nodeB.getId());
return builder.build();
}
private DiscoveryNodes buildNodesForB(boolean master) {
DiscoveryNodes.Builder builder = DiscoveryNodes.builder();
builder.add(nodeA);
builder.add(nodeB);
builder.localNodeId(nodeB.getId());
builder.masterNodeId(master ? nodeB.getId() : nodeA.getId());
return builder.build();
}
public void testNodesFaultDetectionConnectOnDisconnect() throws InterruptedException {
boolean shouldRetry = randomBoolean();
// make sure we don't ping again after the initial ping
final Settings pingSettings = Settings.builder()
.put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry)
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m")
.build();
ClusterState clusterState = ClusterState.builder(new ClusterName("test"))
.version(randomNonNegativeLong())
.nodes(buildNodesForA(true))
.build();
NodesFaultDetection nodesFDA = new NodesFaultDetection(
Settings.builder().put(settingsA).put(pingSettings).build(),
threadPool,
serviceA,
() -> clusterState,
clusterState.getClusterName()
);
nodesFDA.setLocalNode(nodeA);
NodesFaultDetection nodesFDB = new NodesFaultDetection(
Settings.builder().put(settingsB).put(pingSettings).build(),
threadPool,
serviceB,
() -> clusterState,
clusterState.getClusterName()
);
nodesFDB.setLocalNode(nodeB);
final CountDownLatch pingSent = new CountDownLatch(1);
nodesFDB.addListener(new NodesFaultDetection.Listener() {
@Override
public void onPingReceived(NodesFaultDetection.PingRequest pingRequest) {
assertThat(pingRequest.clusterStateVersion(), equalTo(clusterState.version()));
pingSent.countDown();
}
});
nodesFDA.updateNodesAndPing(clusterState);
// wait for the first ping to go out, so we will really respond to a disconnect event rather then
// the ping failing
pingSent.await(30, TimeUnit.SECONDS);
final String[] failureReason = new String[1];
final DiscoveryNode[] failureNode = new DiscoveryNode[1];
final CountDownLatch notified = new CountDownLatch(1);
nodesFDA.addListener(new NodesFaultDetection.Listener() {
@Override
public void onNodeFailure(DiscoveryNode node, String reason) {
failureNode[0] = node;
failureReason[0] = reason;
notified.countDown();
}
});
// will raise a disconnect on A
serviceB.stop();
notified.await(30, TimeUnit.SECONDS);
CircuitBreaker inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS);
assertThat(inFlightRequestsBreaker.getTrippedCount(), equalTo(0L));
assertEquals(nodeB, failureNode[0]);
Matcher<String> matcher = Matchers.containsString("verified");
if (!shouldRetry) {
matcher = Matchers.not(matcher);
}
assertThat(failureReason[0], matcher);
assertWarnings(
"[discovery.zen.fd.connect_on_network_disconnect] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version.",
"[discovery.zen.fd.ping_interval] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version."
);
}
public void testMasterFaultDetectionConnectOnDisconnect() throws InterruptedException {
Settings.Builder settings = Settings.builder();
boolean shouldRetry = randomBoolean();
ClusterName clusterName = new ClusterName(randomAlphaOfLengthBetween(3, 20));
// make sure we don't ping
settings.put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry)
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "5m")
.put("cluster.name", clusterName.value());
final ClusterState state = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build();
AtomicReference<ClusterState> clusterStateSupplier = new AtomicReference<>(state);
MasterFaultDetection masterFD = new MasterFaultDetection(
settings.build(),
threadPool,
serviceA,
clusterStateSupplier::get,
null,
clusterName
);
masterFD.restart(nodeB, "test");
final String[] failureReason = new String[1];
final DiscoveryNode[] failureNode = new DiscoveryNode[1];
final CountDownLatch notified = new CountDownLatch(1);
masterFD.addListener((masterNode, cause, reason) -> {
failureNode[0] = masterNode;
failureReason[0] = reason;
notified.countDown();
});
// will raise a disconnect on A
serviceB.stop();
notified.await(30, TimeUnit.SECONDS);
CircuitBreaker inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS);
assertThat(inFlightRequestsBreaker.getTrippedCount(), equalTo(0L));
assertEquals(nodeB, failureNode[0]);
Matcher<String> matcher = Matchers.containsString("verified");
if (!shouldRetry) {
matcher = Matchers.not(matcher);
}
assertThat(failureReason[0], matcher);
assertWarnings(
"[discovery.zen.fd.connect_on_network_disconnect] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version.",
"[discovery.zen.fd.ping_interval] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version."
);
}
public void testMasterFaultDetectionNotSizeLimited() throws InterruptedException {
boolean shouldRetry = randomBoolean();
ClusterName clusterName = new ClusterName(randomAlphaOfLengthBetween(3, 20));
final Settings settings = Settings.builder()
.put(FaultDetection.CONNECT_ON_NETWORK_DISCONNECT_SETTING.getKey(), shouldRetry)
.put(FaultDetection.PING_INTERVAL_SETTING.getKey(), "1s")
.put("cluster.name", clusterName.value())
.build();
final ClusterState stateNodeA = ClusterState.builder(clusterName).nodes(buildNodesForA(false)).build();
AtomicReference<ClusterState> clusterStateSupplierA = new AtomicReference<>(stateNodeA);
int minExpectedPings = 2;
PingProbe pingProbeA = new PingProbe(minExpectedPings);
PingProbe pingProbeB = new PingProbe(minExpectedPings);
serviceA.addMessageListener(pingProbeA);
serviceB.addMessageListener(pingProbeB);
MasterFaultDetection masterFDNodeA = new MasterFaultDetection(
Settings.builder().put(settingsA).put(settings).build(),
threadPool,
serviceA,
clusterStateSupplierA::get,
null,
clusterName
);
masterFDNodeA.restart(nodeB, "test");
final ClusterState stateNodeB = ClusterState.builder(clusterName).nodes(buildNodesForB(true)).build();
AtomicReference<ClusterState> clusterStateSupplierB = new AtomicReference<>(stateNodeB);
MasterFaultDetection masterFDNodeB = new MasterFaultDetection(
Settings.builder().put(settingsB).put(settings).build(),
threadPool,
serviceB,
clusterStateSupplierB::get,
null,
clusterName
);
masterFDNodeB.restart(nodeB, "test");
// let's do a few pings
pingProbeA.awaitMinCompletedPings();
pingProbeB.awaitMinCompletedPings();
CircuitBreaker inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS);
assertThat(inFlightRequestsBreaker.getTrippedCount(), equalTo(0L));
assertThat(pingProbeA.completedPings(), greaterThanOrEqualTo(minExpectedPings));
assertThat(pingProbeB.completedPings(), greaterThanOrEqualTo(minExpectedPings));
assertWarnings(
"[discovery.zen.fd.connect_on_network_disconnect] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version.",
"[discovery.zen.fd.ping_interval] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version."
);
}
private static class PingProbe implements TransportMessageListener {
private final Set<Tuple<DiscoveryNode, Long>> inflightPings = Collections.newSetFromMap(new ConcurrentHashMap<>());
private final Set<Tuple<DiscoveryNode, Long>> completedPings = Collections.newSetFromMap(new ConcurrentHashMap<>());
private final CountDownLatch waitForPings;
PingProbe(int minCompletedPings) {
this.waitForPings = new CountDownLatch(minCompletedPings);
}
@Override
public void onRequestSent(
DiscoveryNode node,
long requestId,
String action,
TransportRequest request,
TransportRequestOptions options
) {
if (MasterFaultDetection.MASTER_PING_ACTION_NAME.equals(action)) {
inflightPings.add(Tuple.tuple(node, requestId));
}
}
@Override
public void onResponseReceived(long requestId, Transport.ResponseContext context) {
if (MasterFaultDetection.MASTER_PING_ACTION_NAME.equals(context.action())) {
Tuple<DiscoveryNode, Long> ping = Tuple.tuple(context.connection().getNode(), requestId);
if (inflightPings.remove(ping)) {
completedPings.add(ping);
waitForPings.countDown();
}
}
}
public int completedPings() {
return completedPings.size();
}
public void awaitMinCompletedPings() throws InterruptedException {
waitForPings.await();
}
}
}

View File

@ -1,189 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.opensearch.Version;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodeRole;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.zen.ElectMasterService.MasterCandidate;
import org.opensearch.test.OpenSearchTestCase;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
public class ElectMasterServiceTests extends OpenSearchTestCase {
ElectMasterService electMasterService() {
return new ElectMasterService(Settings.EMPTY);
}
List<DiscoveryNode> generateRandomNodes() {
int count = scaledRandomIntBetween(1, 100);
ArrayList<DiscoveryNode> nodes = new ArrayList<>(count);
for (int i = 0; i < count; i++) {
Set<DiscoveryNodeRole> roles = new HashSet<>();
if (randomBoolean()) {
roles.add(DiscoveryNodeRole.MASTER_ROLE);
}
DiscoveryNode node = new DiscoveryNode(
"n_" + i,
"n_" + i,
buildNewFakeTransportAddress(),
Collections.emptyMap(),
roles,
Version.CURRENT
);
nodes.add(node);
}
Collections.shuffle(nodes, random());
return nodes;
}
List<MasterCandidate> generateRandomCandidates() {
int count = scaledRandomIntBetween(1, 100);
ArrayList<MasterCandidate> candidates = new ArrayList<>(count);
for (int i = 0; i < count; i++) {
Set<DiscoveryNodeRole> roles = new HashSet<>();
roles.add(DiscoveryNodeRole.MASTER_ROLE);
DiscoveryNode node = new DiscoveryNode(
"n_" + i,
"n_" + i,
buildNewFakeTransportAddress(),
Collections.emptyMap(),
roles,
Version.CURRENT
);
candidates.add(
new MasterCandidate(node, randomBoolean() ? MasterCandidate.UNRECOVERED_CLUSTER_VERSION : randomNonNegativeLong())
);
}
Collections.shuffle(candidates, random());
return candidates;
}
public void testSortByMasterLikelihood() {
List<DiscoveryNode> nodes = generateRandomNodes();
List<DiscoveryNode> sortedNodes = ElectMasterService.sortByMasterLikelihood(nodes);
assertEquals(nodes.size(), sortedNodes.size());
DiscoveryNode prevNode = sortedNodes.get(0);
for (int i = 1; i < sortedNodes.size(); i++) {
DiscoveryNode node = sortedNodes.get(i);
if (!prevNode.isMasterNode()) {
assertFalse(node.isMasterNode());
} else if (node.isMasterNode()) {
assertTrue(prevNode.getId().compareTo(node.getId()) < 0);
}
prevNode = node;
}
}
public void testTieBreakActiveMasters() {
List<DiscoveryNode> nodes = generateRandomCandidates().stream().map(MasterCandidate::getNode).collect(Collectors.toList());
DiscoveryNode bestMaster = electMasterService().tieBreakActiveMasters(nodes);
for (DiscoveryNode node : nodes) {
if (node.equals(bestMaster) == false) {
assertTrue(bestMaster.getId().compareTo(node.getId()) < 0);
}
}
}
public void testHasEnoughNodes() {
List<DiscoveryNode> nodes = rarely() ? Collections.emptyList() : generateRandomNodes();
ElectMasterService service = electMasterService();
int masterNodes = (int) nodes.stream().filter(DiscoveryNode::isMasterNode).count();
service.minimumMasterNodes(randomIntBetween(-1, masterNodes));
assertThat(service.hasEnoughMasterNodes(nodes), equalTo(masterNodes > 0));
service.minimumMasterNodes(masterNodes + 1 + randomIntBetween(0, nodes.size()));
assertFalse(service.hasEnoughMasterNodes(nodes));
}
public void testHasEnoughCandidates() {
List<MasterCandidate> candidates = rarely() ? Collections.emptyList() : generateRandomCandidates();
ElectMasterService service = electMasterService();
service.minimumMasterNodes(randomIntBetween(-1, candidates.size()));
assertThat(service.hasEnoughCandidates(candidates), equalTo(candidates.size() > 0));
service.minimumMasterNodes(candidates.size() + 1 + randomIntBetween(0, candidates.size()));
assertFalse(service.hasEnoughCandidates(candidates));
}
public void testElectMaster() {
List<MasterCandidate> candidates = generateRandomCandidates();
ElectMasterService service = electMasterService();
int minMasterNodes = randomIntBetween(0, candidates.size());
service.minimumMasterNodes(minMasterNodes);
MasterCandidate master = service.electMaster(candidates);
assertNotNull(master);
for (MasterCandidate candidate : candidates) {
if (candidate.getNode().equals(master.getNode())) {
// nothing much to test here
} else if (candidate.getClusterStateVersion() == master.getClusterStateVersion()) {
assertThat(
"candidate " + candidate + " has a lower or equal id than master " + master,
candidate.getNode().getId(),
greaterThan(master.getNode().getId())
);
} else {
assertThat(
"candidate " + master + " has a higher cluster state version than candidate " + candidate,
master.getClusterStateVersion(),
greaterThan(candidate.getClusterStateVersion())
);
}
}
}
public void testCountMasterNodes() {
List<DiscoveryNode> nodes = generateRandomNodes();
ElectMasterService service = electMasterService();
int masterNodes = 0;
for (DiscoveryNode node : nodes) {
if (node.isMasterNode()) {
masterNodes++;
}
}
assertEquals(masterNodes, service.countMasterNodes(nodes));
}
}

View File

@ -1,307 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.opensearch.OpenSearchException;
import org.opensearch.Version;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.common.settings.Settings;
import org.opensearch.discovery.zen.PendingClusterStatesQueue.ClusterStateContext;
import org.opensearch.test.OpenSearchTestCase;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasKey;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.sameInstance;
public class PendingClusterStatesQueueTests extends OpenSearchTestCase {
public void testSelectNextStateToProcess_empty() {
PendingClusterStatesQueue queue = new PendingClusterStatesQueue(logger, randomIntBetween(1, 200));
assertThat(queue.getNextClusterStateToProcess(), nullValue());
}
public void testDroppingStatesAtCapacity() {
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4");
Collections.shuffle(states, random());
// insert half of the states
final int numberOfStateToDrop = states.size() / 2;
List<ClusterState> stateToDrop = states.subList(0, numberOfStateToDrop);
final int queueSize = states.size() - numberOfStateToDrop;
PendingClusterStatesQueue queue = createQueueWithStates(stateToDrop, queueSize);
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
for (ClusterState state : states.subList(numberOfStateToDrop, states.size())) {
queue.addPending(state);
}
assertThat(queue.pendingClusterStates().length, equalTo(queueSize));
// check all committed states got a failure due to the drop
for (ClusterStateContext context : committedContexts) {
assertThat(((MockListener) context.listener).failure, notNullValue());
}
// all states that should have dropped are indeed dropped.
for (ClusterState state : stateToDrop) {
assertThat(queue.findState(state.stateUUID()), nullValue());
}
}
public void testSimpleQueueSameMaster() {
final int numUpdates = scaledRandomIntBetween(50, 100);
List<ClusterState> states = randomStates(numUpdates, "master");
Collections.shuffle(states, random());
PendingClusterStatesQueue queue;
queue = createQueueWithStates(states);
// no state is committed yet
assertThat(queue.getNextClusterStateToProcess(), nullValue());
ClusterState highestCommitted = null;
for (ClusterStateContext context : randomCommitStates(queue)) {
if (highestCommitted == null || context.state.supersedes(highestCommitted)) {
highestCommitted = context.state;
}
}
assertThat(queue.getNextClusterStateToProcess(), sameInstance(highestCommitted));
queue.markAsProcessed(highestCommitted);
// now there is nothing more to process
assertThat(queue.getNextClusterStateToProcess(), nullValue());
}
public void testProcessedStateCleansStatesFromOtherMasters() {
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4");
PendingClusterStatesQueue queue = createQueueWithStates(states);
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
ClusterState randomCommitted = randomFrom(committedContexts).state;
queue.markAsProcessed(randomCommitted);
final String processedMaster = randomCommitted.nodes().getMasterNodeId();
// now check that queue doesn't contain anything pending from another master
for (ClusterStateContext context : queue.pendingStates) {
final String pendingMaster = context.state.nodes().getMasterNodeId();
assertThat(
"found a cluster state from [" + pendingMaster + "], after a state from [" + processedMaster + "] was processed",
pendingMaster,
equalTo(processedMaster)
);
}
// and check all committed contexts from another master were failed
for (ClusterStateContext context : committedContexts) {
if (context.state.nodes().getMasterNodeId().equals(processedMaster) == false) {
assertThat(((MockListener) context.listener).failure, notNullValue());
}
}
}
public void testFailedStateCleansSupersededStatesOnly() {
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4");
PendingClusterStatesQueue queue = createQueueWithStates(states);
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
ClusterState toFail = randomFrom(committedContexts).state;
queue.markAsFailed(toFail, new OpenSearchException("boo!"));
final Map<String, ClusterStateContext> committedContextsById = new HashMap<>();
for (ClusterStateContext context : committedContexts) {
committedContextsById.put(context.stateUUID(), context);
}
// now check that queue doesn't contain superseded states
for (ClusterStateContext context : queue.pendingStates) {
if (context.committed()) {
assertFalse(
"found a committed cluster state, which is superseded by a failed state.\nFound:"
+ context.state
+ "\nfailed:"
+ toFail,
toFail.supersedes(context.state)
);
}
}
// check no state has been erroneously removed
for (ClusterState state : states) {
ClusterStateContext pendingContext = queue.findState(state.stateUUID());
if (pendingContext != null) {
continue;
}
if (state.equals(toFail)) {
continue;
}
assertThat("non-committed states should never be removed", committedContextsById, hasKey(state.stateUUID()));
final ClusterStateContext context = committedContextsById.get(state.stateUUID());
assertThat(
"removed state is not superseded by failed state. \nRemoved state:" + context + "\nfailed: " + toFail,
toFail.supersedes(context.state),
equalTo(true)
);
assertThat("removed state was failed with wrong exception", ((MockListener) context.listener).failure, notNullValue());
assertThat(
"removed state was failed with wrong exception",
((MockListener) context.listener).failure.getMessage(),
containsString("boo")
);
}
}
public void testFailAllAndClear() {
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4");
PendingClusterStatesQueue queue = createQueueWithStates(states);
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
queue.failAllStatesAndClear(new OpenSearchException("boo!"));
assertThat(queue.pendingStates, empty());
assertThat(queue.getNextClusterStateToProcess(), nullValue());
for (ClusterStateContext context : committedContexts) {
assertThat("state was failed with wrong exception", ((MockListener) context.listener).failure, notNullValue());
assertThat(
"state was failed with wrong exception",
((MockListener) context.listener).failure.getMessage(),
containsString("boo")
);
}
}
public void testQueueStats() {
List<ClusterState> states = randomStates(scaledRandomIntBetween(10, 100), "master");
PendingClusterStatesQueue queue = createQueueWithStates(states);
assertThat(queue.stats().getTotal(), equalTo(states.size()));
assertThat(queue.stats().getPending(), equalTo(states.size()));
assertThat(queue.stats().getCommitted(), equalTo(0));
List<ClusterStateContext> committedContexts = randomCommitStates(queue);
assertThat(queue.stats().getTotal(), equalTo(states.size()));
assertThat(queue.stats().getPending(), equalTo(states.size() - committedContexts.size()));
assertThat(queue.stats().getCommitted(), equalTo(committedContexts.size()));
ClusterState highestCommitted = null;
for (ClusterStateContext context : committedContexts) {
if (highestCommitted == null || context.state.supersedes(highestCommitted)) {
highestCommitted = context.state;
}
}
assert highestCommitted != null;
queue.markAsProcessed(highestCommitted);
assertThat((long) queue.stats().getTotal(), equalTo(states.size() - (1 + highestCommitted.version())));
assertThat((long) queue.stats().getPending(), equalTo(states.size() - (1 + highestCommitted.version())));
assertThat(queue.stats().getCommitted(), equalTo(0));
}
protected List<ClusterStateContext> randomCommitStates(PendingClusterStatesQueue queue) {
List<ClusterStateContext> committedContexts = new ArrayList<>();
for (int iter = randomInt(queue.pendingStates.size() - 1); iter >= 0; iter--) {
ClusterState state = queue.markAsCommitted(randomFrom(queue.pendingStates).stateUUID(), new MockListener());
if (state != null) {
// null cluster state means we committed twice
committedContexts.add(queue.findState(state.stateUUID()));
}
}
return committedContexts;
}
PendingClusterStatesQueue createQueueWithStates(List<ClusterState> states) {
return createQueueWithStates(states, states.size() * 2); // we don't care about limits (there are dedicated tests for that)
}
PendingClusterStatesQueue createQueueWithStates(List<ClusterState> states, int maxQueueSize) {
PendingClusterStatesQueue queue;
queue = new PendingClusterStatesQueue(logger, maxQueueSize);
for (ClusterState state : states) {
queue.addPending(state);
}
return queue;
}
List<ClusterState> randomStates(int count, String... masters) {
ArrayList<ClusterState> states = new ArrayList<>(count);
ClusterState[] lastClusterStatePerMaster = new ClusterState[masters.length];
for (; count > 0; count--) {
int masterIndex = randomInt(masters.length - 1);
ClusterState state = lastClusterStatePerMaster[masterIndex];
if (state == null) {
state = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY))
.nodes(
DiscoveryNodes.builder()
.add(
new DiscoveryNode(
masters[masterIndex],
buildNewFakeTransportAddress(),
emptyMap(),
emptySet(),
Version.CURRENT
)
)
.masterNodeId(masters[masterIndex])
.build()
)
.build();
} else {
state = ClusterState.builder(state).incrementVersion().build();
}
states.add(state);
lastClusterStatePerMaster[masterIndex] = state;
}
return states;
}
static class MockListener implements PendingClusterStatesQueue.StateProcessedListener {
volatile boolean processed;
volatile Throwable failure;
@Override
public void onNewClusterStateProcessed() {
processed = true;
}
@Override
public void onNewClusterStateFailed(Exception e) {
failure = e;
}
}
}

View File

@ -1,711 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.action.support.replication.ClusterStateCreationUtils;
import org.opensearch.cluster.ClusterChangedEvent;
import org.opensearch.cluster.ClusterModule;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.ClusterStateTaskExecutor;
import org.opensearch.cluster.OpenSearchAllocationTestCase;
import org.opensearch.cluster.coordination.FailedToCommitClusterStateException;
import org.opensearch.cluster.coordination.JoinTaskExecutor;
import org.opensearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor;
import org.opensearch.cluster.coordination.ValidateJoinRequest;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodeRole;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.cluster.routing.IndexRoutingTable;
import org.opensearch.cluster.routing.IndexShardRoutingTable;
import org.opensearch.cluster.routing.RoutingTable;
import org.opensearch.cluster.routing.ShardRoutingState;
import org.opensearch.cluster.routing.TestShardRouting;
import org.opensearch.cluster.routing.UnassignedInfo;
import org.opensearch.cluster.routing.allocation.AllocationService;
import org.opensearch.cluster.service.ClusterApplier;
import org.opensearch.cluster.service.MasterService;
import org.opensearch.common.io.stream.NamedWriteableRegistry;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.zen.PublishClusterStateActionTests.AssertingAckListener;
import org.opensearch.discovery.zen.ZenDiscovery.ZenNodeRemovalClusterStateTaskExecutor;
import org.opensearch.index.shard.ShardId;
import org.opensearch.test.ClusterServiceUtils;
import org.opensearch.test.OpenSearchTestCase;
import org.opensearch.test.VersionUtils;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.threadpool.TestThreadPool;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.TransportChannel;
import org.opensearch.transport.TransportResponse;
import org.opensearch.transport.TransportService;
import org.hamcrest.CoreMatchers;
import org.hamcrest.core.IsInstanceOf;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_CREATION_DATE;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_VERSION_CREATED;
import static org.opensearch.cluster.routing.RoutingTableTests.updateActiveAllocations;
import static org.opensearch.cluster.service.MasterServiceTests.discoveryState;
import static org.opensearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING;
import static org.opensearch.discovery.zen.ZenDiscovery.shouldIgnoreOrRejectNewClusterState;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.emptyArray;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasToString;
import static org.hamcrest.Matchers.is;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
public class ZenDiscoveryUnitTests extends OpenSearchTestCase {
public void testShouldIgnoreNewClusterState() {
ClusterName clusterName = new ClusterName("abc");
DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder();
currentNodes.masterNodeId("a").add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT));
DiscoveryNodes.Builder newNodes = DiscoveryNodes.builder();
newNodes.masterNodeId("a").add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT));
ClusterState.Builder currentState = ClusterState.builder(clusterName);
currentState.nodes(currentNodes);
ClusterState.Builder newState = ClusterState.builder(clusterName);
newState.nodes(newNodes);
currentState.version(2);
newState.version(1);
assertTrue(
"should ignore, because new state's version is lower to current state's version",
shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())
);
currentState.version(1);
newState.version(1);
assertTrue(
"should ignore, because new state's version is equal to current state's version",
shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())
);
currentState.version(1);
newState.version(2);
assertFalse(
"should not ignore, because new state's version is higher to current state's version",
shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())
);
currentNodes = DiscoveryNodes.builder();
currentNodes.masterNodeId("b").add(new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT));
// version isn't taken into account, so randomize it to ensure this.
if (randomBoolean()) {
currentState.version(2);
newState.version(1);
} else {
currentState.version(1);
newState.version(2);
}
currentState.nodes(currentNodes);
try {
shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build());
fail("should ignore, because current state's master is not equal to new state's master");
} catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("cluster state from a different master than the current one, rejecting"));
}
currentNodes = DiscoveryNodes.builder();
currentNodes.masterNodeId(null);
currentState.nodes(currentNodes);
// version isn't taken into account, so randomize it to ensure this.
if (randomBoolean()) {
currentState.version(2);
newState.version(1);
} else {
currentState.version(1);
newState.version(2);
}
assertFalse(
"should not ignore, because current state doesn't have a master",
shouldIgnoreOrRejectNewClusterState(logger, currentState.build(), newState.build())
);
}
public void testFilterNonMasterPingResponse() {
ArrayList<ZenPing.PingResponse> responses = new ArrayList<>();
ArrayList<DiscoveryNode> masterNodes = new ArrayList<>();
ArrayList<DiscoveryNode> allNodes = new ArrayList<>();
for (int i = randomIntBetween(10, 20); i >= 0; i--) {
Set<DiscoveryNodeRole> roles = new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES));
DiscoveryNode node = new DiscoveryNode(
"node_" + i,
"id_" + i,
buildNewFakeTransportAddress(),
Collections.emptyMap(),
roles,
Version.CURRENT
);
responses.add(new ZenPing.PingResponse(node, randomBoolean() ? null : node, new ClusterName("test"), randomLong()));
allNodes.add(node);
if (node.isMasterNode()) {
masterNodes.add(node);
}
}
boolean ignore = randomBoolean();
List<ZenPing.PingResponse> filtered = ZenDiscovery.filterPingResponses(responses, ignore, logger);
final List<DiscoveryNode> filteredNodes = filtered.stream().map(ZenPing.PingResponse::node).collect(Collectors.toList());
if (ignore) {
assertThat(filteredNodes, equalTo(masterNodes));
} else {
assertThat(filteredNodes, equalTo(allNodes));
}
}
public void testNodesUpdatedAfterClusterStatePublished() throws Exception {
ThreadPool threadPool = new TestThreadPool(getClass().getName());
// randomly make minimum_master_nodes a value higher than we have nodes for, so it will force failure
int minMasterNodes = randomBoolean() ? 3 : 1;
Settings settings = Settings.builder()
.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.toString(minMasterNodes))
.build();
ArrayDeque<Closeable> toClose = new ArrayDeque<>();
try {
Set<DiscoveryNode> expectedFDNodes = null;
final MockTransportService masterTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null);
masterTransport.start();
DiscoveryNode masterNode = masterTransport.getLocalNode();
toClose.addFirst(masterTransport);
ClusterState state = ClusterStateCreationUtils.state(masterNode, masterNode, masterNode);
// build the zen discovery and discovery service
MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode);
toClose.addFirst(masterMasterService);
// TODO: clustername shouldn't be stored twice in cluster service, but for now, work around it
state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
Settings settingsWithClusterName = Settings.builder()
.put(settings)
.put(ClusterName.CLUSTER_NAME_SETTING.getKey(), discoveryState(masterMasterService).getClusterName().value())
.build();
ZenDiscovery masterZen = buildZenDiscovery(settingsWithClusterName, masterTransport, masterMasterService, threadPool);
masterZen.setCommittedState(state);
toClose.addFirst(masterZen);
masterTransport.acceptIncomingRequests();
final MockTransportService otherTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null);
otherTransport.start();
toClose.addFirst(otherTransport);
DiscoveryNode otherNode = otherTransport.getLocalNode();
final ClusterState otherState = ClusterState.builder(discoveryState(masterMasterService).getClusterName())
.nodes(DiscoveryNodes.builder().add(otherNode).localNodeId(otherNode.getId()))
.build();
MasterService otherMasterService = ClusterServiceUtils.createMasterService(threadPool, otherNode);
toClose.addFirst(otherMasterService);
ZenDiscovery otherZen = buildZenDiscovery(settingsWithClusterName, otherTransport, otherMasterService, threadPool);
otherZen.setCommittedState(otherState);
toClose.addFirst(otherZen);
otherTransport.acceptIncomingRequests();
masterTransport.connectToNode(otherNode);
otherTransport.connectToNode(masterNode);
// a new cluster state with a new discovery node (we will test if the cluster state
// was updated by the presence of this node in NodesFaultDetection)
ClusterState newState = ClusterState.builder(discoveryState(masterMasterService))
.incrementVersion()
.nodes(DiscoveryNodes.builder(state.nodes()).add(otherNode).masterNodeId(masterNode.getId()))
.build();
// publishing a new cluster state
ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent("testing", newState, state);
AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1);
expectedFDNodes = masterZen.getFaultDetectionNodes();
AwaitingPublishListener awaitingPublishListener = new AwaitingPublishListener();
masterZen.publish(clusterChangedEvent, awaitingPublishListener, listener);
awaitingPublishListener.await();
if (awaitingPublishListener.getException() == null) {
// publication succeeded, wait for acks
listener.await(10, TimeUnit.SECONDS);
// publish was a success, update expected FD nodes based on new cluster state
expectedFDNodes = fdNodesForState(newState, masterNode);
} else {
// not successful, so expectedFDNodes above should remain what it was originally assigned
assertEquals(3, minMasterNodes); // ensure min master nodes is the higher value, otherwise we shouldn't fail
}
assertEquals(expectedFDNodes, masterZen.getFaultDetectionNodes());
} finally {
IOUtils.close(toClose);
terminate(threadPool);
}
assertWarnings(
"[discovery.zen.minimum_master_nodes] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version."
);
}
public void testPendingCSQueueIsClearedWhenClusterStatePublished() throws Exception {
ThreadPool threadPool = new TestThreadPool(getClass().getName());
// randomly make minimum_master_nodes a value higher than we have nodes for, so it will force failure
int minMasterNodes = randomBoolean() ? 3 : 1;
Settings settings = Settings.builder()
.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.toString(minMasterNodes))
.build();
ArrayDeque<Closeable> toClose = new ArrayDeque<>();
try {
final MockTransportService masterTransport = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null);
masterTransport.start();
DiscoveryNode masterNode = masterTransport.getLocalNode();
toClose.addFirst(masterTransport);
ClusterState state = ClusterStateCreationUtils.state(masterNode, null, masterNode);
// build the zen discovery and master service for the master node
MasterService masterMasterService = ClusterServiceUtils.createMasterService(threadPool, masterNode);
toClose.addFirst(masterMasterService);
state = ClusterState.builder(discoveryState(masterMasterService).getClusterName()).nodes(state.nodes()).build();
ZenDiscovery masterZen = buildZenDiscovery(settings, masterTransport, masterMasterService, threadPool);
masterZen.setCommittedState(state);
toClose.addFirst(masterZen);
masterTransport.acceptIncomingRequests();
// inject a pending cluster state
masterZen.pendingClusterStatesQueue().addPending(ClusterState.builder(new ClusterName("foreign")).build());
// a new cluster state with a new discovery node (we will test if the cluster state
// was updated by the presence of this node in NodesFaultDetection)
ClusterState newState = ClusterState.builder(discoveryState(masterMasterService))
.incrementVersion()
.nodes(DiscoveryNodes.builder(discoveryState(masterMasterService).nodes()).masterNodeId(masterNode.getId()))
.build();
// publishing a new cluster state
ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent("testing", newState, state);
AssertingAckListener listener = new AssertingAckListener(newState.nodes().getSize() - 1);
AwaitingPublishListener awaitingPublishListener = new AwaitingPublishListener();
masterZen.publish(clusterChangedEvent, awaitingPublishListener, listener);
awaitingPublishListener.await();
if (awaitingPublishListener.getException() == null) {
// publication succeeded, wait for acks
listener.await(1, TimeUnit.HOURS);
}
// queue should be cleared whether successful or not
assertThat(Arrays.toString(masterZen.pendingClusterStates()), masterZen.pendingClusterStates(), emptyArray());
} finally {
IOUtils.close(toClose);
terminate(threadPool);
}
assertWarnings(
"[discovery.zen.minimum_master_nodes] setting was deprecated in OpenSearch and will be removed in a future "
+ "release! See the breaking changes documentation for the next major version."
);
}
private class AwaitingPublishListener implements ActionListener<Void> {
private final CountDownLatch countDownLatch = new CountDownLatch(1);
private FailedToCommitClusterStateException exception;
@Override
public synchronized void onResponse(Void aVoid) {
assertThat(countDownLatch.getCount(), is(1L));
countDownLatch.countDown();
}
@Override
public synchronized void onFailure(Exception e) {
assertThat(e, IsInstanceOf.instanceOf(FailedToCommitClusterStateException.class));
exception = (FailedToCommitClusterStateException) e;
onResponse(null);
}
public void await() throws InterruptedException {
countDownLatch.await();
}
public synchronized FailedToCommitClusterStateException getException() {
return exception;
}
}
private ZenDiscovery buildZenDiscovery(
Settings settings,
TransportService service,
MasterService masterService,
ThreadPool threadPool
) {
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
ClusterApplier clusterApplier = new ClusterApplier() {
@Override
public void setInitialState(ClusterState initialState) {
}
@Override
public void onNewClusterState(String source, Supplier<ClusterState> clusterStateSupplier, ClusterApplyListener listener) {
listener.onSuccess(source);
}
};
ZenDiscovery zenDiscovery = new ZenDiscovery(
settings,
threadPool,
service,
new NamedWriteableRegistry(ClusterModule.getNamedWriteables()),
masterService,
clusterApplier,
clusterSettings,
hostsResolver -> Collections.emptyList(),
OpenSearchAllocationTestCase.createAllocationService(),
Collections.emptyList(),
(s, p, r) -> {}
);
zenDiscovery.start();
return zenDiscovery;
}
private Set<DiscoveryNode> fdNodesForState(ClusterState clusterState, DiscoveryNode localNode) {
final Set<DiscoveryNode> discoveryNodes = new HashSet<>();
clusterState.getNodes()
.getNodes()
.valuesIt()
.forEachRemaining(
discoveryNode -> {
// the local node isn't part of the nodes that are pinged (don't ping ourselves)
if (discoveryNode.getId().equals(localNode.getId()) == false) {
discoveryNodes.add(discoveryNode);
}
}
);
return discoveryNodes;
}
public void testValidateOnUnsupportedIndexVersionCreated() throws Exception {
final int iters = randomIntBetween(3, 10);
for (int i = 0; i < iters; i++) {
ClusterState.Builder stateBuilder = ClusterState.builder(ClusterName.DEFAULT);
final DiscoveryNode otherNode = new DiscoveryNode(
"other_node",
buildNewFakeTransportAddress(),
emptyMap(),
DiscoveryNodeRole.BUILT_IN_ROLES,
Version.CURRENT
);
final DiscoveryNode localNode = new DiscoveryNode(
"other_node",
buildNewFakeTransportAddress(),
emptyMap(),
DiscoveryNodeRole.BUILT_IN_ROLES,
Version.CURRENT
);
MembershipAction.ValidateJoinRequestRequestHandler request = new MembershipAction.ValidateJoinRequestRequestHandler(
() -> localNode,
JoinTaskExecutor.addBuiltInJoinValidators(Collections.emptyList())
);
final boolean incompatible = randomBoolean();
IndexMetadata indexMetadata = IndexMetadata.builder("test")
.settings(
Settings.builder()
.put(
SETTING_VERSION_CREATED,
incompatible
? VersionUtils.getPreviousVersion(Version.CURRENT.minimumIndexCompatibilityVersion())
: VersionUtils.randomVersionBetween(
random(),
Version.CURRENT.minimumIndexCompatibilityVersion(),
Version.CURRENT
)
)
.put(SETTING_NUMBER_OF_SHARDS, 1)
.put(SETTING_NUMBER_OF_REPLICAS, 0)
.put(SETTING_CREATION_DATE, System.currentTimeMillis())
)
.state(IndexMetadata.State.OPEN)
.build();
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(indexMetadata.getIndex());
RoutingTable.Builder routing = new RoutingTable.Builder();
routing.addAsNew(indexMetadata);
final ShardId shardId = new ShardId("test", "_na_", 0);
IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId);
final DiscoveryNode primaryNode = otherNode;
indexShardRoutingBuilder.addShard(
TestShardRouting.newShardRouting(
"test",
0,
primaryNode.getId(),
null,
true,
ShardRoutingState.INITIALIZING,
new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "getting there")
)
);
indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build());
IndexRoutingTable indexRoutingTable = indexRoutingTableBuilder.build();
IndexMetadata updatedIndexMetadata = updateActiveAllocations(indexRoutingTable, indexMetadata);
stateBuilder.metadata(Metadata.builder().put(updatedIndexMetadata, false).generateClusterUuidIfNeeded())
.routingTable(RoutingTable.builder().add(indexRoutingTable).build());
if (incompatible) {
IllegalStateException ex = expectThrows(
IllegalStateException.class,
() -> request.messageReceived(new ValidateJoinRequest(stateBuilder.build()), null, null)
);
assertEquals(
"index [test] version not supported: "
+ VersionUtils.getPreviousVersion(Version.CURRENT.minimumIndexCompatibilityVersion())
+ " minimum compatible index version is: "
+ Version.CURRENT.minimumIndexCompatibilityVersion(),
ex.getMessage()
);
} else {
AtomicBoolean sendResponse = new AtomicBoolean(false);
request.messageReceived(new ValidateJoinRequest(stateBuilder.build()), new TransportChannel() {
@Override
public String getProfileName() {
return null;
}
@Override
public String getChannelType() {
return null;
}
@Override
public void sendResponse(TransportResponse response) throws IOException {
sendResponse.set(true);
}
@Override
public void sendResponse(Exception exception) throws IOException {
}
}, null);
assertTrue(sendResponse.get());
}
}
}
public void testIncomingClusterStateValidation() throws Exception {
ClusterName clusterName = new ClusterName("abc");
DiscoveryNodes.Builder currentNodes = DiscoveryNodes.builder()
.add(new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT))
.localNodeId("a");
ClusterState previousState = ClusterState.builder(clusterName).nodes(currentNodes).build();
logger.info("--> testing acceptances of any master when having no master");
ClusterState state = ClusterState.builder(previousState)
.nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId(randomAlphaOfLength(10)))
.incrementVersion()
.build();
ZenDiscovery.validateIncomingState(logger, state, previousState);
// now set a master node
previousState = state;
state = ClusterState.builder(previousState).nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId("master")).build();
logger.info("--> testing rejection of another master");
try {
ZenDiscovery.validateIncomingState(logger, state, previousState);
fail("node accepted state from another master");
} catch (IllegalStateException OK) {
assertThat(OK.toString(), containsString("cluster state from a different master than the current one, rejecting"));
}
logger.info("--> test state from the current master is accepted");
previousState = state;
ZenDiscovery.validateIncomingState(
logger,
ClusterState.builder(previousState)
.nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId("master"))
.incrementVersion()
.build(),
previousState
);
logger.info("--> testing rejection of another cluster name");
try {
ZenDiscovery.validateIncomingState(
logger,
ClusterState.builder(new ClusterName(randomAlphaOfLength(10))).nodes(previousState.nodes()).build(),
previousState
);
fail("node accepted state with another cluster name");
} catch (IllegalStateException OK) {
assertThat(OK.toString(), containsString("received state from a node that is not part of the cluster"));
}
logger.info("--> testing rejection of a cluster state with wrong local node");
try {
state = ClusterState.builder(previousState)
.nodes(DiscoveryNodes.builder(previousState.nodes()).localNodeId("_non_existing_").build())
.incrementVersion()
.build();
ZenDiscovery.validateIncomingState(logger, state, previousState);
fail("node accepted state with non-existence local node");
} catch (IllegalStateException OK) {
assertThat(OK.toString(), containsString("received state with a local node that does not match the current local node"));
}
try {
DiscoveryNode otherNode = new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
state = ClusterState.builder(previousState)
.nodes(DiscoveryNodes.builder(previousState.nodes()).add(otherNode).localNodeId(otherNode.getId()).build())
.incrementVersion()
.build();
ZenDiscovery.validateIncomingState(logger, state, previousState);
fail("node accepted state with existent but wrong local node");
} catch (IllegalStateException OK) {
assertThat(OK.toString(), containsString("received state with a local node that does not match the current local node"));
}
logger.info("--> testing acceptance of an old cluster state");
final ClusterState incomingState = previousState;
previousState = ClusterState.builder(previousState).incrementVersion().build();
final ClusterState finalPreviousState = previousState;
final IllegalStateException e = expectThrows(
IllegalStateException.class,
() -> ZenDiscovery.validateIncomingState(logger, incomingState, finalPreviousState)
);
final String message = String.format(
Locale.ROOT,
"rejecting cluster state version [%d] uuid [%s] received from [%s]",
incomingState.version(),
incomingState.stateUUID(),
incomingState.nodes().getMasterNodeId()
);
assertThat(e, hasToString("java.lang.IllegalStateException: " + message));
ClusterState higherVersionState = ClusterState.builder(previousState).incrementVersion().build();
// remove the master of the node (but still have a previous cluster state with it)!
higherVersionState = ClusterState.builder(higherVersionState)
.nodes(DiscoveryNodes.builder(higherVersionState.nodes()).masterNodeId(null))
.build();
// an older version from a *new* master is also OK!
state = ClusterState.builder(previousState)
.nodes(DiscoveryNodes.builder(previousState.nodes()).masterNodeId("_new_master_").build())
.build();
ZenDiscovery.validateIncomingState(logger, state, higherVersionState);
}
public void testNotEnoughMasterNodesAfterRemove() throws Exception {
final ElectMasterService electMasterService = mock(ElectMasterService.class);
when(electMasterService.hasEnoughMasterNodes(any(Iterable.class))).thenReturn(false);
final AllocationService allocationService = mock(AllocationService.class);
final AtomicBoolean rejoinCalled = new AtomicBoolean();
final Consumer<String> submitRejoin = source -> rejoinCalled.set(true);
final AtomicReference<ClusterState> remainingNodesClusterState = new AtomicReference<>();
final ZenNodeRemovalClusterStateTaskExecutor executor = new ZenNodeRemovalClusterStateTaskExecutor(
allocationService,
electMasterService,
submitRejoin,
logger
) {
@Override
protected ClusterState remainingNodesClusterState(ClusterState currentState, DiscoveryNodes.Builder remainingNodesBuilder) {
remainingNodesClusterState.set(super.remainingNodesClusterState(currentState, remainingNodesBuilder));
return remainingNodesClusterState.get();
}
};
final DiscoveryNodes.Builder builder = DiscoveryNodes.builder();
final int nodes = randomIntBetween(2, 16);
final List<NodeRemovalClusterStateTaskExecutor.Task> tasks = new ArrayList<>();
// to ensure there is at least one removal
boolean first = true;
for (int i = 0; i < nodes; i++) {
final DiscoveryNode node = node(i);
builder.add(node);
if (first || randomBoolean()) {
tasks.add(new NodeRemovalClusterStateTaskExecutor.Task(node, randomBoolean() ? "left" : "failed"));
}
first = false;
}
final ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(builder).build();
final ClusterStateTaskExecutor.ClusterTasksResult<NodeRemovalClusterStateTaskExecutor.Task> result = executor.execute(
clusterState,
tasks
);
verify(electMasterService).hasEnoughMasterNodes(eq(remainingNodesClusterState.get().nodes()));
verify(electMasterService).countMasterNodes(eq(remainingNodesClusterState.get().nodes()));
verify(electMasterService).minimumMasterNodes();
verifyNoMoreInteractions(electMasterService);
// ensure that we did not reroute
verifyNoMoreInteractions(allocationService);
assertTrue(rejoinCalled.get());
assertThat(result.resultingState, CoreMatchers.equalTo(clusterState));
for (final NodeRemovalClusterStateTaskExecutor.Task task : tasks) {
assertNotNull(result.resultingState.nodes().get(task.node().getId()));
}
}
private DiscoveryNode node(final int id) {
return new DiscoveryNode(Integer.toString(id), buildNewFakeTransportAddress(), Version.CURRENT);
}
}

View File

@ -1,106 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.discovery.zen;
import org.opensearch.Version;
import org.opensearch.cluster.ClusterName;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.settings.Settings;
import org.opensearch.test.OpenSearchTestCase;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.hamcrest.Matchers.equalTo;
public class ZenPingTests extends OpenSearchTestCase {
public void testPingCollection() {
DiscoveryNode[] nodes = new DiscoveryNode[randomIntBetween(1, 30)];
long maxIdPerNode[] = new long[nodes.length];
DiscoveryNode masterPerNode[] = new DiscoveryNode[nodes.length];
long clusterStateVersionPerNode[] = new long[nodes.length];
ArrayList<ZenPing.PingResponse> pings = new ArrayList<>();
for (int i = 0; i < nodes.length; i++) {
nodes[i] = new DiscoveryNode("" + i, buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
}
for (int pingCount = scaledRandomIntBetween(10, nodes.length * 10); pingCount > 0; pingCount--) {
int node = randomInt(nodes.length - 1);
DiscoveryNode masterNode = null;
if (randomBoolean()) {
masterNode = nodes[randomInt(nodes.length - 1)];
}
long clusterStateVersion = randomLong();
ZenPing.PingResponse ping = new ZenPing.PingResponse(
nodes[node],
masterNode,
ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY),
clusterStateVersion
);
if (rarely()) {
// ignore some pings
continue;
}
// update max ping info
maxIdPerNode[node] = ping.id();
masterPerNode[node] = masterNode;
clusterStateVersionPerNode[node] = clusterStateVersion;
pings.add(ping);
}
// shuffle
Collections.shuffle(pings, random());
ZenPing.PingCollection collection = new ZenPing.PingCollection();
pings.forEach(collection::addPing);
List<ZenPing.PingResponse> aggregate = collection.toList();
for (ZenPing.PingResponse ping : aggregate) {
int nodeId = Integer.parseInt(ping.node().getId());
assertThat(maxIdPerNode[nodeId], equalTo(ping.id()));
assertThat(masterPerNode[nodeId], equalTo(ping.master()));
assertThat(clusterStateVersionPerNode[nodeId], equalTo(ping.getClusterStateVersion()));
maxIdPerNode[nodeId] = -1; // mark as seen
}
for (int i = 0; i < maxIdPerNode.length; i++) {
assertTrue("node " + i + " had pings but it was not found in collection", maxIdPerNode[i] <= 0);
}
}
}

View File

@ -1,90 +0,0 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
/*
* 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.
*/
/*
* Modifications Copyright OpenSearch Contributors. See
* GitHub history for details.
*/
package org.opensearch.rest.action.admin.indices;
import org.opensearch.client.node.NodeClient;
import org.opensearch.common.xcontent.NamedXContentRegistry;
import org.opensearch.test.OpenSearchTestCase;
import org.opensearch.test.rest.FakeRestRequest;
import java.io.IOException;
import java.util.Collections;
import java.util.Locale;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.hasToString;
import static org.mockito.Mockito.mock;
public class RestResizeHandlerTests extends OpenSearchTestCase {
public void testShrinkCopySettingsDeprecated() throws IOException {
final RestResizeHandler.RestShrinkIndexAction handler = new RestResizeHandler.RestShrinkIndexAction();
for (final String copySettings : new String[] { null, "", "true", "false" }) {
runTestResizeCopySettingsDeprecated(handler, "shrink", copySettings);
}
}
public void testSplitCopySettingsDeprecated() throws IOException {
final RestResizeHandler.RestSplitIndexAction handler = new RestResizeHandler.RestSplitIndexAction();
for (final String copySettings : new String[] { null, "", "true", "false" }) {
runTestResizeCopySettingsDeprecated(handler, "split", copySettings);
}
}
private void runTestResizeCopySettingsDeprecated(
final RestResizeHandler handler,
final String resizeOperation,
final String copySettings
) throws IOException {
final FakeRestRequest.Builder builder = new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).withParams(
Collections.singletonMap("copy_settings", copySettings)
).withPath(String.format(Locale.ROOT, "source/_%s/target", resizeOperation));
if (copySettings != null) {
builder.withParams(Collections.singletonMap("copy_settings", copySettings));
}
final FakeRestRequest request = builder.build();
if ("false".equals(copySettings)) {
final IllegalArgumentException e = expectThrows(
IllegalArgumentException.class,
() -> handler.prepareRequest(request, mock(NodeClient.class))
);
assertThat(e, hasToString(containsString("parameter [copy_settings] can not be explicitly set to [false]")));
} else {
handler.prepareRequest(request, mock(NodeClient.class));
if ("".equals(copySettings) || "true".equals(copySettings)) {
assertWarnings("parameter [copy_settings] is deprecated and will be removed in 8.0.0");
}
}
}
}

View File

@ -42,7 +42,6 @@ import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.store.AlreadyClosedException;
import org.opensearch.OpenSearchException;
import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction;
import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest;
import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction;
@ -92,8 +91,6 @@ import org.opensearch.common.util.concurrent.FutureUtils;
import org.opensearch.common.util.concurrent.ThreadContext;
import org.opensearch.common.util.set.Sets;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.env.ShardLockObtainFailedException;
@ -118,6 +115,7 @@ import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.opensearch.indices.recovery.RecoverySettings;
import org.opensearch.node.MockNode;
import org.opensearch.node.Node;
import org.opensearch.node.Node.DiscoverySettings;
import org.opensearch.node.NodeRoleSettings;
import org.opensearch.node.NodeService;
import org.opensearch.node.NodeValidationException;
@ -171,10 +169,7 @@ import static org.opensearch.common.unit.TimeValue.timeValueMillis;
import static org.opensearch.common.unit.TimeValue.timeValueSeconds;
import static org.opensearch.discovery.DiscoveryModule.DISCOVERY_TYPE_SETTING;
import static org.opensearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE;
import static org.opensearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE;
import static org.opensearch.discovery.DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING;
import static org.opensearch.discovery.FileBasedSeedHostsProvider.UNICAST_HOSTS_FILE;
import static org.opensearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING;
import static org.opensearch.test.OpenSearchTestCase.assertBusy;
import static org.opensearch.test.OpenSearchTestCase.getTestTransportType;
import static org.opensearch.test.OpenSearchTestCase.randomFrom;
@ -183,7 +178,6 @@ import static org.opensearch.test.NodeRoles.masterOnlyNode;
import static org.opensearch.test.NodeRoles.noRoles;
import static org.opensearch.test.NodeRoles.onlyRole;
import static org.opensearch.test.NodeRoles.removeRoles;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -462,10 +456,6 @@ public final class InternalTestCluster extends TestCluster {
);
}
private static boolean usingZen1(Settings settings) {
return ZEN_DISCOVERY_TYPE.equals(DISCOVERY_TYPE_SETTING.get(settings));
}
/**
* Sets {@link #bootstrapMasterNodeIndex} to the given value, see {@link #bootstrapMasterNodeWithSpecifiedIndex(List)}
* for the description of how this field is used.
@ -483,11 +473,6 @@ public final class InternalTestCluster extends TestCluster {
return clusterName;
}
/** returns true if the {@link ElectMasterService#DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING} setting is auto managed by this cluster */
public boolean getAutoManageMinMasterNode() {
return autoManageMasterNodes;
}
public String[] getNodeNames() {
return nodes.keySet().toArray(Strings.EMPTY_ARRAY);
}
@ -741,34 +726,13 @@ public final class InternalTestCluster extends TestCluster {
final String discoveryType = DISCOVERY_TYPE_SETTING.get(updatedSettings.build());
final boolean usingSingleNodeDiscovery = discoveryType.equals("single-node");
final boolean usingZen1 = usingZen1(updatedSettings.build());
if (usingSingleNodeDiscovery == false) {
if (autoManageMasterNodes) {
assertThat(
"min master nodes may not be set when master nodes are auto managed",
updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()),
nullValue()
);
assertThat(
"if master nodes are automatically managed then nodes must complete a join cycle when starting",
updatedSettings.get(INITIAL_STATE_TIMEOUT_SETTING.getKey()),
updatedSettings.get(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey()),
nullValue()
);
if (usingZen1) {
updatedSettings
// don't wait too long not to slow down tests
.put(ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.getKey(), "5s")
.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), defaultMinMasterNodes);
}
} else {
if (usingZen1) {
assertThat(
DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " must be configured",
updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()),
not(nullValue())
);
}
}
}
@ -1080,10 +1044,6 @@ public final class InternalTestCluster extends TestCluster {
Settings.Builder newSettings = Settings.builder();
newSettings.put(callbackSettings);
if (minMasterNodes >= 0) {
if (usingZen1(newSettings.build())) {
assertFalse("min master nodes is auto managed", DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(newSettings.build()));
newSettings.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes);
}
if (INITIAL_MASTER_NODES_SETTING.exists(callbackSettings) == false) {
newSettings.putList(INITIAL_MASTER_NODES_SETTING.getKey());
}
@ -1115,25 +1075,6 @@ public final class InternalTestCluster extends TestCluster {
.put(newSettings)
.put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newIdSeed)
.build();
if (usingZen1(finalSettings)) {
if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings) == false) {
throw new IllegalStateException(
DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " is not configured after restart of [" + name + "]"
);
}
} else {
if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings)) {
// simulating an upgrade from Zen1 to Zen2, but there's no way to remove a setting when restarting a node, so
// you have to set it to REMOVED_MINIMUM_MASTER_NODES (== Integer.MAX_VALUE) to indicate its removal:
assertTrue(DISCOVERY_TYPE_SETTING.exists(finalSettings));
assertThat(DISCOVERY_TYPE_SETTING.get(finalSettings), equalTo(ZEN2_DISCOVERY_TYPE));
assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(finalSettings), equalTo(REMOVED_MINIMUM_MASTER_NODES));
final Builder builder = Settings.builder().put(finalSettings);
builder.remove(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey());
finalSettings = builder.build();
}
}
Collection<Class<? extends Plugin>> plugins = node.getClasspathPlugins();
node = new MockNode(finalSettings, plugins);
node.injector().getInstance(TransportService.class).addLifecycleListener(new LifecycleListener() {
@ -1247,9 +1188,6 @@ public final class InternalTestCluster extends TestCluster {
if (wipeData) {
wipePendingDataDirectories();
}
if (nodes.size() > 0 && autoManageMasterNodes) {
updateMinMasterNodes(getMasterNodesCount());
}
logger.debug(
"Cluster hasn't changed - moving out - nodes: [{}] nextNodeId: [{}] numSharedNodes: [{}]",
nodes.keySet(),
@ -1891,13 +1829,6 @@ public final class InternalTestCluster extends TestCluster {
.filter(nac -> nodes.containsKey(nac.name) == false) // filter out old masters
.count();
final int currentMasters = getMasterNodesCount();
if (autoManageMasterNodes
&& currentMasters > 0
&& newMasters > 0
&& getMinMasterNodes(currentMasters + newMasters) <= currentMasters) {
// if we're adding too many master-eligible nodes at once, we can't update the min master setting before adding the nodes.
updateMinMasterNodes(currentMasters + newMasters);
}
rebuildUnicastHostFiles(nodeAndClients); // ensure that new nodes can find the existing nodes when they start
List<Future<?>> futures = nodeAndClients.stream().map(node -> executor.submit(node::startNode)).collect(Collectors.toList());
@ -1920,7 +1851,6 @@ public final class InternalTestCluster extends TestCluster {
&& getMinMasterNodes(currentMasters + newMasters) > currentMasters) {
// update once masters have joined
validateClusterFormed();
updateMinMasterNodes(currentMasters + newMasters);
}
}
}
@ -2053,10 +1983,6 @@ public final class InternalTestCluster extends TestCluster {
// we have to validate cluster size to ensure that the restarted node has rejoined the cluster if it was master-eligible;
validateClusterFormed();
}
if (excludedNodeIds.isEmpty() == false) {
updateMinMasterNodes(getMasterNodesCount());
}
}
private NodeAndClient removeNode(NodeAndClient nodeAndClient) {
@ -2094,10 +2020,6 @@ public final class InternalTestCluster extends TestCluster {
throw new AssertionError("unexpected", e);
}
}
if (stoppingMasters > 0) {
updateMinMasterNodes(getMasterNodesCount() - Math.toIntExact(stoppingMasters));
}
}
return excludedNodeNames;
}
@ -2371,35 +2293,6 @@ public final class InternalTestCluster extends TestCluster {
return startNodes(numNodes, Settings.builder().put(onlyRole(settings, DiscoveryNodeRole.DATA_ROLE)).build());
}
/**
* updates the min master nodes setting in the current running cluster.
*
* @param eligibleMasterNodeCount the number of master eligible nodes to use as basis for the min master node setting
*/
private void updateMinMasterNodes(int eligibleMasterNodeCount) {
assert autoManageMasterNodes;
final int minMasterNodes = getMinMasterNodes(eligibleMasterNodeCount);
if (getMasterNodesCount() > 0) {
// there should be at least one master to update
logger.debug("updating min_master_nodes to [{}]", minMasterNodes);
try {
assertAcked(
client().admin()
.cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes))
);
} catch (Exception e) {
throw new OpenSearchException(
"failed to update minimum master node to [{}] (current masters [{}])",
e,
minMasterNodes,
getMasterNodesCount()
);
}
}
}
/** calculates a min master nodes value based on the given number of master nodes */
private static int getMinMasterNodes(int eligibleMasterNodes) {
return eligibleMasterNodes / 2 + 1;

View File

@ -122,9 +122,6 @@ import org.opensearch.common.xcontent.XContentParser;
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.common.xcontent.smile.SmileXContent;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.Discovery;
import org.opensearch.discovery.zen.ElectMasterService;
import org.opensearch.discovery.zen.ZenDiscovery;
import org.opensearch.env.Environment;
import org.opensearch.env.TestEnvironment;
import org.opensearch.http.HttpInfo;
@ -200,7 +197,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.opensearch.client.Requests.syncedFlushRequest;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
@ -217,7 +213,6 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoTimeout;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.emptyArray;
import static org.hamcrest.Matchers.emptyIterable;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -603,33 +598,11 @@ public abstract class OpenSearchIntegTestCase extends OpenSearchTestCase {
assertThat("test leaves persistent cluster metadata behind", persistentKeys, empty());
final Set<String> transientKeys = new HashSet<>(metadata.transientSettings().keySet());
if (isInternalCluster() && internalCluster().getAutoManageMinMasterNode()) {
// this is set by the test infra
transientKeys.remove(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey());
}
assertThat("test leaves transient cluster metadata behind", transientKeys, empty());
}
ensureClusterSizeConsistency();
ensureClusterStateConsistency();
ensureClusterStateCanBeReadByNodeTool();
if (isInternalCluster()) {
// check no pending cluster states are leaked
for (Discovery discovery : internalCluster().getInstances(Discovery.class)) {
if (discovery instanceof ZenDiscovery) {
final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery;
assertBusy(() -> {
final ClusterState[] states = zenDiscovery.pendingClusterStates();
assertThat(
zenDiscovery.clusterState().nodes().getLocalNode().getName()
+ " still having pending states:\n"
+ Stream.of(states).map(ClusterState::toString).collect(Collectors.joining("\n")),
states,
emptyArray()
);
});
}
}
}
beforeIndexDeletion();
cluster().wipe(excludeTemplates()); // wipe after to make sure we fail in the test that didn't ack the delete
if (afterClass || currentClusterScope == Scope.TEST) {
@ -1825,9 +1798,8 @@ public abstract class OpenSearchIntegTestCase extends OpenSearchTestCase {
boolean supportsDedicatedMasters() default true;
/**
* Indicates whether the cluster automatically manages cluster bootstrapping and the removal of any master-eligible nodes as well
* as {@link ElectMasterService#DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING} if running the pre-7.0 cluster coordination
* implementation. If set to {@code false} then the tests must manage these things explicitly.
* Indicates whether the cluster automatically manages cluster bootstrapping. If set to {@code false} then the
* tests must manage these things explicitly.
*/
boolean autoManageMasterNodes() default true;

View File

@ -40,10 +40,10 @@ import org.opensearch.common.network.NetworkModule;
import org.opensearch.common.settings.Settings;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.discovery.DiscoveryModule;
import org.opensearch.discovery.DiscoverySettings;
import org.opensearch.discovery.SettingsBasedSeedHostsProvider;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.node.Node.DiscoverySettings;
import org.opensearch.plugins.Plugin;
import org.opensearch.test.OpenSearchTestCase;
import org.opensearch.test.InternalTestCluster;