diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/UnsafeBootstrapAndDetachCommandIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/UnsafeBootstrapAndDetachCommandIT.java index 37f426fcd73..5c07ef8e7ba 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/UnsafeBootstrapAndDetachCommandIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/UnsafeBootstrapAndDetachCommandIT.java @@ -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; diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/Zen1IT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/Zen1IT.java deleted file mode 100644 index 184ea9247f2..00000000000 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/Zen1IT.java +++ /dev/null @@ -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> 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 nodes = internalCluster().startNodes( - IntStream.range(0, 5).mapToObj(i -> i < 2 ? ZEN1_SETTINGS : ZEN2_SETTINGS).toArray(Settings[]::new) - ); - - final List 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 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 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 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 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 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 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 = 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)); - } -} diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/ZenDiscoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/ZenDiscoveryIT.java index 0c5878265b5..1baac907111 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/ZenDiscoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/ZenDiscoveryIT.java @@ -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); diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java index ea0ecc850d0..30697ba1cfe 100644 --- a/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java @@ -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 nonPreferredNodes = new HashSet<>(nodes); diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/MasterDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/MasterDisruptionIT.java index e2c628ef212..d36be59b1ac 100644 --- a/server/src/internalClusterTest/java/org/opensearch/discovery/MasterDisruptionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/discovery/MasterDisruptionIT.java @@ -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 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); - } - } } diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/single/SingleNodeDiscoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/single/SingleNodeDiscoveryIT.java index da717738b6d..d3af57158b9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/discovery/single/SingleNodeDiscoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/discovery/single/SingleNodeDiscoveryIT.java @@ -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; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java index 280c1316376..42497c52441 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -166,7 +166,6 @@ public class TransportClusterStateAction extends TransportMasterNodeReadAction 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. */ diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index b31b938f956..d5eb550ca4e 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -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); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/DiscoveryUpgradeService.java b/server/src/main/java/org/opensearch/cluster/coordination/DiscoveryUpgradeService.java deleted file mode 100644 index efb1614f67b..00000000000 --- a/server/src/main/java/org/opensearch/cluster/coordination/DiscoveryUpgradeService.java +++ /dev/null @@ -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 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 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> peersSupplier; - private final Consumer 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> peersSupplier, - Consumer 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 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 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 listener; - - ListenableCountDown(int count, ActionListener 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 knownMasterNodeIds; - - JoiningRound(boolean upgrading, int minimumMasterNodes, Set knownMasterNodeIds) { - this.upgrading = upgrading; - this.minimumMasterNodes = minimumMasterNodes; - this.knownMasterNodeIds = knownMasterNodeIds; - } - - private boolean isRunning() { - return joiningRound == this && isBootstrappedSupplier.getAsBoolean() == false; - } - - private boolean canBootstrap(Set 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 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 nodeIds = new HashSet<>(); - discoveryNodes.forEach(n -> nodeIds.add(n.getId())); - - final Iterator 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 discoveryNodes) { - final Set masterCandidates = newConcurrentSet(); - final ListenableCountDown listenableCountDown = new ListenableCountDown( - discoveryNodes.size(), - new ActionListener() { - - @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() { - @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() - ); - } -} diff --git a/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java b/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java index 208d2b63e56..c5fcee712b6 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java @@ -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() { @Override diff --git a/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java index cf573d06ffe..6d2fb99e04f 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java @@ -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 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() { @Override public Empty read(StreamInput in) { @@ -436,15 +374,9 @@ public class JoinHelper { } public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, ActionListener 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) ); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java index 42fb95e9b81..62762937d5e 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java @@ -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 { - 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() { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/NoMasterBlockService.java b/server/src/main/java/org/opensearch/cluster/coordination/NoMasterBlockService.java index 897000e4d1c..a578389f863 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/NoMasterBlockService.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/NoMasterBlockService.java @@ -71,14 +71,6 @@ public class NoMasterBlockService { EnumSet.of(ClusterBlockLevel.METADATA_WRITE) ); - public static final Setting LEGACY_NO_MASTER_BLOCK_SETTING = new Setting<>( - "discovery.zen.no_master_block", - "write", - NoMasterBlockService::parseNoMasterBlock, - Property.Dynamic, - Property.NodeScope, - Property.Deprecated - ); public static final Setting NO_MASTER_BLOCK_SETTING = new Setting<>( "cluster.no_master_block", "write", @@ -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) { diff --git a/server/src/main/java/org/opensearch/discovery/zen/PendingClusterStateStats.java b/server/src/main/java/org/opensearch/cluster/coordination/PendingClusterStateStats.java similarity index 98% rename from server/src/main/java/org/opensearch/discovery/zen/PendingClusterStateStats.java rename to server/src/main/java/org/opensearch/cluster/coordination/PendingClusterStateStats.java index 88807d2c49e..92dd926b2fa 100644 --- a/server/src/main/java/org/opensearch/discovery/zen/PendingClusterStateStats.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PendingClusterStateStats.java @@ -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; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java b/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java index 62e5446bc78..ee97c0e07eb 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PublicationTransportHandler.java @@ -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 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 transportCommitCallback(TransportChannel channel) { @@ -408,19 +371,10 @@ public class PublicationTransportHandler { ActionListener 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() { @@ -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); diff --git a/server/src/main/java/org/opensearch/discovery/zen/PublishClusterStateStats.java b/server/src/main/java/org/opensearch/cluster/coordination/PublishClusterStateStats.java similarity index 98% rename from server/src/main/java/org/opensearch/discovery/zen/PublishClusterStateStats.java rename to server/src/main/java/org/opensearch/cluster/coordination/PublishClusterStateStats.java index 3e6b3a84f4b..77320810eba 100644 --- a/server/src/main/java/org/opensearch/discovery/zen/PublishClusterStateStats.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PublishClusterStateStats.java @@ -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; diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 7a189ebc261..57257bd2063 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -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 ) ) ); diff --git a/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java b/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java index 23495a8afc7..427615da7e4 100644 --- a/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java @@ -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 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 + "]"); } diff --git a/server/src/main/java/org/opensearch/discovery/DiscoverySettings.java b/server/src/main/java/org/opensearch/discovery/DiscoverySettings.java deleted file mode 100644 index c20d922bc55..00000000000 --- a/server/src/main/java/org/opensearch/discovery/DiscoverySettings.java +++ /dev/null @@ -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 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 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 PUBLISH_DIFF_ENABLE_SETTING = Setting.boolSetting( - "discovery.zen.publish_diff.enable", - true, - Property.Dynamic, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting INITIAL_STATE_TIMEOUT_SETTING = Setting.positiveTimeSetting( - "discovery.initial_state_timeout", - TimeValue.timeValueSeconds(30), - Property.NodeScope - ); - - private volatile 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; - } - -} diff --git a/server/src/main/java/org/opensearch/discovery/DiscoveryStats.java b/server/src/main/java/org/opensearch/discovery/DiscoveryStats.java index 230e2462c05..159e997bdf7 100644 --- a/server/src/main/java/org/opensearch/discovery/DiscoveryStats.java +++ b/server/src/main/java/org/opensearch/discovery/DiscoveryStats.java @@ -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; diff --git a/server/src/main/java/org/opensearch/discovery/PeerFinder.java b/server/src/main/java/org/opensearch/discovery/PeerFinder.java index ab0a013f588..37f07c5d56a 100644 --- a/server/src/main/java/org/opensearch/discovery/PeerFinder.java +++ b/server/src/main/java/org/opensearch/discovery/PeerFinder.java @@ -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 optionalMasterNode = Arrays.stream(ucResponse.pingResponses) - .filter(pr -> discoveryNode.equals(pr.node()) && discoveryNode.equals(pr.master())) - .map(ZenPing.PingResponse::node) - .findFirst(); - List 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 { - @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 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]))); - } - } } diff --git a/server/src/main/java/org/opensearch/discovery/zen/ElectMasterService.java b/server/src/main/java/org/opensearch/discovery/zen/ElectMasterService.java deleted file mode 100644 index 9fec972660e..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/ElectMasterService.java +++ /dev/null @@ -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 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 nodes) { - int count = 0; - for (DiscoveryNode node : nodes) { - if (node.isMasterNode()) { - count++; - } - } - return count; - } - - public boolean hasEnoughCandidates(Collection 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 candidates) { - assert hasEnoughCandidates(candidates); - List 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 activeMasters) { - return activeMasters.stream().min(ElectMasterService::compareNodes).get(); - } - - public boolean hasEnoughMasterNodes(Iterable nodes) { - final int count = countMasterNodes(nodes); - return count > 0 && (minimumMasterNodes < 0 || count >= minimumMasterNodes); - } - - public boolean hasTooManyMasterNodes(Iterable 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 sortByMasterLikelihood(Iterable nodes) { - ArrayList sortedNodes = CollectionUtils.iterableAsArrayList(nodes); - CollectionUtil.introSort(sortedNodes, ElectMasterService::compareNodes); - return sortedNodes; - } - - /** - * Returns a list of the next possible masters. - */ - public DiscoveryNode[] nextPossibleMasters(ObjectContainer nodes, int numberOfPossibleMasters) { - List sortedNodes = sortedMasterNodes(Arrays.asList(nodes.toArray(DiscoveryNode.class))); - if (sortedNodes == null) { - return new DiscoveryNode[0]; - } - List 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 sortedMasterNodes(Iterable nodes) { - List 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()); - } -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/FaultDetection.java b/server/src/main/java/org/opensearch/discovery/zen/FaultDetection.java deleted file mode 100644 index 4abf1154019..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/FaultDetection.java +++ /dev/null @@ -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} & {@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 CONNECT_ON_NETWORK_DISCONNECT_SETTING = Setting.boolSetting( - "discovery.zen.fd.connect_on_network_disconnect", - false, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting PING_INTERVAL_SETTING = Setting.positiveTimeSetting( - "discovery.zen.fd.ping_interval", - timeValueSeconds(1), - Property.NodeScope, - Property.Deprecated - ); - public static final Setting PING_TIMEOUT_SETTING = Setting.timeSetting( - "discovery.zen.fd.ping_timeout", - timeValueSeconds(30), - Property.NodeScope, - Property.Deprecated - ); - public static final Setting PING_RETRIES_SETTING = Setting.intSetting( - "discovery.zen.fd.ping_retries", - 3, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting 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); - } - } - -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/MasterFaultDetection.java b/server/src/main/java/org/opensearch/discovery/zen/MasterFaultDetection.java deleted file mode 100644 index 4456e91129a..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/MasterFaultDetection.java +++ /dev/null @@ -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 clusterStateSupplier; - private final CopyOnWriteArrayList 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 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() { - @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 { - - @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 {} - } -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/MembershipAction.java b/server/src/main/java/org/opensearch/discovery/zen/MembershipAction.java deleted file mode 100644 index b15f07f89da..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/MembershipAction.java +++ /dev/null @@ -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> 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 { - - @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 { - private final Supplier localNodeSupplier; - private final Collection> joinValidators; - - ValidateJoinRequestRequestHandler( - Supplier localNodeSupplier, - Collection> 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 { - - @Override - public void messageReceived(LeaveRequest request, TransportChannel channel, Task task) throws Exception { - listener.onLeave(request.node); - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } - } -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/NodeJoinController.java b/server/src/main/java/org/opensearch/discovery/zen/NodeJoinController.java deleted file mode 100644 index 08f4a9e3d0b..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/NodeJoinController.java +++ /dev/null @@ -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 - *

- * You must start accumulating joins before calling this method. See {@link #startElectionContext()} - *

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

- * 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> 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 getPendingAsTasks(String reason) { - Map 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 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 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 callbacks; - private final Logger logger; - - JoinTaskListener(MembershipAction.JoinCallback callback, Logger logger) { - this(Collections.singletonList(callback), logger); - } - - JoinTaskListener(List 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); - } - } - } - } - -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/NodesFaultDetection.java b/server/src/main/java/org/opensearch/discovery/zen/NodesFaultDetection.java deleted file mode 100644 index 740d8ee289e..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/NodesFaultDetection.java +++ /dev/null @@ -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 listeners = new CopyOnWriteArrayList<>(); - - private final ConcurrentMap nodesFD = newConcurrentMap(); - - private final Supplier clusterStateSupplier; - - private volatile DiscoveryNode localNode; - - public NodesFaultDetection( - Settings settings, - ThreadPool threadPool, - TransportService transportService, - Supplier 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 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() { - @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 { - @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 {} - } -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/PendingClusterStatesQueue.java b/server/src/main/java/org/opensearch/discovery/zen/PendingClusterStatesQueue.java deleted file mode 100644 index 14c69a8d585..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/PendingClusterStatesQueue.java +++ /dev/null @@ -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. - *

- * 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 & 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. - *

- * The class is fully thread safe and can be used concurrently. - */ -public class PendingClusterStatesQueue { - - interface StateProcessedListener { - - void onNewClusterStateProcessed(); - - void onNewClusterStateFailed(Exception e); - } - - final ArrayList 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 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. - *

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

- * 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 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); - } - -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/PingContextProvider.java b/server/src/main/java/org/opensearch/discovery/zen/PingContextProvider.java deleted file mode 100644 index 035d2a95792..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/PingContextProvider.java +++ /dev/null @@ -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(); -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/PublishClusterStateAction.java b/server/src/main/java/org/opensearch/discovery/zen/PublishClusterStateAction.java deleted file mode 100644 index 2f7dd13e62a..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/PublishClusterStateAction.java +++ /dev/null @@ -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 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. - *

- * 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 nodesToPublishTo; - final Map serializedStates; - final Map 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 nodesToPublishTo, - final SendingController sendingController, - final Discovery.AckListener ackListener, - final boolean sendFullVersion, - final Map serializedStates, - final Map 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 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 nodesToPublishTo, - boolean sendFullVersion, - Map serializedStates, - Map serializedDiffs - ) { - Diff 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 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 serializedDiffs, - Map 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 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 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() { - - @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 { - - @Override - public void messageReceived(BytesTransportRequest request, final TransportChannel channel, Task task) throws Exception { - handleIncomingClusterStateRequest(request, channel); - } - } - - private class CommitClusterStateRequestHandler implements TransportRequestHandler { - @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 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() - ); - } -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/UnicastZenPing.java b/server/src/main/java/org/opensearch/discovery/zen/UnicastZenPing.java deleted file mode 100644 index 75c974f66af..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/UnicastZenPing.java +++ /dev/null @@ -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 activePingingRounds = newConcurrentMap(); - - // a list of temporal responses a node will return for a request (holds responses from other nodes) - private final Queue 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 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 resultsConsumer, final TimeValue scheduleDuration, final TimeValue requestDuration) { - final List 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 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 tempConnections = new HashMap<>(); - private final KeyedLock connectionLock = new KeyedLock<>(true); - private final PingCollection pingCollection; - private final List seedAddresses; - private final Consumer pingListener; - private final DiscoveryNode localNode; - private final ConnectionProfile connectionProfile; - - private AtomicBoolean closed = new AtomicBoolean(false); - - PingingRound( - int id, - List seedAddresses, - Consumer 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 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 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 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 uniqueAddresses = Stream.concat(pingingRound.getSeedAddresses().stream(), temporalAddresses.stream()) - .distinct(); - - // resolve what we can via the latest cluster state - final Set 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 getPingResponseHandler( - final PingingRound pingingRound, - final DiscoveryNode node - ) { - return new TransportResponseHandler() { - - @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 pingResponses = CollectionUtils.iterableAsArrayList(temporalResponses); - pingResponses.add(createPingResponse(contextProvider.clusterState())); - - return new UnicastPingResponse(request.id, pingResponses.toArray(new PingResponse[pingResponses.size()])); - } - - class UnicastPingRequestHandler implements TransportRequestHandler { - - @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 - } - -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/ZenDiscovery.java b/server/src/main/java/org/opensearch/discovery/zen/ZenDiscovery.java deleted file mode 100644 index 368359b48e7..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/ZenDiscovery.java +++ /dev/null @@ -1,1410 +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.ExceptionsHelper; -import org.opensearch.OpenSearchException; -import org.opensearch.action.ActionListener; -import org.opensearch.cluster.ClusterChangedEvent; -import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.ClusterStateTaskConfig; -import org.opensearch.cluster.NotMasterException; -import org.opensearch.cluster.block.ClusterBlocks; -import org.opensearch.cluster.coordination.FailedToCommitClusterStateException; -import org.opensearch.cluster.coordination.JoinTaskExecutor; -import org.opensearch.cluster.coordination.NoMasterBlockService; -import org.opensearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.RerouteService; -import org.opensearch.cluster.routing.allocation.AllocationService; -import org.opensearch.cluster.service.ClusterApplier; -import org.opensearch.cluster.service.ClusterApplier.ClusterApplyListener; -import org.opensearch.cluster.service.MasterService; -import org.opensearch.common.Priority; -import org.opensearch.common.component.AbstractLifecycleComponent; -import org.opensearch.common.component.Lifecycle; -import org.opensearch.common.io.stream.NamedWriteableRegistry; -import org.opensearch.common.io.stream.StreamInput; -import org.opensearch.common.io.stream.StreamOutput; -import org.opensearch.common.lease.Releasables; -import org.opensearch.common.logging.LoggerMessageFormat; -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; -import org.opensearch.core.internal.io.IOUtils; -import org.opensearch.discovery.Discovery; -import org.opensearch.discovery.DiscoverySettings; -import org.opensearch.discovery.DiscoveryStats; -import org.opensearch.discovery.SeedHostsProvider; -import org.opensearch.discovery.zen.PublishClusterStateAction.IncomingClusterStateListener; -import org.opensearch.tasks.Task; -import org.opensearch.threadpool.ThreadPool; -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.TransportResponse; -import org.opensearch.transport.TransportService; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Locale; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -import static org.opensearch.common.unit.TimeValue.timeValueSeconds; -import static org.opensearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; - -public class ZenDiscovery extends AbstractLifecycleComponent implements Discovery, PingContextProvider, IncomingClusterStateListener { - private static final Logger logger = LogManager.getLogger(ZenDiscovery.class); - - public static final Setting PING_TIMEOUT_SETTING = Setting.positiveTimeSetting( - "discovery.zen.ping_timeout", - timeValueSeconds(3), - Property.NodeScope - ); - public static final Setting JOIN_TIMEOUT_SETTING = Setting.timeSetting( - "discovery.zen.join_timeout", - settings -> TimeValue.timeValueMillis(PING_TIMEOUT_SETTING.get(settings).millis() * 20), - TimeValue.timeValueMillis(0), - Property.NodeScope, - Property.Deprecated - ); - public static final Setting JOIN_RETRY_ATTEMPTS_SETTING = Setting.intSetting( - "discovery.zen.join_retry_attempts", - 3, - 1, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting JOIN_RETRY_DELAY_SETTING = Setting.positiveTimeSetting( - "discovery.zen.join_retry_delay", - TimeValue.timeValueMillis(100), - Property.NodeScope, - Property.Deprecated - ); - public static final Setting MAX_PINGS_FROM_ANOTHER_MASTER_SETTING = Setting.intSetting( - "discovery.zen.max_pings_from_another_master", - 3, - 1, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting SEND_LEAVE_REQUEST_SETTING = Setting.boolSetting( - "discovery.zen.send_leave_request", - true, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING = Setting.timeSetting( - "discovery.zen.master_election.wait_for_joins_timeout", - settings -> TimeValue.timeValueMillis(JOIN_TIMEOUT_SETTING.get(settings).millis() / 2), - TimeValue.timeValueMillis(0), - Property.NodeScope, - Property.Deprecated - ); - public static final Setting MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING = Setting.boolSetting( - "discovery.zen.master_election.ignore_non_master_pings", - false, - Property.NodeScope, - Property.Deprecated - ); - public static final Setting MAX_PENDING_CLUSTER_STATES_SETTING = Setting.intSetting( - "discovery.zen.publish.max_pending_cluster_states", - 25, - 1, - Property.NodeScope, - Property.Deprecated - ); - - public static final String DISCOVERY_REJOIN_ACTION_NAME = "internal:discovery/zen/rejoin"; - - private final TransportService transportService; - private final MasterService masterService; - private final DiscoverySettings discoverySettings; - private final NoMasterBlockService noMasterBlockService; - protected final ZenPing zenPing; // protected to allow tests access - private final MasterFaultDetection masterFD; - private final NodesFaultDetection nodesFD; - private final PublishClusterStateAction publishClusterState; - private final MembershipAction membership; - private final ClusterName clusterName; - private final ThreadPool threadPool; - - private final TimeValue pingTimeout; - private final TimeValue joinTimeout; - - /** how many retry attempts to perform if join request failed with an retryable error */ - private final int joinRetryAttempts; - /** how long to wait before performing another join attempt after a join request failed with an retryable error */ - private final TimeValue joinRetryDelay; - - /** how many pings from *another* master to tolerate before forcing a rejoin on other or local master */ - private final int maxPingsFromAnotherMaster; - - // a flag that should be used only for testing - private final boolean sendLeaveRequest; - - private final ElectMasterService electMaster; - - private final boolean masterElectionIgnoreNonMasters; - private final TimeValue masterElectionWaitForJoinsTimeout; - - private final JoinThreadControl joinThreadControl; - - private final PendingClusterStatesQueue pendingStatesQueue; - - private final NodeJoinController nodeJoinController; - private final NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor; - private final ClusterApplier clusterApplier; - private final AtomicReference committedState; // last committed cluster state - private final Object stateMutex = new Object(); - private final Collection> onJoinValidators; - - public ZenDiscovery( - Settings settings, - ThreadPool threadPool, - TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, - MasterService masterService, - ClusterApplier clusterApplier, - ClusterSettings clusterSettings, - SeedHostsProvider hostsProvider, - AllocationService allocationService, - Collection> onJoinValidators, - RerouteService rerouteService - ) { - this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators); - this.masterService = masterService; - this.clusterApplier = clusterApplier; - this.transportService = transportService; - this.discoverySettings = new DiscoverySettings(settings, clusterSettings); - this.noMasterBlockService = new NoMasterBlockService(settings, clusterSettings); - this.zenPing = newZenPing(settings, threadPool, transportService, hostsProvider); - this.electMaster = new ElectMasterService(settings); - this.pingTimeout = PING_TIMEOUT_SETTING.get(settings); - this.joinTimeout = JOIN_TIMEOUT_SETTING.get(settings); - this.joinRetryAttempts = JOIN_RETRY_ATTEMPTS_SETTING.get(settings); - this.joinRetryDelay = JOIN_RETRY_DELAY_SETTING.get(settings); - this.maxPingsFromAnotherMaster = MAX_PINGS_FROM_ANOTHER_MASTER_SETTING.get(settings); - this.sendLeaveRequest = SEND_LEAVE_REQUEST_SETTING.get(settings); - this.threadPool = threadPool; - this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings); - this.committedState = new AtomicReference<>(); - - this.masterElectionIgnoreNonMasters = MASTER_ELECTION_IGNORE_NON_MASTER_PINGS_SETTING.get(settings); - this.masterElectionWaitForJoinsTimeout = MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.get(settings); - - logger.debug( - "using ping_timeout [{}], join.timeout [{}], master_election.ignore_non_master [{}]", - this.pingTimeout, - joinTimeout, - masterElectionIgnoreNonMasters - ); - - clusterSettings.addSettingsUpdateConsumer( - ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, - this::handleMinimumMasterNodesChanged, - (value) -> { - final ClusterState clusterState = this.clusterState(); - int masterNodes = clusterState.nodes().getMasterNodes().size(); - // the purpose of this validation is to make sure that the master doesn't step down - // due to a change in master nodes, which also means that there is no way to revert - // an accidental change. Since we validate using the current cluster state (and - // not the one from which the settings come from) we have to be careful and only - // validate if the local node is already a master. Doing so all the time causes - // subtle issues. For example, a node that joins a cluster has no nodes in its - // current cluster state. When it receives a cluster state from the master with - // a dynamic minimum master nodes setting int it, we must make sure we don't reject - // it. - - if (clusterState.nodes().isLocalNodeElectedMaster() && value > masterNodes) { - throw new IllegalArgumentException( - "cannot set " - + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() - + " to more than the current" - + " master nodes count [" - + masterNodes - + "]" - ); - } - } - ); - - this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, this::clusterState, masterService, clusterName); - this.masterFD.addListener(new MasterNodeFailureListener()); - this.nodesFD = new NodesFaultDetection(settings, threadPool, transportService, this::clusterState, clusterName); - this.nodesFD.addListener(new NodeFaultDetectionListener()); - this.pendingStatesQueue = new PendingClusterStatesQueue(logger, MAX_PENDING_CLUSTER_STATES_SETTING.get(settings)); - - this.publishClusterState = new PublishClusterStateAction(transportService, namedWriteableRegistry, this, discoverySettings); - this.membership = new MembershipAction(transportService, new MembershipListener(), onJoinValidators); - this.joinThreadControl = new JoinThreadControl(); - - this.nodeJoinController = new NodeJoinController(settings, masterService, allocationService, electMaster, rerouteService); - this.nodeRemovalExecutor = new ZenNodeRemovalClusterStateTaskExecutor(allocationService, electMaster, this::submitRejoin, logger); - - masterService.setClusterStateSupplier(this::clusterState); - - transportService.registerRequestHandler( - DISCOVERY_REJOIN_ACTION_NAME, - ThreadPool.Names.SAME, - RejoinClusterRequest::new, - new RejoinClusterRequestHandler() - ); - } - - // protected to allow overriding in tests - protected ZenPing newZenPing( - Settings settings, - ThreadPool threadPool, - TransportService transportService, - SeedHostsProvider hostsProvider - ) { - return new UnicastZenPing(settings, threadPool, transportService, hostsProvider, this); - } - - @Override - protected void doStart() { - DiscoveryNode localNode = transportService.getLocalNode(); - assert localNode != null; - synchronized (stateMutex) { - // set initial state - assert committedState.get() == null; - assert localNode != null; - ClusterState.Builder builder = ClusterState.builder(clusterName); - ClusterState initialState = builder.blocks( - ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK).addGlobalBlock(noMasterBlockService.getNoMasterBlock()) - ).nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); - committedState.set(initialState); - clusterApplier.setInitialState(initialState); - nodesFD.setLocalNode(localNode); - joinThreadControl.start(); - } - zenPing.start(); - } - - @Override - public void startInitialJoin() { - // start the join thread from a cluster state update. See {@link JoinThreadControl} for details. - synchronized (stateMutex) { - // do the join on a different thread, the caller of this method waits for 30s anyhow till it is discovered - joinThreadControl.startNewThreadIfNotRunning(); - } - } - - @Override - protected void doStop() { - joinThreadControl.stop(); - masterFD.stop("zen disco stop"); - nodesFD.stop(); - Releasables.close(zenPing); // stop any ongoing pinging - DiscoveryNodes nodes = clusterState().nodes(); - if (sendLeaveRequest) { - if (nodes.getMasterNode() == null) { - // if we don't know who the master is, nothing to do here - } else if (!nodes.isLocalNodeElectedMaster()) { - try { - membership.sendLeaveRequestBlocking(nodes.getMasterNode(), nodes.getLocalNode(), TimeValue.timeValueSeconds(1)); - } catch (Exception e) { - logger.debug(() -> new ParameterizedMessage("failed to send leave request to master [{}]", nodes.getMasterNode()), e); - } - } else { - // we're master -> let other potential master we left and start a master election now rather then wait for masterFD - DiscoveryNode[] possibleMasters = electMaster.nextPossibleMasters(nodes.getNodes().values(), 5); - for (DiscoveryNode possibleMaster : possibleMasters) { - if (nodes.getLocalNode().equals(possibleMaster)) { - continue; - } - try { - membership.sendLeaveRequest(nodes.getLocalNode(), possibleMaster); - } catch (Exception e) { - logger.debug( - () -> new ParameterizedMessage( - "failed to send leave request from master [{}] to possible master [{}]", - nodes.getMasterNode(), - possibleMaster - ), - e - ); - } - } - } - } - } - - @Override - protected void doClose() throws IOException { - IOUtils.close(masterFD, nodesFD); - } - - @Override - public ClusterState clusterState() { - ClusterState clusterState = committedState.get(); - assert clusterState != null : "accessing cluster state before it is set"; - return clusterState; - } - - @Override - public void publish(ClusterChangedEvent clusterChangedEvent, ActionListener publishListener, AckListener ackListener) { - ClusterState newState = clusterChangedEvent.state(); - assert newState.getNodes().isLocalNodeElectedMaster() : "Shouldn't publish state when not master " + clusterChangedEvent.source(); - - try { - - // state got changed locally (maybe because another master published to us) - if (clusterChangedEvent.previousState() != this.committedState.get()) { - throw new FailedToCommitClusterStateException("state was mutated while calculating new CS update"); - } - - pendingStatesQueue.addPending(newState); - - publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener); - } catch (FailedToCommitClusterStateException t) { - // cluster service logs a WARN message - logger.debug( - "failed to publish cluster state version [{}] (not enough nodes acknowledged, min master nodes [{}])", - newState.version(), - electMaster.minimumMasterNodes() - ); - - synchronized (stateMutex) { - pendingStatesQueue.failAllStatesAndClear(new OpenSearchException("failed to publish cluster state")); - - rejoin("zen-disco-failed-to-publish"); - } - - publishListener.onFailure(t); - return; - } - - final DiscoveryNode localNode = newState.getNodes().getLocalNode(); - final AtomicBoolean processedOrFailed = new AtomicBoolean(); - pendingStatesQueue.markAsCommitted(newState.stateUUID(), new PendingClusterStatesQueue.StateProcessedListener() { - @Override - public void onNewClusterStateProcessed() { - processedOrFailed.set(true); - publishListener.onResponse(null); - ackListener.onNodeAck(localNode, null); - } - - @Override - public void onNewClusterStateFailed(Exception e) { - processedOrFailed.set(true); - publishListener.onFailure(e); - ackListener.onNodeAck(localNode, e); - logger.warn( - () -> new ParameterizedMessage("failed while applying cluster state locally [{}]", clusterChangedEvent.source()), - e - ); - } - }); - - synchronized (stateMutex) { - if (clusterChangedEvent.previousState() != this.committedState.get()) { - publishListener.onFailure( - new FailedToCommitClusterStateException("local state was mutated while CS update was published to other nodes") - ); - return; - } - - boolean sentToApplier = processNextCommittedClusterState( - "master " - + newState.nodes().getMasterNode() - + " committed version [" - + newState.version() - + "] source [" - + clusterChangedEvent.source() - + "]" - ); - if (sentToApplier == false && processedOrFailed.get() == false) { - assert false : "cluster state published locally neither processed nor failed: " + newState; - logger.warn( - "cluster state with version [{}] that is published locally has neither been processed nor failed", - newState.version() - ); - publishListener.onFailure( - new FailedToCommitClusterStateException( - "cluster state that is published locally has neither " + "been processed nor failed" - ) - ); - } - } - } - - /** - * Gets the current set of nodes involved in the node fault detection. - * NB: for testing purposes - */ - Set getFaultDetectionNodes() { - return nodesFD.getNodes(); - } - - @Override - public DiscoveryStats stats() { - return new DiscoveryStats(pendingStatesQueue.stats(), publishClusterState.stats()); - } - - public DiscoverySettings getDiscoverySettings() { - return discoverySettings; - } - - /** - * returns true if zen discovery is started and there is a currently a background thread active for (re)joining - * the cluster used for testing. - */ - public boolean joiningCluster() { - return joinThreadControl.joinThreadActive(); - } - - // used for testing - public ClusterState[] pendingClusterStates() { - return pendingStatesQueue.pendingClusterStates(); - } - - PendingClusterStatesQueue pendingClusterStatesQueue() { - return pendingStatesQueue; - } - - /** - * the main function of a join thread. This function is guaranteed to join the cluster - * or spawn a new join thread upon failure to do so. - */ - private void innerJoinCluster() { - DiscoveryNode masterNode = null; - final Thread currentThread = Thread.currentThread(); - nodeJoinController.startElectionContext(); - while (masterNode == null && joinThreadControl.joinThreadActive(currentThread)) { - masterNode = findMaster(); - } - - if (!joinThreadControl.joinThreadActive(currentThread)) { - logger.trace("thread is no longer in currentJoinThread. Stopping."); - return; - } - - if (transportService.getLocalNode().equals(masterNode)) { - final int requiredJoins = Math.max(0, electMaster.minimumMasterNodes() - 1); // we count as one - logger.debug("elected as master, waiting for incoming joins ([{}] needed)", requiredJoins); - nodeJoinController.waitToBeElectedAsMaster( - requiredJoins, - masterElectionWaitForJoinsTimeout, - new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - synchronized (stateMutex) { - joinThreadControl.markThreadAsDone(currentThread); - } - } - - @Override - public void onFailure(Throwable t) { - logger.trace("failed while waiting for nodes to join, rejoining", t); - synchronized (stateMutex) { - joinThreadControl.markThreadAsDoneAndStartNew(currentThread); - } - } - } - - ); - } else { - // process any incoming joins (they will fail because we are not the master) - nodeJoinController.stopElectionContext(masterNode + " elected"); - - // send join request - final boolean success = joinElectedMaster(masterNode); - - synchronized (stateMutex) { - if (success) { - DiscoveryNode currentMasterNode = this.clusterState().getNodes().getMasterNode(); - if (currentMasterNode == null) { - // Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have - // a valid master. - logger.debug("no master node is set, despite of join request completing. retrying pings."); - joinThreadControl.markThreadAsDoneAndStartNew(currentThread); - } else if (currentMasterNode.equals(masterNode) == false) { - // update cluster state - joinThreadControl.stopRunningThreadAndRejoin("master_switched_while_finalizing_join"); - } - - joinThreadControl.markThreadAsDone(currentThread); - } else { - // failed to join. Try again... - joinThreadControl.markThreadAsDoneAndStartNew(currentThread); - } - } - } - } - - /** - * Join a newly elected master. - * - * @return true if successful - */ - private boolean joinElectedMaster(DiscoveryNode masterNode) { - try { - // first, make sure we can connect to the master - transportService.connectToNode(masterNode); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to connect to master [{}], retrying...", masterNode), e); - return false; - } - int joinAttempt = 0; // we retry on illegal state if the master is not yet ready - while (true) { - try { - logger.trace("joining master {}", masterNode); - membership.sendJoinRequestBlocking(masterNode, transportService.getLocalNode(), joinTimeout); - return true; - } catch (Exception e) { - final Throwable unwrap = ExceptionsHelper.unwrapCause(e); - if (unwrap instanceof NotMasterException) { - if (++joinAttempt == this.joinRetryAttempts) { - logger.info( - "failed to send join request to master [{}], reason [{}], tried [{}] times", - masterNode, - ExceptionsHelper.detailedMessage(e), - joinAttempt - ); - return false; - } else { - logger.trace( - "master {} failed with [{}]. retrying... (attempts done: [{}])", - masterNode, - ExceptionsHelper.detailedMessage(e), - joinAttempt - ); - } - } else { - if (logger.isTraceEnabled()) { - logger.trace(() -> new ParameterizedMessage("failed to send join request to master [{}]", masterNode), e); - } else { - logger.info( - "failed to send join request to master [{}], reason [{}]", - masterNode, - ExceptionsHelper.detailedMessage(e) - ); - } - return false; - } - } - - try { - Thread.sleep(this.joinRetryDelay.millis()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - - private void submitRejoin(String source) { - synchronized (stateMutex) { - rejoin(source); - } - } - - // visible for testing - void setCommittedState(ClusterState clusterState) { - synchronized (stateMutex) { - committedState.set(clusterState); - } - } - - private void removeNode(final DiscoveryNode node, final String source, final String reason) { - masterService.submitStateUpdateTask( - source + "(" + node + "), reason(" + reason + ")", - new NodeRemovalClusterStateTaskExecutor.Task(node, reason), - ClusterStateTaskConfig.build(Priority.IMMEDIATE), - nodeRemovalExecutor, - nodeRemovalExecutor - ); - } - - private void handleLeaveRequest(final DiscoveryNode node) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a node failure - return; - } - if (localNodeMaster()) { - removeNode(node, "zen-disco-node-left", "left"); - } else if (node.equals(clusterState().nodes().getMasterNode())) { - handleMasterGone(node, null, "shut_down"); - } - } - - private void handleNodeFailure(final DiscoveryNode node, final String reason) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a node failure - return; - } - if (!localNodeMaster()) { - // nothing to do here... - return; - } - removeNode(node, "zen-disco-node-failed", reason); - } - - private void handleMinimumMasterNodesChanged(final int minimumMasterNodes) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a node failure - return; - } - final int prevMinimumMasterNode = ZenDiscovery.this.electMaster.minimumMasterNodes(); - ZenDiscovery.this.electMaster.minimumMasterNodes(minimumMasterNodes); - if (!localNodeMaster()) { - // We only set the new value. If the master doesn't see enough nodes it will revoke it's mastership. - return; - } - synchronized (stateMutex) { - // check if we have enough master nodes, if not, we need to move into joining the cluster again - if (!electMaster.hasEnoughMasterNodes(committedState.get().nodes())) { - rejoin( - "not enough master nodes on change of minimum_master_nodes from [" - + prevMinimumMasterNode - + "] to [" - + minimumMasterNodes - + "]" - ); - } - } - } - - private void handleMasterGone(final DiscoveryNode masterNode, final Throwable cause, final String reason) { - if (lifecycleState() != Lifecycle.State.STARTED) { - // not started, ignore a master failure - return; - } - if (localNodeMaster()) { - // we might get this on both a master telling us shutting down, and then the disconnect failure - return; - } - - logger.info(() -> new ParameterizedMessage("master_left [{}], reason [{}]", masterNode, reason), cause); - - synchronized (stateMutex) { - if (localNodeMaster() == false && masterNode.equals(committedState.get().nodes().getMasterNode())) { - // flush any pending cluster states from old master, so it will not be set as master again - pendingStatesQueue.failAllStatesAndClear(new OpenSearchException("master left [{}]", reason)); - rejoin("master left (reason = " + reason + ")"); - } - } - } - - // return true if state has been sent to applier - boolean processNextCommittedClusterState(String reason) { - assert Thread.holdsLock(stateMutex); - - final ClusterState newClusterState = pendingStatesQueue.getNextClusterStateToProcess(); - final ClusterState currentState = committedState.get(); - // all pending states have been processed - if (newClusterState == null) { - return false; - } - - assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master"; - assert !newClusterState.blocks() - .hasGlobalBlock(noMasterBlockService.getNoMasterBlock()) : "received a cluster state with a master block"; - - if (currentState.nodes().isLocalNodeElectedMaster() && newClusterState.nodes().isLocalNodeElectedMaster() == false) { - handleAnotherMaster( - currentState, - newClusterState.nodes().getMasterNode(), - newClusterState.version(), - "via a new cluster state" - ); - return false; - } - - try { - if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) { - String message = String.format( - Locale.ROOT, - "rejecting cluster state version [%d] uuid [%s] received from [%s]", - newClusterState.version(), - newClusterState.stateUUID(), - newClusterState.nodes().getMasterNodeId() - ); - throw new IllegalStateException(message); - } - } catch (Exception e) { - try { - pendingStatesQueue.markAsFailed(newClusterState, e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.error(() -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner); - } - return false; - } - - if (currentState.blocks().hasGlobalBlock(noMasterBlockService.getNoMasterBlock())) { - // its a fresh update from the master as we transition from a start of not having a master to having one - logger.debug("got first state from fresh master [{}]", newClusterState.nodes().getMasterNodeId()); - } - - if (currentState == newClusterState) { - return false; - } - - committedState.set(newClusterState); - - // update failure detection only after the state has been updated to prevent race condition with handleLeaveRequest - // and handleNodeFailure as those check the current state to determine whether the failure is to be handled by this node - if (newClusterState.nodes().isLocalNodeElectedMaster()) { - // update the set of nodes to ping - nodesFD.updateNodesAndPing(newClusterState); - } else { - // check to see that we monitor the correct master of the cluster - if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().getMasterNode())) { - masterFD.restart( - newClusterState.nodes().getMasterNode(), - "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]" - ); - } - } - - clusterApplier.onNewClusterState( - "apply cluster state (from master [" + reason + "])", - this::clusterState, - new ClusterApplyListener() { - @Override - public void onSuccess(String source) { - try { - pendingStatesQueue.markAsProcessed(newClusterState); - } catch (Exception e) { - onFailure(source, e); - } - } - - @Override - public void onFailure(String source, Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected failure applying [{}]", reason), e); - try { - // TODO: use cluster state uuid instead of full cluster state so that we don't keep reference to CS around - // for too long. - pendingStatesQueue.markAsFailed(newClusterState, e); - } catch (Exception inner) { - inner.addSuppressed(e); - logger.error(() -> new ParameterizedMessage("unexpected exception while failing [{}]", reason), inner); - } - } - } - ); - - return true; - } - - /** - * In the case we follow an elected master the new cluster state needs to have the same elected master and - * the new cluster state version needs to be equal or higher than our cluster state version. - * If the first condition fails we reject the cluster state and throw an error. - * If the second condition fails we ignore the cluster state. - */ - public static boolean shouldIgnoreOrRejectNewClusterState(Logger logger, ClusterState currentState, ClusterState newClusterState) { - validateStateIsFromCurrentMaster(logger, currentState.nodes(), newClusterState); - - // reject cluster states that are not new from the same master - if (currentState.supersedes(newClusterState) - || (newClusterState.nodes().getMasterNodeId().equals(currentState.nodes().getMasterNodeId()) - && currentState.version() == newClusterState.version())) { - // if the new state has a smaller version, and it has the same master node, then no need to process it - logger.debug( - "received a cluster state that is not newer than the current one, ignoring (received {}, current {})", - newClusterState.version(), - currentState.version() - ); - return true; - } - - // reject older cluster states if we are following a master - if (currentState.nodes().getMasterNodeId() != null && newClusterState.version() < currentState.version()) { - logger.debug( - "received a cluster state that has a lower version than the current one, ignoring (received {}, current {})", - newClusterState.version(), - currentState.version() - ); - return true; - } - return false; - } - - /** - * In the case we follow an elected master the new cluster state needs to have the same elected master - * This method checks for this and throws an exception if needed - */ - - public static void validateStateIsFromCurrentMaster(Logger logger, DiscoveryNodes currentNodes, ClusterState newClusterState) { - if (currentNodes.getMasterNodeId() == null) { - return; - } - if (!currentNodes.getMasterNodeId().equals(newClusterState.nodes().getMasterNodeId())) { - logger.warn( - "received a cluster state from a different master than the current one, rejecting (received {}, current {})", - newClusterState.nodes().getMasterNode(), - currentNodes.getMasterNode() - ); - throw new IllegalStateException( - "cluster state from a different master than the current one, rejecting (received " - + newClusterState.nodes().getMasterNode() - + ", current " - + currentNodes.getMasterNode() - + ")" - ); - } - } - - void handleJoinRequest(final DiscoveryNode node, final ClusterState state, final MembershipAction.JoinCallback callback) { - if (nodeJoinController == null) { - throw new IllegalStateException("discovery module is not yet started"); - } else { - // we do this in a couple of places including the cluster update thread. This one here is really just best effort - // to ensure we fail as fast as possible. - onJoinValidators.stream().forEach(a -> a.accept(node, state)); - if (state.getBlocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK) == false) { - JoinTaskExecutor.ensureMajorVersionBarrier(node.getVersion(), state.getNodes().getMinNodeVersion()); - } - // try and connect to the node, if it fails, we can raise an exception back to the client... - transportService.connectToNode(node); - - // validate the join request, will throw a failure if it fails, which will get back to the - // node calling the join request - try { - membership.sendValidateJoinRequestBlocking(node, state, joinTimeout); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to validate incoming join request from node [{}]", node), e); - callback.onFailure(new IllegalStateException("failure when sending a validation request to node", e)); - return; - } - nodeJoinController.handleJoinRequest(node, callback); - } - } - - private DiscoveryNode findMaster() { - logger.trace("starting to ping"); - List fullPingResponses = pingAndWait(pingTimeout).toList(); - if (fullPingResponses == null) { - logger.trace("No full ping responses"); - return null; - } - if (logger.isTraceEnabled()) { - StringBuilder sb = new StringBuilder(); - if (fullPingResponses.size() == 0) { - sb.append(" {none}"); - } else { - for (ZenPing.PingResponse pingResponse : fullPingResponses) { - sb.append("\n\t--> ").append(pingResponse); - } - } - logger.trace("full ping responses:{}", sb); - } - - final DiscoveryNode localNode = transportService.getLocalNode(); - - // add our selves - assert fullPingResponses.stream().map(ZenPing.PingResponse::node).filter(n -> n.equals(localNode)).findAny().isPresent() == false; - - fullPingResponses.add(new ZenPing.PingResponse(localNode, null, this.clusterState())); - - // filter responses - final List pingResponses = filterPingResponses(fullPingResponses, masterElectionIgnoreNonMasters, logger); - - List activeMasters = new ArrayList<>(); - for (ZenPing.PingResponse pingResponse : pingResponses) { - // We can't include the local node in pingMasters list, otherwise we may up electing ourselves without - // any check / verifications from other nodes in ZenDiscover#innerJoinCluster() - if (pingResponse.master() != null && !localNode.equals(pingResponse.master())) { - activeMasters.add(pingResponse.master()); - } - } - - // nodes discovered during pinging - List masterCandidates = new ArrayList<>(); - for (ZenPing.PingResponse pingResponse : pingResponses) { - if (pingResponse.node().isMasterNode()) { - masterCandidates.add(new ElectMasterService.MasterCandidate(pingResponse.node(), pingResponse.getClusterStateVersion())); - } - } - - if (activeMasters.isEmpty()) { - if (electMaster.hasEnoughCandidates(masterCandidates)) { - final ElectMasterService.MasterCandidate winner = electMaster.electMaster(masterCandidates); - logger.trace("candidate {} won election", winner); - return winner.getNode(); - } else { - // if we don't have enough master nodes, we bail, because there are not enough master to elect from - logger.warn( - "not enough master nodes discovered during pinging (found [{}], but needed [{}]), pinging again", - masterCandidates, - electMaster.minimumMasterNodes() - ); - return null; - } - } else { - assert !activeMasters.contains( - localNode - ) : "local node should never be elected as master when other nodes indicate an active master"; - // lets tie break between discovered nodes - return electMaster.tieBreakActiveMasters(activeMasters); - } - } - - static List filterPingResponses( - List fullPingResponses, - boolean masterElectionIgnoreNonMasters, - Logger logger - ) { - List pingResponses; - if (masterElectionIgnoreNonMasters) { - pingResponses = fullPingResponses.stream().filter(ping -> ping.node().isMasterNode()).collect(Collectors.toList()); - } else { - pingResponses = fullPingResponses; - } - - if (logger.isDebugEnabled()) { - StringBuilder sb = new StringBuilder(); - if (pingResponses.isEmpty()) { - sb.append(" {none}"); - } else { - for (ZenPing.PingResponse pingResponse : pingResponses) { - sb.append("\n\t--> ").append(pingResponse); - } - } - logger.debug("filtered ping responses: (ignore_non_masters [{}]){}", masterElectionIgnoreNonMasters, sb); - } - return pingResponses; - } - - protected void rejoin(String reason) { - assert Thread.holdsLock(stateMutex); - ClusterState clusterState = committedState.get(); - - logger.warn("{}, current nodes: {}", reason, clusterState.nodes()); - nodesFD.stop(); - masterFD.stop(reason); - - // TODO: do we want to force a new thread if we actively removed the master? this is to give a full pinging cycle - // before a decision is made. - joinThreadControl.startNewThreadIfNotRunning(); - - if (clusterState.nodes().getMasterNodeId() != null) { - // remove block if it already exists before adding new one - assert clusterState.blocks() - .hasGlobalBlockWithId( - noMasterBlockService.getNoMasterBlock().id() - ) == false : "NO_MASTER_BLOCK should only be added by ZenDiscovery"; - ClusterBlocks clusterBlocks = ClusterBlocks.builder() - .blocks(clusterState.blocks()) - .addGlobalBlock(noMasterBlockService.getNoMasterBlock()) - .build(); - - DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder(clusterState.nodes()).masterNodeId(null).build(); - clusterState = ClusterState.builder(clusterState).blocks(clusterBlocks).nodes(discoveryNodes).build(); - - committedState.set(clusterState); - clusterApplier.onNewClusterState(reason, this::clusterState, (source, e) -> {}); // don't wait for state to be applied - } - } - - private boolean localNodeMaster() { - return clusterState().nodes().isLocalNodeElectedMaster(); - } - - private void handleAnotherMaster( - ClusterState localClusterState, - final DiscoveryNode otherMaster, - long otherClusterStateVersion, - String reason - ) { - assert localClusterState.nodes().isLocalNodeElectedMaster() : "handleAnotherMaster called but current node is not a master"; - assert Thread.holdsLock(stateMutex); - - if (otherClusterStateVersion > localClusterState.version()) { - rejoin("zen-disco-discovered another master with a new cluster_state [" + otherMaster + "][" + reason + "]"); - } else { - // TODO: do this outside mutex - logger.warn( - "discovered [{}] which is also master but with an older cluster_state, telling [{}] to rejoin the cluster ([{}])", - otherMaster, - otherMaster, - reason - ); - try { - // make sure we're connected to this node (connect to node does nothing if we're already connected) - // since the network connections are asymmetric, it may be that we received a state but have disconnected from the node - // in the past (after a master failure, for example) - transportService.connectToNode(otherMaster); - transportService.sendRequest( - otherMaster, - DISCOVERY_REJOIN_ACTION_NAME, - new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), - new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - - @Override - public void handleException(TransportException exp) { - logger.warn(() -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp); - } - } - ); - } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e); - } - } - } - - private ZenPing.PingCollection pingAndWait(TimeValue timeout) { - final CompletableFuture response = new CompletableFuture<>(); - try { - zenPing.ping(response::complete, timeout); - } catch (Exception ex) { - // logged later - response.completeExceptionally(ex); - } - - try { - return response.get(); - } catch (InterruptedException e) { - logger.trace("pingAndWait interrupted"); - return new ZenPing.PingCollection(); - } catch (ExecutionException e) { - logger.warn("Ping execution failed", e); - return new ZenPing.PingCollection(); - } - } - - @Override - public void onIncomingClusterState(ClusterState incomingState) { - validateIncomingState(logger, incomingState, committedState.get()); - pendingStatesQueue.addPending(incomingState); - } - - @Override - public void onClusterStateCommitted(String stateUUID, ActionListener processedListener) { - final ClusterState state = pendingStatesQueue.markAsCommitted(stateUUID, new PendingClusterStatesQueue.StateProcessedListener() { - @Override - public void onNewClusterStateProcessed() { - processedListener.onResponse(null); - } - - @Override - public void onNewClusterStateFailed(Exception e) { - processedListener.onFailure(e); - } - }); - if (state != null) { - synchronized (stateMutex) { - processNextCommittedClusterState( - "master " + state.nodes().getMasterNode() + " committed version [" + state.version() + "]" - ); - } - } - } - - /** - * does simple sanity check of the incoming cluster state. Throws an exception on rejections. - */ - static void validateIncomingState(Logger logger, ClusterState incomingState, ClusterState lastState) { - final ClusterName incomingClusterName = incomingState.getClusterName(); - if (!incomingClusterName.equals(lastState.getClusterName())) { - logger.warn( - "received cluster state from [{}] which is also master but with a different cluster name [{}]", - incomingState.nodes().getMasterNode(), - incomingClusterName - ); - throw new IllegalStateException("received state from a node that is not part of the cluster"); - } - if (lastState.nodes().getLocalNode().equals(incomingState.nodes().getLocalNode()) == false) { - logger.warn( - "received a cluster state from [{}] and not part of the cluster, should not happen", - incomingState.nodes().getMasterNode() - ); - throw new IllegalStateException("received state with a local node that does not match the current local node"); - } - - if (shouldIgnoreOrRejectNewClusterState(logger, lastState, incomingState)) { - String message = String.format( - Locale.ROOT, - "rejecting cluster state version [%d] uuid [%s] received from [%s]", - incomingState.version(), - incomingState.stateUUID(), - incomingState.nodes().getMasterNodeId() - ); - logger.warn(message); - throw new IllegalStateException(message); - } - - } - - private class MembershipListener implements MembershipAction.MembershipListener { - @Override - public void onJoin(DiscoveryNode node, MembershipAction.JoinCallback callback) { - handleJoinRequest(node, ZenDiscovery.this.clusterState(), callback); - } - - @Override - public void onLeave(DiscoveryNode node) { - handleLeaveRequest(node); - } - } - - private class NodeFaultDetectionListener extends NodesFaultDetection.Listener { - - private final AtomicInteger pingsWhileMaster = new AtomicInteger(0); - - @Override - public void onNodeFailure(DiscoveryNode node, String reason) { - handleNodeFailure(node, reason); - } - - @Override - public void onPingReceived(final NodesFaultDetection.PingRequest pingRequest) { - // if we are master, we don't expect any fault detection from another node. If we get it - // means we potentially have two masters in the cluster. - if (!localNodeMaster()) { - pingsWhileMaster.set(0); - return; - } - - if (pingsWhileMaster.incrementAndGet() < maxPingsFromAnotherMaster) { - logger.trace( - "got a ping from another master {}. current ping count: [{}]", - pingRequest.masterNode(), - pingsWhileMaster.get() - ); - return; - } - logger.debug( - "got a ping from another master {}. resolving who should rejoin. current ping count: [{}]", - pingRequest.masterNode(), - pingsWhileMaster.get() - ); - synchronized (stateMutex) { - ClusterState currentState = committedState.get(); - if (currentState.nodes().isLocalNodeElectedMaster()) { - pingsWhileMaster.set(0); - handleAnotherMaster(currentState, pingRequest.masterNode(), pingRequest.clusterStateVersion(), "node fd ping"); - } - } - } - } - - private class MasterNodeFailureListener implements MasterFaultDetection.Listener { - - @Override - public void onMasterFailure(DiscoveryNode masterNode, Throwable cause, String reason) { - handleMasterGone(masterNode, cause, reason); - } - } - - public static class RejoinClusterRequest extends TransportRequest { - - private String fromNodeId; - - RejoinClusterRequest(String fromNodeId) { - this.fromNodeId = fromNodeId; - } - - public RejoinClusterRequest(StreamInput in) throws IOException { - super(in); - fromNodeId = in.readOptionalString(); - - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeOptionalString(fromNodeId); - } - } - - class RejoinClusterRequestHandler implements TransportRequestHandler { - @Override - public void messageReceived(final RejoinClusterRequest request, final TransportChannel channel, Task task) throws Exception { - try { - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } catch (Exception e) { - logger.warn("failed to send response on rejoin cluster request handling", e); - } - synchronized (stateMutex) { - rejoin("received a request to rejoin the cluster from [" + request.fromNodeId + "]"); - } - } - } - - /** - * All control of the join thread should happen under the cluster state update task thread. - * This is important to make sure that the background joining process is always in sync with any cluster state updates - * like master loss, failure to join, received cluster state while joining etc. - */ - private class JoinThreadControl { - - private final AtomicBoolean running = new AtomicBoolean(false); - private final AtomicReference currentJoinThread = new AtomicReference<>(); - - /** returns true if join thread control is started and there is currently an active join thread */ - public boolean joinThreadActive() { - Thread currentThread = currentJoinThread.get(); - return running.get() && currentThread != null && currentThread.isAlive(); - } - - /** returns true if join thread control is started and the supplied thread is the currently active joinThread */ - public boolean joinThreadActive(Thread joinThread) { - return running.get() && joinThread.equals(currentJoinThread.get()); - } - - /** cleans any running joining thread and calls {@link #rejoin} */ - public void stopRunningThreadAndRejoin(String reason) { - assert Thread.holdsLock(stateMutex); - currentJoinThread.set(null); - rejoin(reason); - } - - /** starts a new joining thread if there is no currently active one and join thread controlling is started */ - public void startNewThreadIfNotRunning() { - assert Thread.holdsLock(stateMutex); - if (joinThreadActive()) { - return; - } - threadPool.generic().execute(new Runnable() { - @Override - public void run() { - Thread currentThread = Thread.currentThread(); - if (!currentJoinThread.compareAndSet(null, currentThread)) { - return; - } - while (running.get() && joinThreadActive(currentThread)) { - try { - innerJoinCluster(); - return; - } catch (Exception e) { - logger.error("unexpected error while joining cluster, trying again", e); - // Because we catch any exception here, we want to know in - // tests if an uncaught exception got to this point and the test infra uncaught exception - // leak detection can catch this. In practise no uncaught exception should leak - assert ExceptionsHelper.reThrowIfNotNull(e); - } - } - // cleaning the current thread from currentJoinThread is done by explicit calls. - } - }); - } - - /** - * marks the given joinThread as completed and makes sure another thread is running (starting one if needed) - * If the given thread is not the currently running join thread, the command is ignored. - */ - public void markThreadAsDoneAndStartNew(Thread joinThread) { - assert Thread.holdsLock(stateMutex); - if (!markThreadAsDone(joinThread)) { - return; - } - startNewThreadIfNotRunning(); - } - - /** marks the given joinThread as completed. Returns false if the supplied thread is not the currently active join thread */ - public boolean markThreadAsDone(Thread joinThread) { - assert Thread.holdsLock(stateMutex); - return currentJoinThread.compareAndSet(joinThread, null); - } - - public void stop() { - running.set(false); - Thread joinThread = currentJoinThread.getAndSet(null); - if (joinThread != null) { - joinThread.interrupt(); - } - } - - public void start() { - running.set(true); - } - - } - - public final Collection> getOnJoinValidators() { - return onJoinValidators; - } - - static class ZenNodeRemovalClusterStateTaskExecutor extends NodeRemovalClusterStateTaskExecutor { - - private final ElectMasterService electMasterService; - private final Consumer rejoin; - - ZenNodeRemovalClusterStateTaskExecutor( - final AllocationService allocationService, - final ElectMasterService electMasterService, - final Consumer rejoin, - final Logger logger - ) { - super(allocationService, logger); - this.electMasterService = electMasterService; - this.rejoin = rejoin; - } - - @Override - protected ClusterTasksResult getTaskClusterTasksResult( - ClusterState currentState, - List tasks, - ClusterState remainingNodesClusterState - ) { - if (electMasterService.hasEnoughMasterNodes(remainingNodesClusterState.nodes()) == false) { - final ClusterTasksResult.Builder resultBuilder = ClusterTasksResult.builder().successes(tasks); - final int masterNodes = electMasterService.countMasterNodes(remainingNodesClusterState.nodes()); - rejoin.accept( - LoggerMessageFormat.format( - "not enough master nodes (has [{}], but needed [{}])", - masterNodes, - electMasterService.minimumMasterNodes() - ) - ); - return resultBuilder.build(currentState); - } else { - return super.getTaskClusterTasksResult(currentState, tasks, remainingNodesClusterState); - } - } - } -} diff --git a/server/src/main/java/org/opensearch/discovery/zen/ZenPing.java b/server/src/main/java/org/opensearch/discovery/zen/ZenPing.java deleted file mode 100644 index ef4f247105f..00000000000 --- a/server/src/main/java/org/opensearch/discovery/zen/ZenPing.java +++ /dev/null @@ -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 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 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 toList() { - return new ArrayList<>(pings.values()); - } - - /** the number of nodes for which there are known pings */ - public synchronized int size() { - return pings.size(); - } - } -} diff --git a/server/src/main/java/org/opensearch/gateway/Gateway.java b/server/src/main/java/org/opensearch/gateway/Gateway.java index 8fcc502c03e..c25097723b8 100644 --- a/server/src/main/java/org/opensearch/gateway/Gateway.java +++ b/server/src/main/java/org/opensearch/gateway/Gateway.java @@ -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()) { diff --git a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java index 4d03f429f54..ce4f749be0f 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java @@ -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 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(); diff --git a/server/src/main/java/org/opensearch/gateway/GatewayService.java b/server/src/main/java/org/opensearch/gateway/GatewayService.java index 2c23a4c73ec..47347cea50e 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayService.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayService.java @@ -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; } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index f8e84730bbd..ac0dd8be1a9 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -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 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, diff --git a/server/src/main/java/org/opensearch/transport/TransportSettings.java b/server/src/main/java/org/opensearch/transport/TransportSettings.java index f69d1967dae..cacdbce60c3 100644 --- a/server/src/main/java/org/opensearch/transport/TransportSettings.java +++ b/server/src/main/java/org/opensearch/transport/TransportSettings.java @@ -306,7 +306,7 @@ public final class TransportSettings { ); public static final Setting> 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 diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 85f798f02f8..380f8ce581e 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -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; diff --git a/server/src/test/java/org/opensearch/bootstrap/BootstrapChecksTests.java b/server/src/test/java/org/opensearch/bootstrap/BootstrapChecksTests.java index 464395b7b99..04cc1380615 100644 --- a/server/src/test/java/org/opensearch/bootstrap/BootstrapChecksTests.java +++ b/server/src/test/java/org/opensearch/bootstrap/BootstrapChecksTests.java @@ -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 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()) - ); } } diff --git a/server/src/test/java/org/opensearch/cluster/coordination/ClusterBootstrapServiceTests.java b/server/src/test/java/org/opensearch/cluster/coordination/ClusterBootstrapServiceTests.java index fe18170fc1a..a2fe39ef453 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/ClusterBootstrapServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/ClusterBootstrapServiceTests.java @@ -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() diff --git a/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java index 0d0ccc17426..875b0e2a979 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/CoordinatorTests.java @@ -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; diff --git a/server/src/test/java/org/opensearch/cluster/coordination/DiscoveryUpgradeServiceTests.java b/server/src/test/java/org/opensearch/cluster/coordination/DiscoveryUpgradeServiceTests.java deleted file mode 100644 index c53080eb25c..00000000000 --- a/server/src/test/java/org/opensearch/cluster/coordination/DiscoveryUpgradeServiceTests.java +++ /dev/null @@ -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())); - } -} diff --git a/server/src/test/java/org/opensearch/cluster/coordination/JoinHelperTests.java b/server/src/test/java/org/opensearch/cluster/coordination/JoinHelperTests.java index 4df333a41e5..b53cf9ddf1d 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/JoinHelperTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/JoinHelperTests.java @@ -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, diff --git a/server/src/test/java/org/opensearch/cluster/coordination/NoMasterBlockServiceTests.java b/server/src/test/java/org/opensearch/cluster/coordination/NoMasterBlockServiceTests.java index c345d46a9c8..c1ca775142a 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/NoMasterBlockServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/NoMasterBlockServiceTests.java @@ -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(); - } } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexUpgradeServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexUpgradeServiceTests.java index f4e12ab0364..248bf30f1e1 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexUpgradeServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexUpgradeServiceTests.java @@ -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", diff --git a/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java b/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java index b23f025c3e7..83e99475dc6 100644 --- a/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java +++ b/server/src/test/java/org/opensearch/cluster/serialization/ClusterSerializationTests.java @@ -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 { diff --git a/server/src/test/java/org/opensearch/discovery/ZenFaultDetectionTests.java b/server/src/test/java/org/opensearch/discovery/ZenFaultDetectionTests.java deleted file mode 100644 index 4f17f194748..00000000000 --- a/server/src/test/java/org/opensearch/discovery/ZenFaultDetectionTests.java +++ /dev/null @@ -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 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 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 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 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 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> inflightPings = Collections.newSetFromMap(new ConcurrentHashMap<>()); - private final Set> 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 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(); - } - } -} diff --git a/server/src/test/java/org/opensearch/discovery/zen/ElectMasterServiceTests.java b/server/src/test/java/org/opensearch/discovery/zen/ElectMasterServiceTests.java deleted file mode 100644 index 44d141e18f5..00000000000 --- a/server/src/test/java/org/opensearch/discovery/zen/ElectMasterServiceTests.java +++ /dev/null @@ -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 generateRandomNodes() { - int count = scaledRandomIntBetween(1, 100); - ArrayList nodes = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - Set 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 generateRandomCandidates() { - int count = scaledRandomIntBetween(1, 100); - ArrayList candidates = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - Set 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 nodes = generateRandomNodes(); - List 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 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 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 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 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 nodes = generateRandomNodes(); - ElectMasterService service = electMasterService(); - - int masterNodes = 0; - - for (DiscoveryNode node : nodes) { - if (node.isMasterNode()) { - masterNodes++; - } - } - - assertEquals(masterNodes, service.countMasterNodes(nodes)); - } -} diff --git a/server/src/test/java/org/opensearch/discovery/zen/NodeJoinControllerTests.java b/server/src/test/java/org/opensearch/discovery/zen/NodeJoinControllerTests.java deleted file mode 100644 index acd4a6acebb..00000000000 --- a/server/src/test/java/org/opensearch/discovery/zen/NodeJoinControllerTests.java +++ /dev/null @@ -1,1008 +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.message.ParameterizedMessage; -import org.opensearch.ExceptionsHelper; -import org.opensearch.Version; -import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.ClusterStateUpdateTask; -import org.opensearch.cluster.NotMasterException; -import org.opensearch.cluster.block.ClusterBlocks; -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.ShardRouting; -import org.opensearch.cluster.routing.ShardRoutingState; -import org.opensearch.cluster.routing.TestShardRouting; -import org.opensearch.cluster.routing.UnassignedInfo; -import org.opensearch.cluster.service.MasterService; -import org.opensearch.common.Priority; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.concurrent.AbstractRunnable; -import org.opensearch.common.util.concurrent.BaseFuture; -import org.opensearch.index.shard.ShardId; -import org.opensearch.test.ClusterServiceUtils; -import org.opensearch.test.OpenSearchTestCase; -import org.opensearch.threadpool.TestThreadPool; -import org.opensearch.threadpool.ThreadPool; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; -import static java.util.Collections.shuffle; -import static org.opensearch.cluster.OpenSearchAllocationTestCase.createAllocationService; -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.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; -import static org.opensearch.test.ClusterServiceUtils.setState; -import static org.opensearch.test.VersionUtils.allVersions; -import static org.opensearch.test.VersionUtils.allOpenSearchVersions; -import static org.opensearch.test.VersionUtils.getPreviousVersion; -import static org.opensearch.test.VersionUtils.randomCompatibleVersion; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; - -public class NodeJoinControllerTests extends OpenSearchTestCase { - - private static ThreadPool threadPool; - - private MasterService masterService; - private NodeJoinController nodeJoinController; - - @BeforeClass - public static void beforeClass() { - threadPool = new TestThreadPool("NodeJoinControllerTests"); - } - - @AfterClass - public static void afterClass() { - ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); - threadPool = null; - } - - @Before - public void setUp() throws Exception { - super.setUp(); - } - - @After - public void tearDown() throws Exception { - super.tearDown(); - masterService.close(); - } - - private static ClusterState initialState(boolean withMaster) { - DiscoveryNode localNode = new DiscoveryNode( - "node", - OpenSearchTestCase.buildNewFakeTransportAddress(), - Collections.emptyMap(), - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT - ); - ClusterState initialClusterState = ClusterState.builder(new ClusterName(ClusterServiceUtils.class.getSimpleName())) - .nodes( - DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(withMaster ? localNode.getId() : null) - ) - .blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK) - .build(); - return initialClusterState; - } - - private void setupMasterServiceAndNodeJoinController(ClusterState initialState) { - if (masterService != null || nodeJoinController != null) { - throw new IllegalStateException("method setupMasterServiceAndNodeJoinController can only be called once"); - } - masterService = ClusterServiceUtils.createMasterService(threadPool, initialState); - nodeJoinController = new NodeJoinController( - Settings.EMPTY, - masterService, - createAllocationService(Settings.EMPTY), - new ElectMasterService(Settings.EMPTY), - (s, p, r) -> {} - ); - } - - public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException { - setupMasterServiceAndNodeJoinController(initialState(true)); - List nodes = new ArrayList<>(); - nodes.add(discoveryState(masterService).nodes().getLocalNode()); - - int nodeId = 0; - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - joinNode(node); - } - nodeJoinController.startElectionContext(); - ArrayList> pendingJoins = new ArrayList<>(); - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - pendingJoins.add(joinNodeAsync(node)); - } - nodeJoinController.stopElectionContext("test"); - boolean hadSyncJoin = false; - for (int i = randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - nodes.add(node); - joinNode(node); - hadSyncJoin = true; - } - if (hadSyncJoin) { - for (Future joinFuture : pendingJoins) { - assertThat(joinFuture.isDone(), equalTo(true)); - } - } - for (Future joinFuture : pendingJoins) { - joinFuture.get(); - } - } - - public void testFailingJoinsWhenNotMaster() throws ExecutionException, InterruptedException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - - logger.debug("--> testing joins fail post accumulation"); - ArrayList> pendingJoins = new ArrayList<>(); - nodeJoinController.startElectionContext(); - for (int i = 1 + randomInt(5); i > 0; i--) { - DiscoveryNode node = newNode(nodeId++); - final Future future = joinNodeAsync(node); - pendingJoins.add(future); - assertThat(future.isDone(), equalTo(false)); - } - nodeJoinController.stopElectionContext("test"); - for (Future future : pendingJoins) { - try { - future.get(); - fail("failed to fail accumulated node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - } - - public void testSimpleMasterElectionWithoutRequiredJoins() throws InterruptedException, ExecutionException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - final int requiredJoins = 0; - logger.debug("--> using requiredJoins [{}]", requiredJoins); - // initial (failing) joins shouldn't count - for (int i = randomInt(5); i > 0; i--) { - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - - nodeJoinController.startElectionContext(); - final SimpleFuture electionFuture = new SimpleFuture("master election"); - final Thread masterElection = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error from waitToBeElectedAsMaster", e); - electionFuture.markAsFailed(e); - } - - @Override - protected void doRun() throws Exception { - nodeJoinController.waitToBeElectedAsMaster( - requiredJoins, - TimeValue.timeValueHours(30), - new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat( - "callback called with elected as master, but state disagrees", - state.nodes().isLocalNodeElectedMaster(), - equalTo(true) - ); - electionFuture.markAsDone(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("unexpected error while waiting to be elected as master", t); - electionFuture.markAsFailed(t); - } - } - ); - } - }); - masterElection.start(); - - logger.debug("--> requiredJoins is set to 0. verifying election finished"); - electionFuture.get(); - } - - public void testSimpleMasterElection() throws InterruptedException, ExecutionException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - final int requiredJoins = 1 + randomInt(5); - logger.debug("--> using requiredJoins [{}]", requiredJoins); - // initial (failing) joins shouldn't count - for (int i = randomInt(5); i > 0; i--) { - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - - nodeJoinController.startElectionContext(); - final SimpleFuture electionFuture = new SimpleFuture("master election"); - final Thread masterElection = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error from waitToBeElectedAsMaster", e); - electionFuture.markAsFailed(e); - } - - @Override - protected void doRun() throws Exception { - nodeJoinController.waitToBeElectedAsMaster( - requiredJoins, - TimeValue.timeValueHours(30), - new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat( - "callback called with elected as master, but state disagrees", - state.nodes().isLocalNodeElectedMaster(), - equalTo(true) - ); - electionFuture.markAsDone(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("unexpected error while waiting to be elected as master", t); - electionFuture.markAsFailed(t); - } - } - ); - } - }); - masterElection.start(); - assertThat("election finished immediately but required joins is [" + requiredJoins + "]", electionFuture.isDone(), equalTo(false)); - - final int initialJoins = randomIntBetween(0, requiredJoins - 1); - final ArrayList pendingJoins = new ArrayList<>(); - ArrayList nodesToJoin = new ArrayList<>(); - for (int i = 0; i < initialJoins; i++) { - DiscoveryNode node = newNode(nodeId++, true); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - // data nodes shouldn't count - for (int i = 0; i < requiredJoins; i++) { - DiscoveryNode node = newNode(nodeId++, false); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - // add - - shuffle(nodesToJoin, random()); - logger.debug("--> joining [{}] unique master nodes. Total of [{}] join requests", initialJoins, nodesToJoin.size()); - for (DiscoveryNode node : nodesToJoin) { - pendingJoins.add(joinNodeAsync(node)); - } - - logger.debug("--> asserting master election didn't finish yet"); - assertThat( - "election finished after [" + initialJoins + "] master nodes but required joins is [" + requiredJoins + "]", - electionFuture.isDone(), - equalTo(false) - ); - - final int finalJoins = requiredJoins - initialJoins + randomInt(5); - nodesToJoin.clear(); - for (int i = 0; i < finalJoins; i++) { - DiscoveryNode node = newNode(nodeId++, true); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - for (int i = 0; i < requiredJoins; i++) { - DiscoveryNode node = newNode(nodeId++, false); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - - shuffle(nodesToJoin, random()); - logger.debug("--> joining [{}] nodes, with repetition a total of [{}]", finalJoins, nodesToJoin.size()); - for (DiscoveryNode node : nodesToJoin) { - pendingJoins.add(joinNodeAsync(node)); - } - logger.debug("--> waiting for master election to with no exception"); - electionFuture.get(); - - logger.debug("--> waiting on all joins to be processed"); - for (SimpleFuture future : pendingJoins) { - logger.debug("waiting on {}", future); - future.get(); // throw any exception - } - - logger.debug("--> testing accumulation stopped"); - nodeJoinController.startElectionContext(); - nodeJoinController.stopElectionContext("test"); - - } - - public void testMasterElectionTimeout() throws InterruptedException { - setupMasterServiceAndNodeJoinController(initialState(false)); - int nodeId = 0; - final int requiredJoins = 1 + randomInt(5); - logger.debug("--> using requiredJoins [{}]", requiredJoins); - // initial (failing) joins shouldn't count - for (int i = randomInt(5); i > 0; i--) { - try { - joinNode(newNode(nodeId++)); - fail("failed to fail node join when not a master"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - - nodeJoinController.startElectionContext(); - final int initialJoins = randomIntBetween(0, requiredJoins - 1); - final ArrayList pendingJoins = new ArrayList<>(); - ArrayList nodesToJoin = new ArrayList<>(); - for (int i = 0; i < initialJoins; i++) { - DiscoveryNode node = newNode(nodeId++); - for (int j = 1 + randomInt(3); j > 0; j--) { - nodesToJoin.add(node); - } - } - shuffle(nodesToJoin, random()); - logger.debug("--> joining [{}] nodes, with repetition a total of [{}]", initialJoins, nodesToJoin.size()); - for (DiscoveryNode node : nodesToJoin) { - pendingJoins.add(joinNodeAsync(node)); - } - - final AtomicReference failure = new AtomicReference<>(); - final CountDownLatch latch = new CountDownLatch(1); - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueMillis(1), new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat( - "callback called with elected as master, but state disagrees", - state.nodes().isLocalNodeElectedMaster(), - equalTo(true) - ); - latch.countDown(); - } - - @Override - public void onFailure(Throwable t) { - failure.set(t); - latch.countDown(); - } - }); - latch.await(); - logger.debug("--> verifying election timed out"); - assertThat(failure.get(), instanceOf(NotMasterException.class)); - - logger.debug("--> verifying all joins are failed"); - for (SimpleFuture future : pendingJoins) { - logger.debug("waiting on {}", future); - try { - future.get(); // throw any exception - fail("failed to fail node join [" + future + "]"); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(NotMasterException.class)); - } - } - } - - public void testNewClusterStateOnExistingNodeJoin() throws InterruptedException, ExecutionException { - ClusterState state = initialState(true); - final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); - final DiscoveryNode other_node = new DiscoveryNode( - "other_node", - buildNewFakeTransportAddress(), - emptyMap(), - emptySet(), - Version.CURRENT - ); - nodesBuilder.add(other_node); - setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build()); - - state = discoveryState(masterService); - joinNode(other_node); - assertTrue("failed to publish a new state upon existing join", discoveryState(masterService) != state); - } - - public void testNormalConcurrentJoins() throws InterruptedException { - setupMasterServiceAndNodeJoinController(initialState(true)); - Thread[] threads = new Thread[3 + randomInt(5)]; - ArrayList nodes = new ArrayList<>(); - nodes.add(discoveryState(masterService).nodes().getLocalNode()); - final CyclicBarrier barrier = new CyclicBarrier(threads.length); - final List backgroundExceptions = new CopyOnWriteArrayList<>(); - for (int i = 0; i < threads.length; i++) { - final DiscoveryNode node = newNode(i); - final int iterations = rarely() ? randomIntBetween(1, 4) : 1; - nodes.add(node); - threads[i] = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error in join thread", e); - backgroundExceptions.add(e); - } - - @Override - protected void doRun() throws Exception { - barrier.await(); - for (int i = 0; i < iterations; i++) { - logger.debug("{} joining", node); - joinNode(node); - } - } - }, "t_" + i); - threads[i].start(); - } - - logger.info("--> waiting for joins to complete"); - for (Thread thread : threads) { - thread.join(); - } - - assertNodesInCurrentState(nodes); - } - - public void testElectionWithConcurrentJoins() throws InterruptedException, BrokenBarrierException { - setupMasterServiceAndNodeJoinController(initialState(false)); - - nodeJoinController.startElectionContext(); - - Thread[] threads = new Thread[3 + randomInt(5)]; - final int requiredJoins = randomInt(threads.length); - ArrayList nodes = new ArrayList<>(); - nodes.add(discoveryState(masterService).nodes().getLocalNode()); - final CyclicBarrier barrier = new CyclicBarrier(threads.length + 1); - final List backgroundExceptions = new CopyOnWriteArrayList<>(); - for (int i = 0; i < threads.length; i++) { - final DiscoveryNode node = newNode(i, true); - final int iterations = rarely() ? randomIntBetween(1, 4) : 1; - nodes.add(node); - threads[i] = new Thread(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logger.error("unexpected error in join thread", e); - backgroundExceptions.add(e); - } - - @Override - protected void doRun() throws Exception { - barrier.await(); - for (int i = 0; i < iterations; i++) { - logger.debug("{} joining", node); - joinNode(node); - } - } - }, "t_" + i); - threads[i].start(); - } - - barrier.await(); - logger.info("--> waiting to be elected as master (required joins [{}])", requiredJoins); - final AtomicReference failure = new AtomicReference<>(); - final CountDownLatch latch = new CountDownLatch(1); - nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - assertThat( - "callback called with elected as master, but state disagrees", - state.nodes().isLocalNodeElectedMaster(), - equalTo(true) - ); - latch.countDown(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("unexpected error while waiting to be elected as master", t); - failure.set(t); - latch.countDown(); - } - }); - latch.await(); - ExceptionsHelper.reThrowIfNotNull(failure.get()); - - logger.info("--> waiting for joins to complete"); - for (Thread thread : threads) { - thread.join(); - } - - assertNodesInCurrentState(nodes); - } - - public void testRejectingJoinWithSameAddressButDifferentId() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState state = discoveryState(masterService); - final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); - final DiscoveryNode other_node = new DiscoveryNode("other_node", existing.getAddress(), emptyMap(), emptySet(), Version.CURRENT); - - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); - } - - public void testRejectingJoinWithSameIdButDifferentNode() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState state = discoveryState(masterService); - final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); - final DiscoveryNode other_node = new DiscoveryNode( - randomBoolean() ? existing.getName() : "other_name", - existing.getId(), - randomBoolean() ? existing.getAddress() : buildNewFakeTransportAddress(), - randomBoolean() ? existing.getAttributes() : Collections.singletonMap("attr", "other"), - randomBoolean() ? existing.getRoles() : new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)), - existing.getVersion() - ); - - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); - } - - public void testRejectingRestartedNodeJoinsBeforeProcessingNodeLeft() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState state = discoveryState(masterService); - final DiscoveryNode existing = randomFrom(StreamSupport.stream(state.nodes().spliterator(), false).collect(Collectors.toList())); - joinNode(existing); // OK - - final DiscoveryNode other_node = new DiscoveryNode( - existing.getId(), - existing.getAddress(), - existing.getAttributes(), - existing.getRoles(), - Version.CURRENT - ); - - ExecutionException e = expectThrows(ExecutionException.class, () -> joinNode(other_node)); - assertThat(e.getMessage(), containsString("found existing node")); - } - - public void testRejectingJoinWithBeforeMinCompatibleVersion() throws ExecutionException, InterruptedException { - final Version badVersion = getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion()); - assertRejectingJoinWithIncompatibleVersion(badVersion); - } - - public void testRejectingJoinWithPreviousMajorVersion() throws ExecutionException, InterruptedException { - final Version badVersion = randomFrom( - allVersions().stream().filter(v -> v.compareMajor(Version.CURRENT) < 0).collect(Collectors.toList()) - ); - assertRejectingJoinWithIncompatibleVersion(badVersion); - } - - private void assertRejectingJoinWithIncompatibleVersion(final Version badVersion) throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - final DiscoveryNode badNode = new DiscoveryNode( - "badNode", - buildNewFakeTransportAddress(), - emptyMap(), - new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)), - badVersion - ); - - final Version goodVersion = randomFrom( - allOpenSearchVersions().stream().filter(v -> v.compareMajor(Version.CURRENT) >= 0).collect(Collectors.toList()) - ); - final DiscoveryNode goodNode = new DiscoveryNode( - "goodNode", - buildNewFakeTransportAddress(), - emptyMap(), - new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)), - goodVersion - ); - - CountDownLatch latch = new CountDownLatch(1); - // block cluster state - masterService.submitStateUpdateTask("test", new ClusterStateUpdateTask(Priority.IMMEDIATE) { - - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - latch.await(); - return currentState; - } - - @Override - public void onFailure(String source, Exception e) { - throw new AssertionError(e); - } - }); - - final SimpleFuture badJoin; - final SimpleFuture goodJoin; - if (randomBoolean()) { - badJoin = joinNodeAsync(badNode); - goodJoin = joinNodeAsync(goodNode); - } else { - goodJoin = joinNodeAsync(goodNode); - badJoin = joinNodeAsync(badNode); - } - assert goodJoin.isDone() == false; - assert badJoin.isDone() == false; - latch.countDown(); - goodJoin.get(); - ExecutionException e = expectThrows(ExecutionException.class, badJoin::get); - assertThat(e.getCause(), instanceOf(IllegalStateException.class)); - assertThat(e.getCause().getMessage(), allOf(containsString("node version"), containsString("not supported"))); - } - - public void testRejectingJoinWithIncompatibleVersionWithUnrecoveredState() throws InterruptedException, ExecutionException { - addNodes(randomInt(5)); - ClusterState.Builder builder = ClusterState.builder(discoveryState(masterService)); - builder.blocks(ClusterBlocks.builder().addGlobalBlock(STATE_NOT_RECOVERED_BLOCK)); - setState(masterService, builder.build()); - final Version badVersion = getPreviousVersion(Version.CURRENT.minimumCompatibilityVersion()); - final DiscoveryNode badNode = new DiscoveryNode( - "badNode", - buildNewFakeTransportAddress(), - emptyMap(), - new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)), - badVersion - ); - - final Version goodVersion = randomFrom(randomCompatibleVersion(random(), Version.CURRENT)); - final DiscoveryNode goodNode = new DiscoveryNode( - "goodNode", - buildNewFakeTransportAddress(), - emptyMap(), - new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)), - goodVersion - ); - - CountDownLatch latch = new CountDownLatch(1); - // block cluster state - masterService.submitStateUpdateTask("test", new ClusterStateUpdateTask(Priority.IMMEDIATE) { - @Override - public ClusterState execute(ClusterState currentState) throws Exception { - latch.await(); - return currentState; - } - - @Override - public void onFailure(String source, Exception e) { - throw new AssertionError(e); - } - }); - - final SimpleFuture badJoin; - final SimpleFuture goodJoin; - if (randomBoolean()) { - badJoin = joinNodeAsync(badNode); - goodJoin = joinNodeAsync(goodNode); - } else { - goodJoin = joinNodeAsync(goodNode); - badJoin = joinNodeAsync(badNode); - } - assert goodJoin.isDone() == false; - assert badJoin.isDone() == false; - latch.countDown(); - goodJoin.get(); - ExecutionException e = expectThrows(ExecutionException.class, badJoin::get); - assertThat(e.getCause(), instanceOf(IllegalStateException.class)); - assertThat(e.getCause().getMessage(), allOf(containsString("node version"), containsString("not supported"))); - } - - /** - * Tests tha node can become a master, even though the last cluster state it knows contains - * nodes that conflict with the joins it got and needs to become a master - */ - public void testElectionBasedOnConflictingNodes() throws InterruptedException, ExecutionException { - ClusterState initialState = initialState(true); - final DiscoveryNode masterNode = initialState.nodes().getLocalNode(); - final DiscoveryNode otherNode = new DiscoveryNode( - "other_node", - buildNewFakeTransportAddress(), - emptyMap(), - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT - ); - // simulate master going down with stale nodes in it's cluster state (for example when min master nodes is set to 2) - // also add some shards to that node - DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(initialState.nodes()); - discoBuilder.masterNodeId(null); - discoBuilder.add(otherNode); - ClusterState.Builder stateBuilder = ClusterState.builder(initialState).nodes(discoBuilder); - if (randomBoolean()) { - IndexMetadata indexMetadata = IndexMetadata.builder("test") - .settings( - Settings.builder() - .put(SETTING_VERSION_CREATED, Version.CURRENT) - .put(SETTING_NUMBER_OF_SHARDS, 1) - .put(SETTING_NUMBER_OF_REPLICAS, 1) - .put(SETTING_CREATION_DATE, System.currentTimeMillis()) - ) - .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 = randomBoolean() ? masterNode : otherNode; - final DiscoveryNode replicaNode = primaryNode.equals(masterNode) ? otherNode : masterNode; - final boolean primaryStarted = randomBoolean(); - indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting( - "test", - 0, - primaryNode.getId(), - null, - true, - primaryStarted ? ShardRoutingState.STARTED : ShardRoutingState.INITIALIZING, - primaryStarted ? null : new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "getting there") - ) - ); - if (primaryStarted) { - boolean replicaStared = randomBoolean(); - indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting( - "test", - 0, - replicaNode.getId(), - null, - false, - replicaStared ? ShardRoutingState.STARTED : ShardRoutingState.INITIALIZING, - replicaStared ? null : new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "getting there") - ) - ); - } else { - indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting( - "test", - 0, - null, - null, - false, - ShardRoutingState.UNASSIGNED, - new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "life sucks") - ) - ); - } - 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()); - } - - setupMasterServiceAndNodeJoinController(stateBuilder.build()); - - // conflict on node id or address - final DiscoveryNode conflictingNode = randomBoolean() - ? new DiscoveryNode( - otherNode.getId(), - randomBoolean() ? otherNode.getAddress() : buildNewFakeTransportAddress(), - otherNode.getAttributes(), - otherNode.getRoles(), - Version.CURRENT - ) - : new DiscoveryNode( - "conflicting_address_node", - otherNode.getAddress(), - otherNode.getAttributes(), - otherNode.getRoles(), - Version.CURRENT - ); - - nodeJoinController.startElectionContext(); - final SimpleFuture joinFuture = joinNodeAsync(conflictingNode); - final CountDownLatch elected = new CountDownLatch(1); - nodeJoinController.waitToBeElectedAsMaster(1, TimeValue.timeValueHours(5), new NodeJoinController.ElectionCallback() { - @Override - public void onElectedAsMaster(ClusterState state) { - elected.countDown(); - } - - @Override - public void onFailure(Throwable t) { - logger.error("failed to be elected as master", t); - throw new AssertionError("failed to be elected as master", t); - } - }); - - elected.await(); - - joinFuture.get(); // throw any exception - - final ClusterState finalState = discoveryState(masterService); - final DiscoveryNodes finalNodes = finalState.nodes(); - assertTrue(finalNodes.isLocalNodeElectedMaster()); - assertThat(finalNodes.getLocalNode(), equalTo(masterNode)); - assertThat(finalNodes.getSize(), equalTo(2)); - assertThat(finalNodes.get(conflictingNode.getId()), equalTo(conflictingNode)); - List activeShardsOnRestartedNode = StreamSupport.stream( - finalState.getRoutingNodes().node(conflictingNode.getId()).spliterator(), - false - ).filter(ShardRouting::active).collect(Collectors.toList()); - assertThat(activeShardsOnRestartedNode, empty()); - } - - private void addNodes(int count) { - ClusterState state = initialState(true); - final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(state.nodes()); - for (int i = 0; i < count; i++) { - final DiscoveryNode node = new DiscoveryNode( - "node_" + state.nodes().getSize() + i, - buildNewFakeTransportAddress(), - emptyMap(), - new HashSet<>(randomSubsetOf(DiscoveryNodeRole.BUILT_IN_ROLES)), - Version.CURRENT - ); - nodesBuilder.add(node); - } - setupMasterServiceAndNodeJoinController(ClusterState.builder(state).nodes(nodesBuilder).build()); - } - - protected void assertNodesInCurrentState(List expectedNodes) { - final ClusterState state = discoveryState(masterService); - logger.info("assert for [{}] in:\n{}", expectedNodes, state); - DiscoveryNodes discoveryNodes = state.nodes(); - for (DiscoveryNode node : expectedNodes) { - assertThat("missing " + node + "\n" + discoveryNodes, discoveryNodes.get(node.getId()), equalTo(node)); - } - assertThat(discoveryNodes.getSize(), equalTo(expectedNodes.size())); - } - - static class SimpleFuture extends BaseFuture { - final String description; - - SimpleFuture(String description) { - this.description = description; - } - - public void markAsDone() { - set(null); - } - - public void markAsFailed(Throwable t) { - setException(t); - } - - @Override - public String toString() { - return "future [" + description + "]"; - } - } - - static final AtomicInteger joinId = new AtomicInteger(); - - private SimpleFuture joinNodeAsync(final DiscoveryNode node) throws InterruptedException { - final SimpleFuture future = new SimpleFuture("join of " + node + " (id [" + joinId.incrementAndGet() + "]"); - logger.debug("starting {}", future); - // clone the node before submitting to simulate an incoming join, which is guaranteed to have a new - // disco node object serialized off the network - nodeJoinController.handleJoinRequest(cloneNode(node), new MembershipAction.JoinCallback() { - @Override - public void onSuccess() { - logger.debug("{} completed", future); - future.markAsDone(); - } - - @Override - public void onFailure(Exception e) { - logger.error(() -> new ParameterizedMessage("unexpected error for {}", future), e); - future.markAsFailed(e); - } - }); - return future; - } - - /** - * creates an object clone of node, so it will be a different object instance - */ - private DiscoveryNode cloneNode(DiscoveryNode node) { - return new DiscoveryNode( - node.getName(), - node.getId(), - node.getEphemeralId(), - node.getHostName(), - node.getHostAddress(), - node.getAddress(), - node.getAttributes(), - node.getRoles(), - node.getVersion() - ); - } - - private void joinNode(final DiscoveryNode node) throws InterruptedException, ExecutionException { - joinNodeAsync(node).get(); - } - - protected DiscoveryNode newNode(int i) { - return newNode(i, randomBoolean()); - } - - protected DiscoveryNode newNode(int i, boolean master) { - final Set roles; - if (master) { - roles = Collections.singleton(DiscoveryNodeRole.MASTER_ROLE); - } else { - roles = Collections.emptySet(); - } - final String prefix = master ? "master_" : "data_"; - return new DiscoveryNode(prefix + i, i + "", buildNewFakeTransportAddress(), emptyMap(), roles, Version.CURRENT); - } -} diff --git a/server/src/test/java/org/opensearch/discovery/zen/PendingClusterStatesQueueTests.java b/server/src/test/java/org/opensearch/discovery/zen/PendingClusterStatesQueueTests.java deleted file mode 100644 index 1b93724f27d..00000000000 --- a/server/src/test/java/org/opensearch/discovery/zen/PendingClusterStatesQueueTests.java +++ /dev/null @@ -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 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 stateToDrop = states.subList(0, numberOfStateToDrop); - final int queueSize = states.size() - numberOfStateToDrop; - PendingClusterStatesQueue queue = createQueueWithStates(stateToDrop, queueSize); - List 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 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 states = randomStates(scaledRandomIntBetween(10, 300), "master1", "master2", "master3", "master4"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - List 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 states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - List committedContexts = randomCommitStates(queue); - ClusterState toFail = randomFrom(committedContexts).state; - queue.markAsFailed(toFail, new OpenSearchException("boo!")); - final Map 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 states = randomStates(scaledRandomIntBetween(10, 50), "master1", "master2", "master3", "master4"); - PendingClusterStatesQueue queue = createQueueWithStates(states); - List 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 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 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 randomCommitStates(PendingClusterStatesQueue queue) { - List 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 states) { - return createQueueWithStates(states, states.size() * 2); // we don't care about limits (there are dedicated tests for that) - } - - PendingClusterStatesQueue createQueueWithStates(List states, int maxQueueSize) { - PendingClusterStatesQueue queue; - queue = new PendingClusterStatesQueue(logger, maxQueueSize); - for (ClusterState state : states) { - queue.addPending(state); - } - return queue; - } - - List randomStates(int count, String... masters) { - ArrayList 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; - } - } - -} diff --git a/server/src/test/java/org/opensearch/discovery/zen/PublishClusterStateActionTests.java b/server/src/test/java/org/opensearch/discovery/zen/PublishClusterStateActionTests.java deleted file mode 100644 index 9da567752ab..00000000000 --- a/server/src/test/java/org/opensearch/discovery/zen/PublishClusterStateActionTests.java +++ /dev/null @@ -1,1026 +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.Version; -import org.opensearch.action.ActionListener; -import org.opensearch.cluster.ClusterChangedEvent; -import org.opensearch.cluster.ClusterModule; -import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.ClusterStateListener; -import org.opensearch.cluster.Diff; -import org.opensearch.cluster.block.ClusterBlocks; -import org.opensearch.cluster.coordination.FailedToCommitClusterStateException; -import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Nullable; -import org.opensearch.common.Randomness; -import org.opensearch.common.collect.ImmutableOpenMap; -import org.opensearch.common.collect.Tuple; -import org.opensearch.common.io.stream.NamedWriteableRegistry; -import org.opensearch.common.io.stream.StreamOutput; -import org.opensearch.common.settings.ClusterSettings; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.discovery.Discovery; -import org.opensearch.discovery.DiscoverySettings; -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.BytesTransportRequest; -import org.opensearch.transport.Transport; -import org.opensearch.transport.TransportChannel; -import org.opensearch.transport.TransportConnectionListener; -import org.opensearch.transport.TransportResponse; -import org.opensearch.transport.TransportService; -import org.opensearch.transport.TransportSettings; -import org.junit.After; -import org.junit.Before; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -import static org.opensearch.test.NodeRoles.nonMasterNode; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.emptyIterable; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; - -public class PublishClusterStateActionTests extends OpenSearchTestCase { - - private static final ClusterName CLUSTER_NAME = ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY); - - protected ThreadPool threadPool; - protected Map nodes = new HashMap<>(); - - public static class MockNode implements PublishClusterStateAction.IncomingClusterStateListener { - public final DiscoveryNode discoveryNode; - public final MockTransportService service; - public MockPublishAction action; - public final ClusterStateListener listener; - private final PendingClusterStatesQueue pendingStatesQueue; - - public volatile ClusterState clusterState; - - private final Logger logger; - - public MockNode(DiscoveryNode discoveryNode, MockTransportService service, @Nullable ClusterStateListener listener, Logger logger) { - this.discoveryNode = discoveryNode; - this.service = service; - this.listener = listener; - this.logger = logger; - this.clusterState = ClusterState.builder(CLUSTER_NAME) - .nodes(DiscoveryNodes.builder().add(discoveryNode).localNodeId(discoveryNode.getId()).build()) - .build(); - this.pendingStatesQueue = new PendingClusterStatesQueue(logger, 25); - } - - public MockNode setAsMaster() { - this.clusterState = ClusterState.builder(clusterState) - .nodes(DiscoveryNodes.builder(clusterState.nodes()).masterNodeId(discoveryNode.getId())) - .build(); - return this; - } - - public MockNode resetMasterId() { - this.clusterState = ClusterState.builder(clusterState) - .nodes(DiscoveryNodes.builder(clusterState.nodes()).masterNodeId(null)) - .build(); - return this; - } - - public void connectTo(DiscoveryNode node) { - service.connectToNode(node); - } - - @Override - public void onIncomingClusterState(ClusterState incomingState) { - ZenDiscovery.validateIncomingState(logger, incomingState, clusterState); - pendingStatesQueue.addPending(incomingState); - } - - public void onClusterStateCommitted(String stateUUID, ActionListener processedListener) { - final ClusterState state = pendingStatesQueue.markAsCommitted( - stateUUID, - new PendingClusterStatesQueue.StateProcessedListener() { - @Override - public void onNewClusterStateProcessed() { - processedListener.onResponse(null); - } - - @Override - public void onNewClusterStateFailed(Exception e) { - processedListener.onFailure(e); - } - } - ); - if (state != null) { - ClusterState newClusterState = pendingStatesQueue.getNextClusterStateToProcess(); - logger.debug( - "[{}] received version [{}], uuid [{}]", - discoveryNode.getName(), - newClusterState.version(), - newClusterState.stateUUID() - ); - if (listener != null) { - ClusterChangedEvent event = new ClusterChangedEvent("", newClusterState, clusterState); - listener.clusterChanged(event); - } - if (clusterState.nodes().getMasterNode() == null || newClusterState.supersedes(clusterState)) { - clusterState = newClusterState; - } - pendingStatesQueue.markAsProcessed(newClusterState); - } - } - - public DiscoveryNodes nodes() { - return clusterState.nodes(); - } - - } - - public MockNode createMockNode(final String name) throws Exception { - return createMockNode(name, Settings.EMPTY, null); - } - - public MockNode createMockNode(String name, final Settings basSettings, @Nullable ClusterStateListener listener) throws Exception { - return createMockNode(name, basSettings, listener, threadPool, logger, nodes); - } - - public static MockNode createMockNode( - String name, - final Settings basSettings, - @Nullable ClusterStateListener listener, - ThreadPool threadPool, - Logger logger, - Map nodes - ) throws Exception { - final Settings settings = Settings.builder() - .put("name", name) - .put(TransportSettings.TRACE_LOG_INCLUDE_SETTING.getKey(), "") - .put(TransportSettings.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING") - .put(basSettings) - .build(); - - MockTransportService service = buildTransportService(settings, threadPool); - DiscoveryNode discoveryNode = service.getLocalDiscoNode(); - MockNode node = new MockNode(discoveryNode, service, listener, logger); - node.action = buildPublishClusterStateAction(settings, service, node); - final CountDownLatch latch = new CountDownLatch(nodes.size() * 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); - } - }; - node.service.addConnectionListener(waitForConnection); - for (MockNode curNode : nodes.values()) { - curNode.service.addConnectionListener(waitForConnection); - curNode.connectTo(node.discoveryNode); - node.connectTo(curNode.discoveryNode); - } - assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true)); - for (MockNode curNode : nodes.values()) { - curNode.service.removeConnectionListener(waitForConnection); - } - node.service.removeConnectionListener(waitForConnection); - if (nodes.put(name, node) != null) { - fail("Node with the name " + name + " already exist"); - } - return node; - } - - public MockTransportService service(String name) { - MockNode node = nodes.get(name); - if (node != null) { - return node.service; - } - return null; - } - - public PublishClusterStateAction action(String name) { - MockNode node = nodes.get(name); - if (node != null) { - return node.action; - } - return null; - } - - @Override - @Before - public void setUp() throws Exception { - super.setUp(); - threadPool = new TestThreadPool(getClass().getName()); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - for (MockNode curNode : nodes.values()) { - curNode.service.close(); - } - terminate(threadPool); - } - - private static MockTransportService buildTransportService(Settings settings, ThreadPool threadPool) { - MockTransportService transportService = MockTransportService.createNewService(settings, Version.CURRENT, threadPool, null); - transportService.start(); - transportService.acceptIncomingRequests(); - return transportService; - } - - private static MockPublishAction buildPublishClusterStateAction( - Settings settings, - MockTransportService transportService, - PublishClusterStateAction.IncomingClusterStateListener listener - ) { - DiscoverySettings discoverySettings = new DiscoverySettings( - settings, - new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(ClusterModule.getNamedWriteables()); - return new MockPublishAction(transportService, namedWriteableRegistry, listener, discoverySettings); - } - - public void testSimpleClusterStatePublishing() throws Exception { - MockNode nodeA = createMockNode("nodeA").setAsMaster(); - MockNode nodeB = createMockNode("nodeB"); - - // Initial cluster state - ClusterState clusterState = nodeA.clusterState; - - // cluster state update - add nodeB - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(clusterState.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState) - .blocks(ClusterBlocks.builder().addGlobalBlock(Metadata.CLUSTER_READ_ONLY_BLOCK)) - .incrementVersion() - .build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - assertThat(nodeB.clusterState.blocks().global().size(), equalTo(1)); - - // cluster state update - remove block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - assertTrue(nodeB.clusterState.wasReadFromDiff()); - - // Adding new node - this node should get full cluster state while nodeB should still be getting diffs - - MockNode nodeC = createMockNode("nodeC"); - - // cluster state update 3 - register node C - previousClusterState = clusterState; - discoveryNodes = DiscoveryNodes.builder(discoveryNodes).add(nodeC.discoveryNode).build(); - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - // First state - assertSameStateFromFull(nodeC.clusterState, clusterState); - - // cluster state update 4 - update settings - previousClusterState = clusterState; - Metadata metadata = Metadata.builder(clusterState.metadata()) - .transientSettings(Settings.builder().put("foo", "bar").build()) - .build(); - clusterState = ClusterState.builder(clusterState).metadata(metadata).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - assertThat(nodeB.clusterState.blocks().global().size(), equalTo(0)); - assertSameStateFromDiff(nodeC.clusterState, clusterState); - assertThat(nodeC.clusterState.blocks().global().size(), equalTo(0)); - - // cluster state update - skipping one version change - should request full cluster state - previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); - clusterState = ClusterState.builder(clusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - assertSameStateFromFull(nodeC.clusterState, clusterState); - assertFalse(nodeC.clusterState.wasReadFromDiff()); - - // node A steps down from being master - nodeA.resetMasterId(); - nodeB.resetMasterId(); - nodeC.resetMasterId(); - - // node B becomes the master and sends a version of the cluster state that goes back - discoveryNodes = DiscoveryNodes.builder(discoveryNodes) - .add(nodeA.discoveryNode) - .add(nodeB.discoveryNode) - .add(nodeC.discoveryNode) - .masterNodeId(nodeB.discoveryNode.getId()) - .localNodeId(nodeB.discoveryNode.getId()) - .build(); - previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeB.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeA.clusterState, clusterState); - assertSameStateFromFull(nodeC.clusterState, clusterState); - } - - public void testUnexpectedDiffPublishing() throws Exception { - MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, event -> { fail("Shouldn't send cluster state to myself"); }) - .setAsMaster(); - - MockNode nodeB = createMockNode("nodeB"); - - // Initial cluster state with both states - the second node still shouldn't - // get diff even though it's present in the previous cluster state - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(nodeA.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = ClusterState.builder(CLUSTER_NAME).nodes(discoveryNodes).build(); - ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState) - .blocks(ClusterBlocks.builder().addGlobalBlock(Metadata.CLUSTER_READ_ONLY_BLOCK)) - .incrementVersion() - .build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromDiff(nodeB.clusterState, clusterState); - } - - public void testDisablingDiffPublishing() throws Exception { - Settings noDiffPublishingSettings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), false).build(); - - MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - fail("Shouldn't send cluster state to myself"); - } - }); - - MockNode nodeB = createMockNode("nodeB", noDiffPublishingSettings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - assertFalse(event.state().wasReadFromDiff()); - } - }); - - // Initial cluster state - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() - .add(nodeA.discoveryNode) - .localNodeId(nodeA.discoveryNode.getId()) - .masterNodeId(nodeA.discoveryNode.getId()) - .build(); - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).nodes(discoveryNodes).build(); - - // cluster state update - add nodeB - discoveryNodes = DiscoveryNodes.builder(discoveryNodes).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState) - .blocks(ClusterBlocks.builder().addGlobalBlock(Metadata.CLUSTER_READ_ONLY_BLOCK)) - .incrementVersion() - .build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - assertWarnings( - "[discovery.zen.publish_diff.enable] setting was deprecated in OpenSearch and will be removed in a future release! " - + "See the breaking changes documentation for the next major version." - ); - } - - /** - * Test not waiting on publishing works correctly (i.e., publishing times out) - */ - public void testSimultaneousClusterStatePublishing() throws Exception { - int numberOfNodes = randomIntBetween(2, 10); - int numberOfIterations = scaledRandomIntBetween(5, 50); - Settings settings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), randomBoolean()).build(); - MockNode master = createMockNode("node0", settings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - assertProperMetadataForVersion(event.state().metadata(), event.state().version()); - } - }).setAsMaster(); - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(master.nodes()); - for (int i = 1; i < numberOfNodes; i++) { - final String name = "node" + i; - final MockNode node = createMockNode(name, settings, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - assertProperMetadataForVersion(event.state().metadata(), event.state().version()); - } - }); - discoveryNodesBuilder.add(node.discoveryNode); - } - - AssertingAckListener[] listeners = new AssertingAckListener[numberOfIterations]; - DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); - Metadata metadata = Metadata.EMPTY_METADATA; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).metadata(metadata).build(); - ClusterState previousState; - for (int i = 0; i < numberOfIterations; i++) { - previousState = clusterState; - metadata = buildMetadataForVersion(metadata, i + 1); - clusterState = ClusterState.builder(clusterState).incrementVersion().metadata(metadata).nodes(discoveryNodes).build(); - listeners[i] = publishState(master.action, clusterState, previousState); - } - - for (int i = 0; i < numberOfIterations; i++) { - listeners[i].await(1, TimeUnit.SECONDS); - } - - // set the master cs - master.clusterState = clusterState; - - for (MockNode node : nodes.values()) { - assertSameState(node.clusterState, clusterState); - assertThat(node.clusterState.nodes().getLocalNode(), equalTo(node.discoveryNode)); - } - - assertWarnings( - "[discovery.zen.publish_diff.enable] 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 testSerializationFailureDuringDiffPublishing() throws Exception { - MockNode nodeA = createMockNode("nodeA", Settings.EMPTY, new ClusterStateListener() { - @Override - public void clusterChanged(ClusterChangedEvent event) { - fail("Shouldn't send cluster state to myself"); - } - }).setAsMaster(); - - MockNode nodeB = createMockNode("nodeB"); - - // Initial cluster state with both states - the second node still shouldn't get - // diff even though it's present in the previous cluster state - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(nodeA.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = ClusterState.builder(CLUSTER_NAME).nodes(discoveryNodes).build(); - ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - assertSameStateFromFull(nodeB.clusterState, clusterState); - - // cluster state update - add block - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState) - .blocks(ClusterBlocks.builder().addGlobalBlock(Metadata.CLUSTER_READ_ONLY_BLOCK)) - .incrementVersion() - .build(); - - ClusterState unserializableClusterState = new ClusterState(clusterState.version(), clusterState.stateUUID(), clusterState) { - @Override - public Diff diff(ClusterState previousState) { - return new Diff() { - @Override - public ClusterState apply(ClusterState part) { - fail("this diff shouldn't be applied"); - return part; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - throw new IOException("Simulated failure of diff serialization"); - } - }; - } - }; - try { - publishStateAndWait(nodeA.action, unserializableClusterState, previousClusterState); - fail("cluster state published despite of diff errors"); - } catch (FailedToCommitClusterStateException e) { - assertThat(e.getCause(), notNullValue()); - assertThat(e.getCause().getMessage(), containsString("failed to serialize")); - } - } - - public void testFailToPublishWithLessThanMinMasterNodes() throws Exception { - final int masterNodes = randomIntBetween(1, 10); - - MockNode master = createMockNode("master"); - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder().add(master.discoveryNode); - for (int i = 1; i < masterNodes; i++) { - discoveryNodesBuilder.add(createMockNode("node" + i).discoveryNode); - } - final int dataNodes = randomIntBetween(0, 5); - final Settings dataSettings = Settings.builder().put(nonMasterNode()).build(); - for (int i = 0; i < dataNodes; i++) { - discoveryNodesBuilder.add(createMockNode("data_" + i, dataSettings, null).discoveryNode); - } - discoveryNodesBuilder.localNodeId(master.discoveryNode.getId()).masterNodeId(master.discoveryNode.getId()); - DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); - Metadata metadata = Metadata.EMPTY_METADATA; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).metadata(metadata).nodes(discoveryNodes).build(); - ClusterState previousState = master.clusterState; - try { - publishState(master.action, clusterState, previousState, masterNodes + randomIntBetween(1, 5)); - fail("cluster state publishing didn't fail despite of not having enough nodes"); - } catch (FailedToCommitClusterStateException expected) { - logger.debug("failed to publish as expected", expected); - } - } - - public void testPublishingWithSendingErrors() throws Exception { - int goodNodes = randomIntBetween(2, 5); - int errorNodes = randomIntBetween(1, 5); - int timeOutNodes = randomBoolean() ? 0 : randomIntBetween(1, 5); // adding timeout nodes will force timeout errors - final int numberOfMasterNodes = goodNodes + errorNodes + timeOutNodes + 1; // master - final boolean expectingToCommit = randomBoolean(); - Settings.Builder settings = Settings.builder(); - // make sure we have a reasonable timeout if we expect to timeout, o.w. one that will make the test "hang" - settings.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), expectingToCommit == false && timeOutNodes > 0 ? "100ms" : "1h") - .put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "5ms"); // test is about committing - - MockNode master = createMockNode("master", settings.build(), null); - - // randomize things a bit - int[] nodeTypes = new int[goodNodes + errorNodes + timeOutNodes]; - for (int i = 0; i < goodNodes; i++) { - nodeTypes[i] = 0; - } - for (int i = goodNodes; i < goodNodes + errorNodes; i++) { - nodeTypes[i] = 1; - } - for (int i = goodNodes + errorNodes; i < nodeTypes.length; i++) { - nodeTypes[i] = 2; - } - Collections.shuffle(Arrays.asList(nodeTypes), random()); - - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder().add(master.discoveryNode); - for (int i = 0; i < nodeTypes.length; i++) { - final MockNode mockNode = createMockNode("node" + i); - discoveryNodesBuilder.add(mockNode.discoveryNode); - switch (nodeTypes[i]) { - case 1: - mockNode.action.errorOnSend.set(true); - break; - case 2: - mockNode.action.timeoutOnSend.set(true); - break; - } - } - final int dataNodes = randomIntBetween(0, 3); // data nodes don't matter - for (int i = 0; i < dataNodes; i++) { - final MockNode mockNode = createMockNode("data_" + i, Settings.builder().put(nonMasterNode()).build(), null); - discoveryNodesBuilder.add(mockNode.discoveryNode); - if (randomBoolean()) { - // we really don't care - just chaos monkey - mockNode.action.errorOnCommit.set(randomBoolean()); - mockNode.action.errorOnSend.set(randomBoolean()); - mockNode.action.timeoutOnCommit.set(randomBoolean()); - mockNode.action.timeoutOnSend.set(randomBoolean()); - } - } - - final int minMasterNodes; - final String expectedBehavior; - if (expectingToCommit) { - minMasterNodes = randomIntBetween(0, goodNodes + 1); // count master - expectedBehavior = "succeed"; - } else { - minMasterNodes = randomIntBetween(goodNodes + 2, numberOfMasterNodes); // +2 because of master - expectedBehavior = timeOutNodes > 0 ? "timeout" : "fail"; - } - logger.info( - "--> expecting commit to {}. good nodes [{}], errors [{}], timeouts [{}]. min_master_nodes [{}]", - expectedBehavior, - goodNodes + 1, - errorNodes, - timeOutNodes, - minMasterNodes - ); - - discoveryNodesBuilder.localNodeId(master.discoveryNode.getId()).masterNodeId(master.discoveryNode.getId()); - DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); - Metadata metadata = Metadata.EMPTY_METADATA; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME).metadata(metadata).nodes(discoveryNodes).build(); - ClusterState previousState = master.clusterState; - try { - publishState(master.action, clusterState, previousState, minMasterNodes); - if (expectingToCommit == false) { - fail("cluster state publishing didn't fail despite of not have enough nodes"); - } - } catch (FailedToCommitClusterStateException exception) { - logger.debug("failed to publish as expected", exception); - if (expectingToCommit) { - throw exception; - } - assertThat(exception.getMessage(), containsString(timeOutNodes > 0 ? "timed out" : "failed")); - } - - assertWarnings( - "[discovery.zen.publish_timeout] 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.commit_timeout] 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 testOutOfOrderCommitMessages() throws Throwable { - MockNode node = createMockNode("node").setAsMaster(); - final CapturingTransportChannel channel = new CapturingTransportChannel(); - - List states = new ArrayList<>(); - final int numOfStates = scaledRandomIntBetween(3, 25); - for (int i = 1; i <= numOfStates; i++) { - states.add(ClusterState.builder(node.clusterState).version(i).stateUUID(ClusterState.UNKNOWN_UUID).build()); - } - - final ClusterState finalState = states.get(numOfStates - 1); - - logger.info("--> publishing states"); - for (ClusterState state : states) { - node.action.handleIncomingClusterStateRequest( - new BytesTransportRequest(PublishClusterStateAction.serializeFullClusterState(state, Version.CURRENT), Version.CURRENT), - channel - ); - assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE)); - assertThat(channel.error.get(), nullValue()); - channel.clear(); - - } - - logger.info("--> committing states"); - - long largestVersionSeen = Long.MIN_VALUE; - Randomness.shuffle(states); - for (ClusterState state : states) { - node.action.handleCommitRequest(new PublishClusterStateAction.CommitClusterStateRequest(state.stateUUID()), channel); - if (largestVersionSeen < state.getVersion()) { - assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE)); - if (channel.error.get() != null) { - throw channel.error.get(); - } - largestVersionSeen = state.getVersion(); - } else { - // older cluster states will be rejected - assertNotNull(channel.error.get()); - assertThat(channel.error.get(), instanceOf(IllegalStateException.class)); - } - channel.clear(); - } - - // now check the last state held - assertSameState(node.clusterState, finalState); - } - - /** - * Tests that cluster is committed or times out. It should never be the case that we fail - * an update due to a commit timeout, but it ends up being committed anyway - */ - public void testTimeoutOrCommit() throws Exception { - Settings settings = Settings.builder() - // short but so we will sometime commit sometime timeout - .put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "1ms") - .build(); - - MockNode master = createMockNode("master", settings, null); - MockNode node = createMockNode("node", settings, null); - ClusterState state = ClusterState.builder(master.clusterState) - .nodes(DiscoveryNodes.builder(master.clusterState.nodes()).add(node.discoveryNode).masterNodeId(master.discoveryNode.getId())) - .build(); - - for (int i = 0; i < 10; i++) { - state = ClusterState.builder(state).incrementVersion().build(); - logger.debug("--> publishing version [{}], UUID [{}]", state.version(), state.stateUUID()); - boolean success; - try { - publishState(master.action, state, master.clusterState, 2).await(1, TimeUnit.HOURS); - success = true; - } catch (FailedToCommitClusterStateException OK) { - success = false; - } - logger.debug("--> publishing [{}], verifying...", success ? "succeeded" : "failed"); - - if (success) { - assertSameState(node.clusterState, state); - } else { - assertThat(node.clusterState.stateUUID(), not(equalTo(state.stateUUID()))); - } - } - - assertWarnings( - "[discovery.zen.commit_timeout] setting was deprecated in OpenSearch and will be removed in a future release! " - + "See the breaking changes documentation for the next major version." - ); - } - - private void assertPublishClusterStateStats( - String description, - MockNode node, - long expectedFull, - long expectedIncompatibleDiffs, - long expectedCompatibleDiffs - ) { - PublishClusterStateStats stats = node.action.stats(); - assertThat(description + ": full cluster states", stats.getFullClusterStateReceivedCount(), equalTo(expectedFull)); - assertThat( - description + ": incompatible cluster state diffs", - stats.getIncompatibleClusterStateDiffReceivedCount(), - equalTo(expectedIncompatibleDiffs) - ); - assertThat( - description + ": compatible cluster state diffs", - stats.getCompatibleClusterStateDiffReceivedCount(), - equalTo(expectedCompatibleDiffs) - ); - } - - public void testPublishClusterStateStats() throws Exception { - MockNode nodeA = createMockNode("nodeA").setAsMaster(); - MockNode nodeB = createMockNode("nodeB"); - - assertPublishClusterStateStats("nodeA: initial state", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: initial state", nodeB, 0, 0, 0); - - // Initial cluster state - ClusterState clusterState = nodeA.clusterState; - - // cluster state update - add nodeB - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(clusterState.nodes()).add(nodeB.discoveryNode).build(); - ClusterState previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // Sent as a full cluster state update - assertPublishClusterStateStats("nodeA: after full update", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: after full update", nodeB, 1, 0, 0); - - // Increment cluster state version - previousClusterState = clusterState; - clusterState = ClusterState.builder(clusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // Sent, successfully, as a cluster state diff - assertPublishClusterStateStats("nodeA: after successful diff update", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: after successful diff update", nodeB, 1, 0, 1); - - // Increment cluster state version twice - previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); - clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); - publishStateAndWait(nodeA.action, clusterState, previousClusterState); - - // Sent, unsuccessfully, as a diff and then retried as a full update - assertPublishClusterStateStats("nodeA: after unsuccessful diff update", nodeA, 0, 0, 0); - assertPublishClusterStateStats("nodeB: after unsuccessful diff update", nodeB, 2, 1, 1); - - // node A steps down from being master - nodeA.resetMasterId(); - nodeB.resetMasterId(); - - // node B becomes the master and sends a version of the cluster state that goes back - discoveryNodes = DiscoveryNodes.builder(discoveryNodes) - .add(nodeA.discoveryNode) - .add(nodeB.discoveryNode) - .masterNodeId(nodeB.discoveryNode.getId()) - .localNodeId(nodeB.discoveryNode.getId()) - .build(); - previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); - clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); - publishStateAndWait(nodeB.action, clusterState, previousClusterState); - - // Sent, unsuccessfully, as a diff, and then retried as a full update - assertPublishClusterStateStats("nodeA: B became master", nodeA, 1, 1, 0); - assertPublishClusterStateStats("nodeB: B became master", nodeB, 2, 1, 1); - } - - private Metadata buildMetadataForVersion(Metadata metadata, long version) { - ImmutableOpenMap.Builder indices = ImmutableOpenMap.builder(metadata.indices()); - indices.put( - "test" + version, - IndexMetadata.builder("test" + version) - .settings(Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards((int) version) - .numberOfReplicas(0) - .build() - ); - return Metadata.builder(metadata) - .transientSettings(Settings.builder().put("test", version).build()) - .indices(indices.build()) - .build(); - } - - private void assertProperMetadataForVersion(Metadata metadata, long version) { - for (long i = 1; i <= version; i++) { - assertThat(metadata.index("test" + i), notNullValue()); - assertThat(metadata.index("test" + i).getNumberOfShards(), equalTo((int) i)); - } - assertThat(metadata.index("test" + (version + 1)), nullValue()); - assertThat(metadata.transientSettings().get("test"), equalTo(Long.toString(version))); - } - - public void publishStateAndWait(PublishClusterStateAction action, ClusterState state, ClusterState previousState) - throws InterruptedException { - publishState(action, state, previousState).await(1, TimeUnit.SECONDS); - } - - public AssertingAckListener publishState(PublishClusterStateAction action, ClusterState state, ClusterState previousState) - throws InterruptedException { - final int minimumMasterNodes = randomIntBetween(-1, state.nodes().getMasterNodes().size()); - return publishState(action, state, previousState, minimumMasterNodes); - } - - public AssertingAckListener publishState( - PublishClusterStateAction action, - ClusterState state, - ClusterState previousState, - int minMasterNodes - ) throws InterruptedException { - AssertingAckListener assertingAckListener = new AssertingAckListener(state.nodes().getSize() - 1); - ClusterChangedEvent changedEvent = new ClusterChangedEvent("test update", state, previousState); - action.publish(changedEvent, minMasterNodes, assertingAckListener); - return assertingAckListener; - } - - public static class AssertingAckListener implements Discovery.AckListener { - private final List> errors = new CopyOnWriteArrayList<>(); - private final Set successfulAcks = Collections.synchronizedSet(new HashSet<>()); - private final CountDownLatch countDown; - private final CountDownLatch commitCountDown; - - public AssertingAckListener(int nodeCount) { - countDown = new CountDownLatch(nodeCount); - commitCountDown = new CountDownLatch(1); - } - - @Override - public void onCommit(TimeValue commitTime) { - commitCountDown.countDown(); - } - - @Override - public void onNodeAck(DiscoveryNode node, @Nullable Exception e) { - if (e != null) { - errors.add(new Tuple<>(node, e)); - } else { - successfulAcks.add(node); - } - countDown.countDown(); - } - - public Set await(long timeout, TimeUnit unit) throws InterruptedException { - assertThat(awaitErrors(timeout, unit), emptyIterable()); - assertTrue(commitCountDown.await(timeout, unit)); - return new HashSet<>(successfulAcks); - } - - public List> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException { - countDown.await(timeout, unit); - return errors; - } - - } - - void assertSameState(ClusterState actual, ClusterState expected) { - assertThat(actual, notNullValue()); - final String reason = "\n--> actual ClusterState: " + actual + "\n" + "--> expected ClusterState:" + expected; - assertThat("unequal UUIDs" + reason, actual.stateUUID(), equalTo(expected.stateUUID())); - assertThat("unequal versions" + reason, actual.version(), equalTo(expected.version())); - } - - void assertSameStateFromDiff(ClusterState actual, ClusterState expected) { - assertSameState(actual, expected); - assertTrue(actual.wasReadFromDiff()); - } - - void assertSameStateFromFull(ClusterState actual, ClusterState expected) { - assertSameState(actual, expected); - assertFalse(actual.wasReadFromDiff()); - } - - public static class MockPublishAction extends PublishClusterStateAction { - - AtomicBoolean timeoutOnSend = new AtomicBoolean(); - AtomicBoolean errorOnSend = new AtomicBoolean(); - AtomicBoolean timeoutOnCommit = new AtomicBoolean(); - AtomicBoolean errorOnCommit = new AtomicBoolean(); - - public MockPublishAction( - TransportService transportService, - NamedWriteableRegistry namedWriteableRegistry, - IncomingClusterStateListener listener, - DiscoverySettings discoverySettings - ) { - super(transportService, namedWriteableRegistry, listener, discoverySettings); - } - - @Override - protected void handleIncomingClusterStateRequest(BytesTransportRequest request, TransportChannel channel) throws IOException { - if (errorOnSend.get()) { - throw new OpenSearchException("forced error on incoming cluster state"); - } - if (timeoutOnSend.get()) { - return; - } - super.handleIncomingClusterStateRequest(request, channel); - } - - @Override - protected void handleCommitRequest(PublishClusterStateAction.CommitClusterStateRequest request, TransportChannel channel) { - if (errorOnCommit.get()) { - throw new OpenSearchException("forced error on incoming commit"); - } - if (timeoutOnCommit.get()) { - return; - } - super.handleCommitRequest(request, channel); - } - } - - static class CapturingTransportChannel implements TransportChannel { - - AtomicReference response = new AtomicReference<>(); - AtomicReference error = new AtomicReference<>(); - - public void clear() { - response.set(null); - error.set(null); - } - - @Override - public String getProfileName() { - return "_noop_"; - } - - @Override - public void sendResponse(TransportResponse response) throws IOException { - this.response.set(response); - assertThat(error.get(), nullValue()); - } - - @Override - public void sendResponse(Exception exception) throws IOException { - this.error.set(exception); - assertThat(response.get(), nullValue()); - } - - @Override - public String getChannelType() { - return "capturing"; - } - } -} diff --git a/server/src/test/java/org/opensearch/discovery/zen/ZenDiscoveryUnitTests.java b/server/src/test/java/org/opensearch/discovery/zen/ZenDiscoveryUnitTests.java deleted file mode 100644 index c2046d4c2d6..00000000000 --- a/server/src/test/java/org/opensearch/discovery/zen/ZenDiscoveryUnitTests.java +++ /dev/null @@ -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 responses = new ArrayList<>(); - ArrayList masterNodes = new ArrayList<>(); - ArrayList allNodes = new ArrayList<>(); - for (int i = randomIntBetween(10, 20); i >= 0; i--) { - Set 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 filtered = ZenDiscovery.filterPingResponses(responses, ignore, logger); - final List 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 toClose = new ArrayDeque<>(); - try { - Set 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 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 { - 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 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 fdNodesForState(ClusterState clusterState, DiscoveryNode localNode) { - final Set 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 submitRejoin = source -> rejoinCalled.set(true); - - final AtomicReference 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 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 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); - } -} diff --git a/server/src/test/java/org/opensearch/discovery/zen/ZenPingTests.java b/server/src/test/java/org/opensearch/discovery/zen/ZenPingTests.java deleted file mode 100644 index ea48aca4ead..00000000000 --- a/server/src/test/java/org/opensearch/discovery/zen/ZenPingTests.java +++ /dev/null @@ -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 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 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); - } - - } -} diff --git a/server/src/test/java/org/opensearch/rest/action/admin/indices/RestResizeHandlerTests.java b/server/src/test/java/org/opensearch/rest/action/admin/indices/RestResizeHandlerTests.java deleted file mode 100644 index e07442ecfbf..00000000000 --- a/server/src/test/java/org/opensearch/rest/action/admin/indices/RestResizeHandlerTests.java +++ /dev/null @@ -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"); - } - } - } - -} diff --git a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java index 890f2c9a15f..21aa6a1606c 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -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> 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> 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; diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 251d7480be4..b7715fef1df 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -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 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; diff --git a/test/framework/src/test/java/org/opensearch/test/test/InternalTestClusterTests.java b/test/framework/src/test/java/org/opensearch/test/test/InternalTestClusterTests.java index 2374f9b1fa6..c3b483eb85e 100644 --- a/test/framework/src/test/java/org/opensearch/test/test/InternalTestClusterTests.java +++ b/test/framework/src/test/java/org/opensearch/test/test/InternalTestClusterTests.java @@ -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;