|
@@ -22,7 +22,10 @@ import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.assertNull;
|
|
|
+import static org.junit.Assert.assertSame;
|
|
|
import static org.junit.Assert.fail;
|
|
|
+import static org.mockito.Mockito.spy;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.net.HttpURLConnection;
|
|
@@ -37,14 +40,18 @@ import org.apache.hadoop.ha.ClientBaseWithFixes;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
import org.apache.hadoop.service.Service.STATE;
|
|
|
+import org.apache.hadoop.util.ExitUtil;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.client.api.YarnClient;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServer;
|
|
|
import org.apache.hadoop.yarn.webapp.YarnWebParams;
|
|
|
import org.junit.After;
|
|
@@ -174,7 +181,7 @@ public class TestRMFailover extends ClientBaseWithFixes {
|
|
|
// so it transitions to standby.
|
|
|
ResourceManager rm = cluster.getResourceManager(
|
|
|
cluster.getActiveRMIndex());
|
|
|
- rm.handleTransitionToStandBy();
|
|
|
+ rm.handleTransitionToStandByInNewThread();
|
|
|
int maxWaitingAttempts = 2000;
|
|
|
while (maxWaitingAttempts-- > 0 ) {
|
|
|
if (rm.getRMContext().getHAServiceState() == HAServiceState.STANDBY) {
|
|
@@ -349,4 +356,95 @@ public class TestRMFailover extends ClientBaseWithFixes {
|
|
|
}
|
|
|
return redirectUrl;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Throw {@link RuntimeException} inside a thread of
|
|
|
+ * {@link ResourceManager} with HA enabled and check if the
|
|
|
+ * {@link ResourceManager} is transited to standby state.
|
|
|
+ *
|
|
|
+ * @throws InterruptedException if any
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testUncaughtExceptionHandlerWithHAEnabled()
|
|
|
+ throws InterruptedException {
|
|
|
+ conf.set(YarnConfiguration.RM_CLUSTER_ID, "yarn-test-cluster");
|
|
|
+ conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
|
|
|
+ cluster.init(conf);
|
|
|
+ cluster.start();
|
|
|
+ assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
|
|
|
+
|
|
|
+ ResourceManager resourceManager = cluster.getResourceManager(
|
|
|
+ cluster.getActiveRMIndex());
|
|
|
+
|
|
|
+ final RMCriticalThreadUncaughtExceptionHandler exHandler =
|
|
|
+ new RMCriticalThreadUncaughtExceptionHandler(
|
|
|
+ resourceManager.getRMContext());
|
|
|
+
|
|
|
+ // Create a thread and throw a RTE inside it
|
|
|
+ final RuntimeException rte = new RuntimeException("TestRuntimeException");
|
|
|
+ final Thread testThread = new Thread(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ throw rte;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ testThread.setName("TestThread");
|
|
|
+ testThread.setUncaughtExceptionHandler(exHandler);
|
|
|
+ testThread.start();
|
|
|
+ testThread.join();
|
|
|
+
|
|
|
+ int maxWaitingAttempts = 2000;
|
|
|
+ while (maxWaitingAttempts-- > 0) {
|
|
|
+ if (resourceManager.getRMContext().getHAServiceState()
|
|
|
+ == HAServiceState.STANDBY) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ Thread.sleep(1);
|
|
|
+ }
|
|
|
+ assertFalse("RM didn't transition to Standby ", maxWaitingAttempts < 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Throw {@link RuntimeException} inside a thread of
|
|
|
+ * {@link ResourceManager} with HA disabled and check
|
|
|
+ * {@link RMCriticalThreadUncaughtExceptionHandler} instance.
|
|
|
+ *
|
|
|
+ * Used {@link ExitUtil} class to avoid jvm exit through
|
|
|
+ * {@code System.exit(-1)}.
|
|
|
+ *
|
|
|
+ * @throws InterruptedException if any
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testUncaughtExceptionHandlerWithoutHA()
|
|
|
+ throws InterruptedException {
|
|
|
+ ExitUtil.disableSystemExit();
|
|
|
+
|
|
|
+ // Create a MockRM and start it
|
|
|
+ ResourceManager resourceManager = new MockRM();
|
|
|
+ ((AsyncDispatcher) resourceManager.getRMContext().getDispatcher()).start();
|
|
|
+ resourceManager.getRMContext().getStateStore().start();
|
|
|
+ resourceManager.getRMContext().getContainerTokenSecretManager().
|
|
|
+ rollMasterKey();
|
|
|
+
|
|
|
+ final RMCriticalThreadUncaughtExceptionHandler exHandler =
|
|
|
+ new RMCriticalThreadUncaughtExceptionHandler(
|
|
|
+ resourceManager.getRMContext());
|
|
|
+ final RMCriticalThreadUncaughtExceptionHandler spyRTEHandler =
|
|
|
+ spy(exHandler);
|
|
|
+
|
|
|
+ // Create a thread and throw a RTE inside it
|
|
|
+ final RuntimeException rte = new RuntimeException("TestRuntimeException");
|
|
|
+ final Thread testThread = new Thread(new Runnable() {
|
|
|
+ @Override public void run() {
|
|
|
+ throw rte;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ testThread.setName("TestThread");
|
|
|
+ testThread.setUncaughtExceptionHandler(spyRTEHandler);
|
|
|
+ assertSame(spyRTEHandler, testThread.getUncaughtExceptionHandler());
|
|
|
+ testThread.start();
|
|
|
+ testThread.join();
|
|
|
+
|
|
|
+ verify(spyRTEHandler).uncaughtException(testThread, rte);
|
|
|
+ }
|
|
|
}
|