7777import org .elasticsearch .search .SearchService ;
7878import org .elasticsearch .test .cache .recycler .MockBigArraysModule ;
7979import org .elasticsearch .test .cache .recycler .MockPageCacheRecyclerModule ;
80+ import org .elasticsearch .test .disruption .ServiceDisruptionScheme ;
8081import org .elasticsearch .test .engine .MockEngineModule ;
8182import org .elasticsearch .test .store .MockFSIndexStoreModule ;
8283import org .elasticsearch .test .transport .AssertingLocalTransportModule ;
@@ -185,6 +186,8 @@ public final class InternalTestCluster extends TestCluster {
185186
186187 private final boolean hasFilterCache ;
187188
189+ private ServiceDisruptionScheme activeDisruptionScheme ;
190+
188191 public InternalTestCluster (long clusterSeed , String clusterName ) {
189192 this (clusterSeed , DEFAULT_MIN_NUM_DATA_NODES , DEFAULT_MAX_NUM_DATA_NODES , clusterName , SettingsSource .EMPTY , DEFAULT_NUM_CLIENT_NODES , DEFAULT_ENABLE_RANDOM_BENCH_NODES );
190193 }
@@ -288,6 +291,10 @@ public String getClusterName() {
288291 return clusterName ;
289292 }
290293
294+ public String [] getNodeNames () {
295+ return nodes .keySet ().toArray (Strings .EMPTY_ARRAY );
296+ }
297+
291298 private static boolean isLocalTransportConfigured () {
292299 if ("local" .equals (System .getProperty ("es.node.mode" , "network" ))) {
293300 return true ;
@@ -487,6 +494,7 @@ public synchronized void ensureAtMostNumDataNodes(int n) throws IOException {
487494 while (limit .hasNext ()) {
488495 NodeAndClient next = limit .next ();
489496 nodesToRemove .add (next );
497+ removeDistruptionSchemeFromNode (next );
490498 next .close ();
491499 }
492500 for (NodeAndClient toRemove : nodesToRemove ) {
@@ -661,6 +669,10 @@ public boolean apply(NodeAndClient nodeAndClient) {
661669 @ Override
662670 public void close () {
663671 if (this .open .compareAndSet (true , false )) {
672+ if (activeDisruptionScheme != null ) {
673+ activeDisruptionScheme .testClusterClosed ();
674+ activeDisruptionScheme = null ;
675+ }
664676 IOUtils .closeWhileHandlingException (nodes .values ());
665677 nodes .clear ();
666678 executor .shutdownNow ();
@@ -858,6 +870,7 @@ public synchronized void beforeTest(Random random, double transportClientRatio)
858870 }
859871
860872 private synchronized void reset (boolean wipeData ) throws IOException {
873+ clearDisruptionScheme ();
861874 resetClients (); /* reset all clients - each test gets its own client based on the Random instance created above. */
862875 if (wipeData ) {
863876 wipeDataDirectories ();
@@ -1054,6 +1067,7 @@ public synchronized void stopRandomDataNode() throws IOException {
10541067 NodeAndClient nodeAndClient = getRandomNodeAndClient (new DataNodePredicate ());
10551068 if (nodeAndClient != null ) {
10561069 logger .info ("Closing random node [{}] " , nodeAndClient .name );
1070+ removeDistruptionSchemeFromNode (nodeAndClient );
10571071 nodes .remove (nodeAndClient .name );
10581072 nodeAndClient .close ();
10591073 }
@@ -1073,6 +1087,7 @@ public boolean apply(NodeAndClient nodeAndClient) {
10731087 });
10741088 if (nodeAndClient != null ) {
10751089 logger .info ("Closing filtered random node [{}] " , nodeAndClient .name );
1090+ removeDistruptionSchemeFromNode (nodeAndClient );
10761091 nodes .remove (nodeAndClient .name );
10771092 nodeAndClient .close ();
10781093 }
@@ -1087,6 +1102,7 @@ public synchronized void stopCurrentMasterNode() throws IOException {
10871102 String masterNodeName = getMasterName ();
10881103 assert nodes .containsKey (masterNodeName );
10891104 logger .info ("Closing master node [{}] " , masterNodeName );
1105+ removeDistruptionSchemeFromNode (nodes .get (masterNodeName ));
10901106 NodeAndClient remove = nodes .remove (masterNodeName );
10911107 remove .close ();
10921108 }
@@ -1098,6 +1114,7 @@ public void stopRandomNonMasterNode() throws IOException {
10981114 NodeAndClient nodeAndClient = getRandomNodeAndClient (Predicates .not (new MasterNodePredicate (getMasterName ())));
10991115 if (nodeAndClient != null ) {
11001116 logger .info ("Closing random non master node [{}] current master [{}] " , nodeAndClient .name , getMasterName ());
1117+ removeDistruptionSchemeFromNode (nodeAndClient );
11011118 nodes .remove (nodeAndClient .name );
11021119 nodeAndClient .close ();
11031120 }
@@ -1151,6 +1168,9 @@ private void restartAllNodes(boolean rollingRestart, RestartCallback callback) t
11511168 if (!callback .doRestart (nodeAndClient .name )) {
11521169 logger .info ("Closing node [{}] during restart" , nodeAndClient .name );
11531170 toRemove .add (nodeAndClient );
1171+ if (activeDisruptionScheme != null ) {
1172+ activeDisruptionScheme .removeFromNode (nodeAndClient .name , this );
1173+ }
11541174 nodeAndClient .close ();
11551175 }
11561176 }
@@ -1165,18 +1185,33 @@ private void restartAllNodes(boolean rollingRestart, RestartCallback callback) t
11651185 for (NodeAndClient nodeAndClient : nodes .values ()) {
11661186 callback .doAfterNodes (numNodesRestarted ++, nodeAndClient .nodeClient ());
11671187 logger .info ("Restarting node [{}] " , nodeAndClient .name );
1188+ if (activeDisruptionScheme != null ) {
1189+ activeDisruptionScheme .removeFromNode (nodeAndClient .name , this );
1190+ }
11681191 nodeAndClient .restart (callback );
1192+ if (activeDisruptionScheme != null ) {
1193+ activeDisruptionScheme .applyToNode (nodeAndClient .name , this );
1194+ }
11691195 }
11701196 } else {
11711197 int numNodesRestarted = 0 ;
11721198 for (NodeAndClient nodeAndClient : nodes .values ()) {
11731199 callback .doAfterNodes (numNodesRestarted ++, nodeAndClient .nodeClient ());
11741200 logger .info ("Stopping node [{}] " , nodeAndClient .name );
1201+ if (activeDisruptionScheme != null ) {
1202+ activeDisruptionScheme .removeFromNode (nodeAndClient .name , this );
1203+ }
11751204 nodeAndClient .node .close ();
11761205 }
11771206 for (NodeAndClient nodeAndClient : nodes .values ()) {
11781207 logger .info ("Starting node [{}] " , nodeAndClient .name );
1208+ if (activeDisruptionScheme != null ) {
1209+ activeDisruptionScheme .removeFromNode (nodeAndClient .name , this );
1210+ }
11791211 nodeAndClient .restart (callback );
1212+ if (activeDisruptionScheme != null ) {
1213+ activeDisruptionScheme .applyToNode (nodeAndClient .name , this );
1214+ }
11801215 }
11811216 }
11821217 }
@@ -1374,6 +1409,7 @@ private synchronized void publishNode(NodeAndClient nodeAndClient) {
13741409 dataDirToClean .addAll (Arrays .asList (nodeEnv .nodeDataLocations ()));
13751410 }
13761411 nodes .put (nodeAndClient .name , nodeAndClient );
1412+ applyDisruptionSchemeToNode (nodeAndClient );
13771413 }
13781414
13791415 public void closeNonSharedNodes (boolean wipeData ) throws IOException {
@@ -1395,6 +1431,33 @@ public boolean hasFilterCache() {
13951431 return hasFilterCache ;
13961432 }
13971433
1434+ public void setDisruptionScheme (ServiceDisruptionScheme scheme ) {
1435+ clearDisruptionScheme ();
1436+ scheme .applyToCluster (this );
1437+ activeDisruptionScheme = scheme ;
1438+ }
1439+
1440+ public void clearDisruptionScheme () {
1441+ if (activeDisruptionScheme != null ) {
1442+ activeDisruptionScheme .removeFromCluster (this );
1443+ }
1444+ activeDisruptionScheme = null ;
1445+ }
1446+
1447+ private void applyDisruptionSchemeToNode (NodeAndClient nodeAndClient ) {
1448+ if (activeDisruptionScheme != null ) {
1449+ assert nodes .containsKey (nodeAndClient .name );
1450+ activeDisruptionScheme .applyToNode (nodeAndClient .name , this );
1451+ }
1452+ }
1453+
1454+ private void removeDistruptionSchemeFromNode (NodeAndClient nodeAndClient ) {
1455+ if (activeDisruptionScheme != null ) {
1456+ assert nodes .containsKey (nodeAndClient .name );
1457+ activeDisruptionScheme .removeFromNode (nodeAndClient .name , this );
1458+ }
1459+ }
1460+
13981461 private synchronized Collection <NodeAndClient > dataNodeAndClients () {
13991462 return Collections2 .filter (nodes .values (), new DataNodePredicate ());
14001463 }
0 commit comments