Skip to content

Commit

Permalink
Zen2: Move disruption tests to Zen2 (#35724)
Browse files Browse the repository at this point in the history
- Moves disruption tests to Zen2
- Registers a few missing settings
- Removes .put(TestZenDiscovery.USE_ZEN2.getKey(), true) from tests where Zen2 is now enabled
by default through the parent test class
- Moves QuorumGatewayIT back to Zen1, as it is not stable with Zen2 as it currently relies on
dangling indices due to the lack of proper CS persistence, which triggers secondary failures
  • Loading branch information
ywelsch authored Nov 21, 2018
1 parent 47ada69 commit 8939a78
Show file tree
Hide file tree
Showing 10 changed files with 68 additions and 58 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,9 @@
import org.elasticsearch.cluster.coordination.ClusterBootstrapService;
import org.elasticsearch.cluster.coordination.Coordinator;
import org.elasticsearch.cluster.coordination.ElectionSchedulerFactory;
import org.elasticsearch.cluster.coordination.FollowersChecker;
import org.elasticsearch.cluster.coordination.JoinHelper;
import org.elasticsearch.cluster.coordination.LeaderChecker;
import org.elasticsearch.cluster.coordination.Reconfigurator;
import org.elasticsearch.cluster.metadata.IndexGraveyard;
import org.elasticsearch.cluster.metadata.MetaData;
Expand Down Expand Up @@ -459,6 +461,12 @@ public void apply(Settings value, Settings current, Settings previous) {
ElectionSchedulerFactory.ELECTION_DURATION_SETTING,
Coordinator.PUBLISH_TIMEOUT_SETTING,
JoinHelper.JOIN_TIMEOUT_SETTING,
FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING,
FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING,
FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING,
LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING,
LeaderChecker.LEADER_CHECK_INTERVAL_SETTING,
LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING,
Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION,
TransportAddVotingTombstonesAction.MAXIMUM_VOTING_TOMBSTONES_SETTING,
ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,8 @@

import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.discovery.TestZenDiscovery;

import java.util.Arrays;

Expand All @@ -36,13 +34,6 @@
@ClusterScope(scope = ESIntegTestCase.Scope.TEST)
public class IndicesStatsBlocksIT extends ESIntegTestCase {

@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put(TestZenDiscovery.USE_ZEN2.getKey(), true)
.build();
}

public void testIndicesStatsWithBlocks() {
createIndex("ro");
ensureGreen("ro");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalSettingsPlugin;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.discovery.TestZenDiscovery;
import org.elasticsearch.test.transport.MockTransportService;

import java.io.IOException;
Expand All @@ -67,13 +66,6 @@
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0)
public class AllocationIdIT extends ESIntegTestCase {

@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put(TestZenDiscovery.USE_ZEN2.getKey(), true)
.build();
}

@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(MockTransportService.TestPlugin.class, MockEngineFactoryPlugin.class, InternalSettingsPlugin.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,10 @@
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.coordination.Coordinator;
import org.elasticsearch.cluster.coordination.FollowersChecker;
import org.elasticsearch.cluster.coordination.JoinHelper;
import org.elasticsearch.cluster.coordination.LeaderChecker;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
Expand Down Expand Up @@ -63,7 +67,6 @@ public abstract class AbstractDisruptionTestCase extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(DEFAULT_SETTINGS)
.put(TestZenDiscovery.USE_ZEN2.getKey(), false) // requires more work
.put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false).build();
}

Expand Down Expand Up @@ -114,18 +117,31 @@ List<String> startCluster(int numberOfNodes) {
ensureStableCluster(numberOfNodes);

// TODO: this is a temporary solution so that nodes will not base their reaction to a partition based on previous successful results
ZenPing zenPing = ((TestZenDiscovery) internalCluster().getInstance(Discovery.class)).getZenPing();
if (zenPing instanceof UnicastZenPing) {
((UnicastZenPing) zenPing).clearTemporalResponses();
}
clearTemporalResponses();
return nodes;
}

protected void clearTemporalResponses() {
final Discovery discovery = internalCluster().getInstance(Discovery.class);
if (discovery instanceof TestZenDiscovery) {
ZenPing zenPing = ((TestZenDiscovery) discovery).getZenPing();
if (zenPing instanceof UnicastZenPing) {
((UnicastZenPing) zenPing).clearTemporalResponses();
}
}
}

static final Settings DEFAULT_SETTINGS = Settings.builder()
.put(FaultDetection.PING_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly
.put(FaultDetection.PING_RETRIES_SETTING.getKey(), "1") // for hitting simulated network failures quickly
.put(LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly
.put(LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) // for hitting simulated network failures quickly
.put(FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING.getKey(), "1s") // for hitting simulated network failures quickly
.put(FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) // for hitting simulated network failures quickly
.put("discovery.zen.join_timeout", "10s") // still long to induce failures but to long so test won't time out
.put(JoinHelper.JOIN_TIMEOUT_SETTING.getKey(), "10s") // still long to induce failures but to long so test won't time out
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put(Coordinator.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put(TransportService.TCP_CONNECT_TIMEOUT.getKey(), "10s") // Network delay disruption waits for the min between this
// value and the time of disruption and does not recover immediately
// when disruption is stop. We should make sure we recover faster
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.discovery.TestZenDiscovery;
import org.elasticsearch.test.disruption.NetworkDisruption;
import org.elasticsearch.test.disruption.NetworkDisruption.Bridge;
import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect;
Expand Down Expand Up @@ -356,8 +357,9 @@ public void onFailure(Exception e) {

public void testIndexImportedFromDataOnlyNodesIfMasterLostDataFolder() throws Exception {
// test for https://github.com/elastic/elasticsearch/issues/8823
String masterNode = internalCluster().startMasterOnlyNode(Settings.EMPTY);
internalCluster().startDataOnlyNode(Settings.EMPTY);
Settings zen1Settings = Settings.builder().put(TestZenDiscovery.USE_ZEN2.getKey(), false).build(); // TODO: needs adaptions for Zen2
String masterNode = internalCluster().startMasterOnlyNode(zen1Settings);
internalCluster().startDataOnlyNode(zen1Settings);
ensureStableCluster(2);
assertAcked(prepareCreate("index").setSettings(Settings.builder().put("index.number_of_replicas", 0)));
index("index", "_doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,16 @@
package org.elasticsearch.discovery;

import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.coordination.JoinHelper;
import org.elasticsearch.cluster.coordination.PublicationTransportHandler;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.zen.MembershipAction;
import org.elasticsearch.discovery.zen.PublishClusterStateAction;
import org.elasticsearch.discovery.zen.UnicastZenPing;
import org.elasticsearch.discovery.zen.ZenPing;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.discovery.TestZenDiscovery;
import org.elasticsearch.test.disruption.NetworkDisruption;
import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect;
import org.elasticsearch.test.disruption.NetworkDisruption.TwoPartitions;
Expand Down Expand Up @@ -73,10 +73,7 @@ public void testIsolatedUnicastNodes() throws Exception {

// Forcefully clean temporal response lists on all nodes. Otherwise the node in the unicast host list
// includes all the other nodes that have pinged it and the issue doesn't manifest
ZenPing zenPing = ((TestZenDiscovery) internalCluster().getInstance(Discovery.class)).getZenPing();
if (zenPing instanceof UnicastZenPing) {
((UnicastZenPing) zenPing).clearTemporalResponses();
}
clearTemporalResponses();

// Simulate a network issue between the unicast target node and the rest of the cluster
NetworkDisruption networkDisconnect = new NetworkDisruption(new TwoPartitions(unicastTargetSide, restOfClusterSide),
Expand Down Expand Up @@ -111,10 +108,7 @@ public void testUnicastSinglePingResponseContainsMaster() throws Exception {

// Forcefully clean temporal response lists on all nodes. Otherwise the node in the unicast host list
// includes all the other nodes that have pinged it and the issue doesn't manifest
ZenPing zenPing = ((TestZenDiscovery) internalCluster().getInstance(Discovery.class)).getZenPing();
if (zenPing instanceof UnicastZenPing) {
((UnicastZenPing) zenPing).clearTemporalResponses();
}
clearTemporalResponses();

// Simulate a network issue between the unlucky node and elected master node in both directions.
NetworkDisruption networkDisconnect = new NetworkDisruption(new TwoPartitions(masterNode, isolatedNode),
Expand Down Expand Up @@ -160,14 +154,17 @@ public void testClusterJoinDespiteOfPublishingIssues() throws Exception {
internalCluster().getInstance(TransportService.class, discoveryNodes.getLocalNode().getName());
if (randomBoolean()) {
masterTransportService.addFailToSendNoConnectRule(localTransportService, PublishClusterStateAction.SEND_ACTION_NAME);
masterTransportService.addFailToSendNoConnectRule(localTransportService, PublicationTransportHandler.PUBLISH_STATE_ACTION_NAME);
} else {
masterTransportService.addFailToSendNoConnectRule(localTransportService, PublishClusterStateAction.COMMIT_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)) {
if (action.equals(MembershipAction.DISCOVERY_JOIN_ACTION_NAME) ||
action.equals(JoinHelper.JOIN_ACTION_NAME)) {
countDownLatch.countDown();
}
connection.sendRequest(requestId, action, request, options);
Expand Down Expand Up @@ -219,9 +216,13 @@ public void testElectMasterWithLatestVersion() throws Exception {
ensureStableCluster(3);
final String preferredMasterName = internalCluster().getMasterName();
final DiscoveryNode preferredMaster = internalCluster().clusterService(preferredMasterName).localNode();
for (String node : nodes) {
DiscoveryNode discoveryNode = internalCluster().clusterService(node).localNode();
assertThat(discoveryNode.getId(), greaterThanOrEqualTo(preferredMaster.getId()));
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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.monitor.jvm.HotThreads;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.discovery.TestZenDiscovery;
import org.elasticsearch.test.disruption.BlockMasterServiceOnMaster;
import org.elasticsearch.test.disruption.IntermittentLongGCDisruption;
import org.elasticsearch.test.disruption.LongGCDisruption;
Expand Down Expand Up @@ -379,7 +380,8 @@ public void testIsolateMasterAndVerifyClusterStateConsensus() throws Exception {
* Verify that the proper block is applied when nodes loose their master
*/
public void testVerifyApiBlocksDuringPartition() throws Exception {
startCluster(3);
// TODO: NO_MASTER_BLOCKS not dynamic in Zen2 yet
internalCluster().startNodes(3, Settings.builder().put(TestZenDiscovery.USE_ZEN2.getKey(), false).build());

// Makes sure that the get request can be executed on each node locally:
assertAcked(prepareCreate("test").setSettings(Settings.builder()
Expand Down Expand Up @@ -511,7 +513,13 @@ void assertDiscoveryCompleted(List<String> nodes) throws InterruptedException {
assertTrue(
"node [" + node + "] is still joining master",
awaitBusy(
() -> !((ZenDiscovery) internalCluster().getInstance(Discovery.class, node)).joiningCluster(),
() -> {
final Discovery discovery = internalCluster().getInstance(Discovery.class, node);
if (discovery instanceof ZenDiscovery) {
return !((ZenDiscovery) discovery).joiningCluster();
}
return true;
},
30,
TimeUnit.SECONDS
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.elasticsearch.test.InternalTestCluster.RestartCallback;
import org.elasticsearch.test.discovery.TestZenDiscovery;

import java.util.concurrent.TimeUnit;

Expand All @@ -38,6 +39,14 @@

@ClusterScope(numDataNodes = 0, scope = Scope.TEST)
public class QuorumGatewayIT extends ESIntegTestCase {

@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put(TestZenDiscovery.USE_ZEN2.getKey(), false) // no state persistence yet
.build();
}

@Override
protected int numberOfReplicas() {
return 2;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@
import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.elasticsearch.test.InternalSettingsPlugin;
import org.elasticsearch.test.MockIndexEventListener;
import org.elasticsearch.test.discovery.TestZenDiscovery;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.test.transport.StubbableTransport;
Expand Down Expand Up @@ -100,13 +99,6 @@ protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(InternalSettingsPlugin.class, MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class);
}

@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put(TestZenDiscovery.USE_ZEN2.getKey(), true)
.build();
}

@Override
protected void beforeIndexDeletion() throws Exception {
super.beforeIndexDeletion();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,8 @@
*/
package org.elasticsearch.test.test;

import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.discovery.TestZenDiscovery;

import java.io.IOException;

Expand All @@ -30,13 +28,6 @@
@ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class InternalTestClusterIT extends ESIntegTestCase {

@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put(TestZenDiscovery.USE_ZEN2.getKey(), true)
.build();
}

public void testStartingAndStoppingNodes() throws IOException {
logger.info("--> cluster has [{}] nodes", internalCluster().size());
if (internalCluster().size() < 5) {
Expand Down

0 comments on commit 8939a78

Please sign in to comment.