|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.yarn.client.api.async.impl;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Matchers.anyBoolean;
|
|
|
import static org.mockito.Mockito.doNothing;
|
|
|
import static org.mockito.Mockito.doThrow;
|
|
|
import static org.mockito.Mockito.mock;
|
|
@@ -27,6 +28,8 @@ import static org.mockito.Mockito.when;
|
|
|
import java.io.IOException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.Collections;
|
|
|
+import java.util.EnumSet;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
@@ -37,6 +40,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicIntegerArray;
|
|
|
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
+import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.junit.Assert;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -69,6 +73,22 @@ public class TestNMClientAsync {
|
|
|
private NodeId nodeId;
|
|
|
private Token containerToken;
|
|
|
|
|
|
+ enum OpsToTest {
|
|
|
+ START, QUERY, STOP, INCR, REINIT, RESTART, ROLLBACK, COMMIT
|
|
|
+ }
|
|
|
+
|
|
|
+ final static class TestData {
|
|
|
+ AtomicInteger success = new AtomicInteger(0);
|
|
|
+ AtomicInteger failure = new AtomicInteger(0);
|
|
|
+ final AtomicIntegerArray successArray;
|
|
|
+ final AtomicIntegerArray failureArray;
|
|
|
+
|
|
|
+ private TestData(int expectedSuccess, int expectedFailure) {
|
|
|
+ this.successArray = new AtomicIntegerArray(expectedSuccess);
|
|
|
+ this.failureArray = new AtomicIntegerArray(expectedFailure);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@After
|
|
|
public void teardown() {
|
|
|
ServiceOperations.stop(asyncClient);
|
|
@@ -194,25 +214,7 @@ public class TestNMClientAsync {
|
|
|
private int expectedSuccess;
|
|
|
private int expectedFailure;
|
|
|
|
|
|
- private AtomicInteger actualStartSuccess = new AtomicInteger(0);
|
|
|
- private AtomicInteger actualStartFailure = new AtomicInteger(0);
|
|
|
- private AtomicInteger actualQuerySuccess = new AtomicInteger(0);
|
|
|
- private AtomicInteger actualQueryFailure = new AtomicInteger(0);
|
|
|
- private AtomicInteger actualStopSuccess = new AtomicInteger(0);
|
|
|
- private AtomicInteger actualStopFailure = new AtomicInteger(0);
|
|
|
- private AtomicInteger actualIncreaseResourceSuccess =
|
|
|
- new AtomicInteger(0);
|
|
|
- private AtomicInteger actualIncreaseResourceFailure =
|
|
|
- new AtomicInteger(0);
|
|
|
-
|
|
|
- private AtomicIntegerArray actualStartSuccessArray;
|
|
|
- private AtomicIntegerArray actualStartFailureArray;
|
|
|
- private AtomicIntegerArray actualQuerySuccessArray;
|
|
|
- private AtomicIntegerArray actualQueryFailureArray;
|
|
|
- private AtomicIntegerArray actualStopSuccessArray;
|
|
|
- private AtomicIntegerArray actualStopFailureArray;
|
|
|
- private AtomicIntegerArray actualIncreaseResourceSuccessArray;
|
|
|
- private AtomicIntegerArray actualIncreaseResourceFailureArray;
|
|
|
+ private final Map<OpsToTest, TestData> testMap = new HashMap<>();
|
|
|
|
|
|
private Set<String> errorMsgs =
|
|
|
Collections.synchronizedSet(new HashSet<String>());
|
|
@@ -221,16 +223,9 @@ public class TestNMClientAsync {
|
|
|
this.expectedSuccess = expectedSuccess;
|
|
|
this.expectedFailure = expectedFailure;
|
|
|
|
|
|
- actualStartSuccessArray = new AtomicIntegerArray(expectedSuccess);
|
|
|
- actualStartFailureArray = new AtomicIntegerArray(expectedFailure);
|
|
|
- actualQuerySuccessArray = new AtomicIntegerArray(expectedSuccess);
|
|
|
- actualQueryFailureArray = new AtomicIntegerArray(expectedFailure);
|
|
|
- actualStopSuccessArray = new AtomicIntegerArray(expectedSuccess);
|
|
|
- actualStopFailureArray = new AtomicIntegerArray(expectedFailure);
|
|
|
- actualIncreaseResourceSuccessArray =
|
|
|
- new AtomicIntegerArray(expectedSuccess);
|
|
|
- actualIncreaseResourceFailureArray =
|
|
|
- new AtomicIntegerArray(expectedFailure);
|
|
|
+ for (OpsToTest op : OpsToTest.values()) {
|
|
|
+ testMap.put(op, new TestData(expectedSuccess, expectedFailure));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@SuppressWarnings("deprecation")
|
|
@@ -243,8 +238,9 @@ public class TestNMClientAsync {
|
|
|
" should throw the exception onContainerStarted");
|
|
|
return;
|
|
|
}
|
|
|
- actualStartSuccess.addAndGet(1);
|
|
|
- actualStartSuccessArray.set(containerId.getId(), 1);
|
|
|
+ TestData td = testMap.get(OpsToTest.START);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
|
|
|
// move on to the following success tests
|
|
|
asyncClient.getContainerStatusAsync(containerId, nodeId);
|
|
@@ -254,7 +250,28 @@ public class TestNMClientAsync {
|
|
|
// containerId
|
|
|
Container container = Container.newInstance(
|
|
|
containerId, nodeId, null, null, null, containerToken);
|
|
|
- asyncClient.increaseContainerResourceAsync(container);
|
|
|
+ int t = containerId.getId() % 5;
|
|
|
+ switch (t) {
|
|
|
+ case 0:
|
|
|
+ asyncClient.increaseContainerResourceAsync(container);
|
|
|
+ break;
|
|
|
+ case 1:
|
|
|
+ asyncClient.reInitializeContainerAsync(containerId,
|
|
|
+ recordFactory.newRecordInstance(ContainerLaunchContext.class),
|
|
|
+ true);
|
|
|
+ break;
|
|
|
+ case 2:
|
|
|
+ asyncClient.restartContainerAsync(containerId);
|
|
|
+ break;
|
|
|
+ case 3:
|
|
|
+ asyncClient.rollbackLastReInitializationAsync(containerId);
|
|
|
+ break;
|
|
|
+ case 4:
|
|
|
+ asyncClient.commitLastReInitializationAsync(containerId);
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ break;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
// Shouldn't crash the test thread
|
|
@@ -270,8 +287,9 @@ public class TestNMClientAsync {
|
|
|
" should throw the exception onContainerStatusReceived");
|
|
|
return;
|
|
|
}
|
|
|
- actualQuerySuccess.addAndGet(1);
|
|
|
- actualQuerySuccessArray.set(containerId.getId(), 1);
|
|
|
+ TestData td = testMap.get(OpsToTest.QUERY);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
// move on to the following success tests
|
|
|
// make sure we pass in the container with the same
|
|
|
// containerId
|
|
@@ -292,8 +310,78 @@ public class TestNMClientAsync {
|
|
|
" should throw the exception onContainerResourceIncreased");
|
|
|
return;
|
|
|
}
|
|
|
- actualIncreaseResourceSuccess.addAndGet(1);
|
|
|
- actualIncreaseResourceSuccessArray.set(containerId.getId(), 1);
|
|
|
+ TestData td = testMap.get(OpsToTest.INCR);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
+ // move on to the following success tests
|
|
|
+ asyncClient.reInitializeContainerAsync(containerId,
|
|
|
+ Records.newRecord(ContainerLaunchContext.class), true);
|
|
|
+ // throw a fake user exception, and shouldn't crash the test
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onContainerReInitialize(ContainerId containerId) {
|
|
|
+ if (containerId.getId() >= expectedSuccess) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " should throw the exception onContainerReInitialize");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.REINIT);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
+ // move on to the following success tests
|
|
|
+ asyncClient.restartContainerAsync(containerId);
|
|
|
+ // throw a fake user exception, and shouldn't crash the test
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onContainerRestart(ContainerId containerId) {
|
|
|
+ if (containerId.getId() >= expectedSuccess) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " should throw the exception onContainerReInitialize");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.RESTART);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
+ // move on to the following success tests
|
|
|
+ asyncClient.rollbackLastReInitializationAsync(containerId);
|
|
|
+ // throw a fake user exception, and shouldn't crash the test
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onRollbackLastReInitialization(ContainerId containerId) {
|
|
|
+ if (containerId.getId() >= expectedSuccess) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " should throw the exception onContainerReInitialize");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.ROLLBACK);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
+ // move on to the following success tests
|
|
|
+ asyncClient.commitLastReInitializationAsync(containerId);
|
|
|
+ // throw a fake user exception, and shouldn't crash the test
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onCommitLastReInitialization(ContainerId containerId) {
|
|
|
+ if (containerId.getId() >= expectedSuccess) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " should throw the exception onContainerReInitialize");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.COMMIT);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
// move on to the following success tests
|
|
|
asyncClient.stopContainerAsync(containerId, nodeId);
|
|
|
// throw a fake user exception, and shouldn't crash the test
|
|
@@ -308,8 +396,9 @@ public class TestNMClientAsync {
|
|
|
" should throw the exception onContainerStopped");
|
|
|
return;
|
|
|
}
|
|
|
- actualStopSuccess.addAndGet(1);
|
|
|
- actualStopSuccessArray.set(containerId.getId(), 1);
|
|
|
+ TestData td = testMap.get(OpsToTest.STOP);
|
|
|
+ td.success.addAndGet(1);
|
|
|
+ td.successArray.set(containerId.getId(), 1);
|
|
|
|
|
|
// Shouldn't crash the test thread
|
|
|
throw new RuntimeException("Ignorable Exception");
|
|
@@ -330,8 +419,9 @@ public class TestNMClientAsync {
|
|
|
" shouldn't throw the exception onStartContainerError");
|
|
|
return;
|
|
|
}
|
|
|
- actualStartFailure.addAndGet(1);
|
|
|
- actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1);
|
|
|
+ TestData td = testMap.get(OpsToTest.START);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(containerId.getId() - expectedSuccess, 1);
|
|
|
// move on to the following failure tests
|
|
|
asyncClient.getContainerStatusAsync(containerId, nodeId);
|
|
|
|
|
@@ -348,8 +438,9 @@ public class TestNMClientAsync {
|
|
|
" shouldn't throw the exception onIncreaseContainerResourceError");
|
|
|
return;
|
|
|
}
|
|
|
- actualIncreaseResourceFailure.addAndGet(1);
|
|
|
- actualIncreaseResourceFailureArray.set(
|
|
|
+ TestData td = testMap.get(OpsToTest.INCR);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
// increase container resource error should NOT change the
|
|
|
// the container status to FAILED
|
|
@@ -359,6 +450,102 @@ public class TestNMClientAsync {
|
|
|
throw new RuntimeException("Ignorable Exception");
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onContainerReInitializeError(ContainerId containerId,
|
|
|
+ Throwable t) {
|
|
|
+ if (containerId.getId() < expectedSuccess + expectedFailure) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " shouldn't throw the exception onContainerReInitializeError");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.REINIT);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+
|
|
|
+ // increment the stop counters here.. since the container will fail
|
|
|
+ td = testMap.get(OpsToTest.STOP);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+ // reInit container changes the container status to FAILED
|
|
|
+ // Shouldn't crash the test thread
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onContainerRestartError(ContainerId containerId, Throwable t) {
|
|
|
+ if (containerId.getId() < expectedSuccess + expectedFailure) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " shouldn't throw the exception onContainerRestartError");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.RESTART);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+
|
|
|
+ // increment the stop counters here.. since the container will fail
|
|
|
+ td = testMap.get(OpsToTest.STOP);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+ // restart container changes the container status to FAILED
|
|
|
+ // Shouldn't crash the test thread
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onRollbackLastReInitializationError(ContainerId containerId,
|
|
|
+ Throwable t) {
|
|
|
+ if (containerId.getId() < expectedSuccess + expectedFailure) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " shouldn't throw the exception" +
|
|
|
+ " onRollbackLastReInitializationError");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.ROLLBACK);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+
|
|
|
+ // increment the stop counters here.. since the container will fail
|
|
|
+ td = testMap.get(OpsToTest.STOP);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+ // rollback container changes the container status to FAILED
|
|
|
+ // Shouldn't crash the test thread
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ @Override
|
|
|
+ public void onCommitLastReInitializationError(ContainerId containerId,
|
|
|
+ Throwable t) {
|
|
|
+ if (containerId.getId() < expectedSuccess + expectedFailure) {
|
|
|
+ errorMsgs.add("Container " + containerId +
|
|
|
+ " shouldn't throw the exception onCommitLastReInitializationError");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ TestData td = testMap.get(OpsToTest.COMMIT);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+
|
|
|
+ // increment the stop counters here.. since the container will fail
|
|
|
+ td = testMap.get(OpsToTest.STOP);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
+ containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
+ // commit container changes the container status to FAILED
|
|
|
+ // Shouldn't crash the test thread
|
|
|
+ throw new RuntimeException("Ignorable Exception");
|
|
|
+ }
|
|
|
+
|
|
|
@SuppressWarnings("deprecation")
|
|
|
@Override
|
|
|
public void onStopContainerError(ContainerId containerId, Throwable t) {
|
|
@@ -371,9 +558,9 @@ public class TestNMClientAsync {
|
|
|
" shouldn't throw the exception onStopContainerError");
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
- actualStopFailure.addAndGet(1);
|
|
|
- actualStopFailureArray.set(
|
|
|
+ TestData td = testMap.get(OpsToTest.STOP);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(
|
|
|
containerId.getId() - expectedSuccess - expectedFailure, 1);
|
|
|
|
|
|
// Shouldn't crash the test thread
|
|
@@ -393,8 +580,9 @@ public class TestNMClientAsync {
|
|
|
" shouldn't throw the exception onGetContainerStatusError");
|
|
|
return;
|
|
|
}
|
|
|
- actualQueryFailure.addAndGet(1);
|
|
|
- actualQueryFailureArray.set(containerId.getId() - expectedSuccess, 1);
|
|
|
+ TestData td = testMap.get(OpsToTest.QUERY);
|
|
|
+ td.failure.addAndGet(1);
|
|
|
+ td.failureArray.set(containerId.getId() - expectedSuccess, 1);
|
|
|
|
|
|
// Shouldn't crash the test thread
|
|
|
throw new RuntimeException("Ignorable Exception");
|
|
@@ -402,44 +590,67 @@ public class TestNMClientAsync {
|
|
|
|
|
|
public boolean isAllSuccessCallsExecuted() {
|
|
|
boolean isAllSuccessCallsExecuted =
|
|
|
- actualStartSuccess.get() == expectedSuccess &&
|
|
|
- actualQuerySuccess.get() == expectedSuccess &&
|
|
|
- actualIncreaseResourceSuccess.get() == expectedSuccess &&
|
|
|
- actualStopSuccess.get() == expectedSuccess;
|
|
|
+ testMap.get(OpsToTest.START).success.get() == expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.QUERY).success.get() == expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.INCR).success.get() == expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.REINIT).success.get() == expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.RESTART).success.get() == expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.ROLLBACK).success.get() ==
|
|
|
+ expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.COMMIT).success.get() == expectedSuccess &&
|
|
|
+ testMap.get(OpsToTest.STOP).success.get() == expectedSuccess;
|
|
|
if (isAllSuccessCallsExecuted) {
|
|
|
- assertAtomicIntegerArray(actualStartSuccessArray);
|
|
|
- assertAtomicIntegerArray(actualQuerySuccessArray);
|
|
|
- assertAtomicIntegerArray(actualIncreaseResourceSuccessArray);
|
|
|
- assertAtomicIntegerArray(actualStopSuccessArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.START).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.QUERY).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.INCR).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.REINIT).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.RESTART).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.ROLLBACK).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.COMMIT).successArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.STOP).successArray);
|
|
|
}
|
|
|
return isAllSuccessCallsExecuted;
|
|
|
}
|
|
|
|
|
|
public boolean isStartAndQueryFailureCallsExecuted() {
|
|
|
boolean isStartAndQueryFailureCallsExecuted =
|
|
|
- actualStartFailure.get() == expectedFailure &&
|
|
|
- actualQueryFailure.get() == expectedFailure;
|
|
|
+ testMap.get(OpsToTest.START).failure.get() == expectedFailure &&
|
|
|
+ testMap.get(OpsToTest.QUERY).failure.get() == expectedFailure;
|
|
|
if (isStartAndQueryFailureCallsExecuted) {
|
|
|
- assertAtomicIntegerArray(actualStartFailureArray);
|
|
|
- assertAtomicIntegerArray(actualQueryFailureArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.START).failureArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.QUERY).failureArray);
|
|
|
}
|
|
|
return isStartAndQueryFailureCallsExecuted;
|
|
|
}
|
|
|
|
|
|
public boolean isIncreaseResourceFailureCallsExecuted() {
|
|
|
boolean isIncreaseResourceFailureCallsExecuted =
|
|
|
- actualIncreaseResourceFailure.get() == expectedFailure;
|
|
|
+ testMap.get(OpsToTest.INCR).failure.get()
|
|
|
+ + testMap.get(OpsToTest.REINIT).failure.get()
|
|
|
+ + testMap.get(OpsToTest.RESTART).failure.get()
|
|
|
+ + testMap.get(OpsToTest.ROLLBACK).failure.get()
|
|
|
+ + testMap.get(OpsToTest.COMMIT).failure.get()
|
|
|
+ == expectedFailure;
|
|
|
if (isIncreaseResourceFailureCallsExecuted) {
|
|
|
- assertAtomicIntegerArray(actualIncreaseResourceFailureArray);
|
|
|
+ AtomicIntegerArray testArray =
|
|
|
+ new AtomicIntegerArray(
|
|
|
+ testMap.get(OpsToTest.INCR).failureArray.length());
|
|
|
+ for (int i = 0; i < testArray.length(); i++) {
|
|
|
+ for (OpsToTest op : EnumSet.of(OpsToTest.REINIT, OpsToTest.RESTART,
|
|
|
+ OpsToTest.ROLLBACK, OpsToTest.COMMIT, OpsToTest.INCR)) {
|
|
|
+ testArray.addAndGet(i, testMap.get(op).failureArray.get(i));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ assertAtomicIntegerArray(testArray);
|
|
|
}
|
|
|
return isIncreaseResourceFailureCallsExecuted;
|
|
|
}
|
|
|
|
|
|
public boolean isStopFailureCallsExecuted() {
|
|
|
boolean isStopFailureCallsExecuted =
|
|
|
- actualStopFailure.get() == expectedFailure;
|
|
|
+ testMap.get(OpsToTest.STOP).failure.get() == expectedFailure;
|
|
|
if (isStopFailureCallsExecuted) {
|
|
|
- assertAtomicIntegerArray(actualStopFailureArray);
|
|
|
+ assertAtomicIntegerArray(testMap.get(OpsToTest.STOP).failureArray);
|
|
|
}
|
|
|
return isStopFailureCallsExecuted;
|
|
|
}
|
|
@@ -464,6 +675,14 @@ public class TestNMClientAsync {
|
|
|
recordFactory.newRecordInstance(ContainerStatus.class));
|
|
|
doNothing().when(client).increaseContainerResource(
|
|
|
any(Container.class));
|
|
|
+ doNothing().when(client).reInitializeContainer(
|
|
|
+ any(ContainerId.class), any(ContainerLaunchContext.class),
|
|
|
+ anyBoolean());
|
|
|
+ doNothing().when(client).restartContainer(any(ContainerId.class));
|
|
|
+ doNothing().when(client).rollbackLastReInitialization(
|
|
|
+ any(ContainerId.class));
|
|
|
+ doNothing().when(client).commitLastReInitialization(
|
|
|
+ any(ContainerId.class));
|
|
|
doNothing().when(client).stopContainer(any(ContainerId.class),
|
|
|
any(NodeId.class));
|
|
|
break;
|
|
@@ -485,9 +704,23 @@ public class TestNMClientAsync {
|
|
|
recordFactory.newRecordInstance(ContainerStatus.class));
|
|
|
doThrow(RPCUtil.getRemoteException("Increase Resource Exception"))
|
|
|
.when(client).increaseContainerResource(any(Container.class));
|
|
|
+ doThrow(RPCUtil.getRemoteException("ReInitialize Exception"))
|
|
|
+ .when(client).reInitializeContainer(
|
|
|
+ any(ContainerId.class), any(ContainerLaunchContext.class),
|
|
|
+ anyBoolean());
|
|
|
+ doThrow(RPCUtil.getRemoteException("Restart Exception"))
|
|
|
+ .when(client).restartContainer(any(ContainerId.class));
|
|
|
+ doThrow(RPCUtil.getRemoteException("Rollback upgrade Exception"))
|
|
|
+ .when(client).rollbackLastReInitialization(
|
|
|
+ any(ContainerId.class));
|
|
|
+ doThrow(RPCUtil.getRemoteException("Commit upgrade Exception"))
|
|
|
+ .when(client).commitLastReInitialization(
|
|
|
+ any(ContainerId.class));
|
|
|
doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
|
|
|
.stopContainer(any(ContainerId.class), any(NodeId.class));
|
|
|
}
|
|
|
+ when(client.getNodeIdOfStartedContainer(any(ContainerId.class)))
|
|
|
+ .thenReturn(NodeId.newInstance("localhost", 0));
|
|
|
return client;
|
|
|
}
|
|
|
|