YARN-1591. Fixed AsyncDispatcher to handle interrupts on shutdown in a sane manner and thus fix failure of TestResourceTrackerService. Contributed by Tsuyoshi Ozawa.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1578628 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6a89e57b8d
commit
81a456e638
|
@ -501,6 +501,10 @@ Release 2.4.0 - UNRELEASED
|
|||
YARN-1206. Fixed AM container log to show on NM web page after application
|
||||
finishes if log-aggregation is disabled. (Rohith Sharmaks via jianhe)
|
||||
|
||||
YARN-1591. Fixed AsyncDispatcher to handle interrupts on shutdown in a sane
|
||||
manner and thus fix failure of TestResourceTrackerService. (Tsuyoshi Ozawa
|
||||
via vinodkv)
|
||||
|
||||
Release 2.3.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -174,12 +174,13 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
|||
} else {
|
||||
throw new Exception("No handler for registered for " + type);
|
||||
}
|
||||
}
|
||||
catch (Throwable t) {
|
||||
} catch (Throwable t) {
|
||||
//TODO Maybe log the state of the queue
|
||||
LOG.fatal("Error in dispatcher thread", t);
|
||||
// If serviceStop is called, we should exit this thread gracefully.
|
||||
if (exitOnDispatchException
|
||||
&& (ShutdownHookManager.get().isShutdownInProgress()) == false) {
|
||||
&& (ShutdownHookManager.get().isShutdownInProgress()) == false
|
||||
&& stopped == false) {
|
||||
LOG.info("Exiting, bbye..");
|
||||
System.exit(-1);
|
||||
}
|
||||
|
|
|
@ -630,7 +630,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|||
}
|
||||
this.eventQueue.put(event);
|
||||
} catch (InterruptedException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
LOG.info("Interrupted. Trying to exit gracefully.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.junit.Assert;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.metrics2.MetricsSystem;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
|
@ -63,7 +65,7 @@ public class TestResourceTrackerService {
|
|||
|
||||
private final static File TEMP_DIR = new File(System.getProperty(
|
||||
"test.build.data", "/tmp"), "decommision");
|
||||
private File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
|
||||
private final File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
|
||||
private MockRM rm;
|
||||
|
||||
/**
|
||||
|
@ -468,7 +470,7 @@ public class TestResourceTrackerService {
|
|||
|
||||
@Test
|
||||
public void testNodeRegistrationWithContainers() throws Exception {
|
||||
MockRM rm = new MockRM();
|
||||
rm = new MockRM();
|
||||
rm.init(new YarnConfiguration());
|
||||
rm.start();
|
||||
RMApp app = rm.submitApp(1024);
|
||||
|
@ -491,7 +493,7 @@ public class TestResourceTrackerService {
|
|||
@Test
|
||||
public void testReconnectNode() throws Exception {
|
||||
final DrainDispatcher dispatcher = new DrainDispatcher();
|
||||
MockRM rm = new MockRM() {
|
||||
rm = new MockRM() {
|
||||
@Override
|
||||
protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
|
||||
return new SchedulerEventDispatcher(this.scheduler) {
|
||||
|
@ -593,9 +595,15 @@ public class TestResourceTrackerService {
|
|||
if (hostFile != null && hostFile.exists()) {
|
||||
hostFile.delete();
|
||||
}
|
||||
|
||||
ClusterMetrics.destroy();
|
||||
if (rm != null) {
|
||||
rm.stop();
|
||||
}
|
||||
|
||||
MetricsSystem ms = DefaultMetricsSystem.instance();
|
||||
if (ms.getSource("ClusterMetrics") != null) {
|
||||
DefaultMetricsSystem.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue